diff --git a/.github/workflows/velox_backend_x86.yml b/.github/workflows/velox_backend_x86.yml index 5ef70a79bc0f..51f71e829c5e 100644 --- a/.github/workflows/velox_backend_x86.yml +++ b/.github/workflows/velox_backend_x86.yml @@ -107,7 +107,7 @@ jobs: fail-fast: false matrix: os: [ "ubuntu:20.04", "ubuntu:22.04" ] - spark: [ "spark-3.2", "spark-3.3", "spark-3.4", "spark-3.5", "spark-4.0" ] + spark: [ "spark-3.2", "spark-3.3", "spark-3.4", "spark-3.5", "spark-4.0", "spark-4.1" ] java: [ "java-8", "java-11", "java-17", "java-21" ] # Spark supports JDK17 since 3.3. exclude: @@ -141,6 +141,10 @@ jobs: java: java-8 - spark: spark-4.0 java: java-11 + - spark: spark-4.1 + java: java-8 + - spark: spark-4.1 + java: java-11 runs-on: ubuntu-22.04 container: ${{ matrix.os }} @@ -182,11 +186,14 @@ jobs: cd $GITHUB_WORKSPACE/ export JAVA_HOME=/usr/lib/jvm/${{ matrix.java }}-openjdk-amd64 echo "JAVA_HOME: $JAVA_HOME" - if [ "${{ matrix.spark }}" = "spark-4.0" ]; then - $MVN_CMD clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pscala-2.13 -Pbackends-velox -DskipTests - else - $MVN_CMD clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pbackends-velox -DskipTests - fi + case "${{ matrix.spark }}" in + spark-4.0|spark-4.1) + $MVN_CMD clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pscala-2.13 -Pbackends-velox -DskipTests + ;; + *) + $MVN_CMD clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pbackends-velox -DskipTests + ;; + esac cd $GITHUB_WORKSPACE/tools/gluten-it $GITHUB_WORKSPACE/$MVN_CMD clean install -P${{ matrix.spark }} -P${{ matrix.java }} GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ @@ -200,7 +207,7 @@ jobs: fail-fast: false matrix: os: [ "centos:8" ] - spark: [ "spark-3.2", "spark-3.3", "spark-3.4", "spark-3.5", "spark-4.0" ] + spark: [ "spark-3.2", "spark-3.3", "spark-3.4", "spark-3.5", "spark-4.0", "spark-4.1" ] java: [ "java-8", "java-11", "java-17" ] # Spark supports JDK17 since 3.3. exclude: @@ -220,6 +227,10 @@ jobs: java: java-8 - spark: spark-4.0 java: java-11 + - spark: spark-4.1 + java: java-8 + - spark: spark-4.1 + java: java-11 runs-on: ubuntu-22.04 container: ${{ matrix.os }} @@ -263,11 +274,14 @@ jobs: run: | echo "JAVA_HOME: $JAVA_HOME" cd $GITHUB_WORKSPACE/ - if [ "${{ matrix.spark }}" = "spark-4.0" ]; then - $MVN_CMD clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pscala-2.13 -Pbackends-velox -DskipTests - else - $MVN_CMD clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pbackends-velox -DskipTests - fi + case "${{ matrix.spark }}" in + spark-4.0|spark-4.1) + $MVN_CMD clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pscala-2.13 -Pbackends-velox -DskipTests + ;; + *) + $MVN_CMD clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pbackends-velox -DskipTests + ;; + esac cd $GITHUB_WORKSPACE/tools/gluten-it $GITHUB_WORKSPACE/build/mvn clean install -P${{ matrix.spark }} -P${{ matrix.java }} - name: Run TPC-H / TPC-DS @@ -1521,7 +1535,7 @@ jobs: export PATH=$JAVA_HOME/bin:$PATH 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/" \ + -Pspark-ut -DargLine="-Dspark.test.home=/opt/shims/spark41/spark_home/ -Dspark.sql.unionOutputPartitioning=false" \ -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.EnhancedFeaturesTest,org.apache.gluten.tags.SkipTest - name: Upload test report if: always() @@ -1570,7 +1584,7 @@ jobs: export PATH=$JAVA_HOME/bin:$PATH 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/" \ + -DargLine="-Dspark.test.home=/opt/shims/spark41/spark_home/ -Dspark.sql.unionOutputPartitioning=false" \ -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest - name: Upload test report if: always() diff --git a/backends-velox/src/main/scala/org/apache/gluten/expression/aggregate/VeloxBloomFilterAggregate.scala b/backends-velox/src/main/scala/org/apache/gluten/expression/aggregate/VeloxBloomFilterAggregate.scala index 976abb9e21fb..a3d6f738a2b5 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/expression/aggregate/VeloxBloomFilterAggregate.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/expression/aggregate/VeloxBloomFilterAggregate.scala @@ -25,10 +25,14 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate import org.apache.spark.sql.catalyst.trees.TernaryLike import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ import org.apache.spark.sql.types.DataType import org.apache.spark.task.TaskResources +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.sketch.BloomFilter +import java.io.Serializable + /** * Velox's bloom-filter implementation uses different algorithms internally comparing to vanilla * Spark so produces different intermediate aggregate data. Thus we use different filter function / @@ -61,6 +65,15 @@ case class VeloxBloomFilterAggregate( .toLong ) + // Mark as lazy so that `updater` is not evaluated during tree transformation. + private lazy val updater: BloomFilterUpdater = child.dataType match { + case LongType => LongUpdater + case IntegerType => IntUpdater + case ShortType => ShortUpdater + case ByteType => ByteUpdater + case _: StringType => BinaryUpdater + } + override def first: Expression = child override def second: Expression = estimatedNumItemsExpression @@ -97,7 +110,7 @@ case class VeloxBloomFilterAggregate( if (value == null) { return buffer } - buffer.putLong(value.asInstanceOf[Long]) + updater.update(buffer, value) buffer } @@ -128,3 +141,33 @@ case class VeloxBloomFilterAggregate( copy(inputAggBufferOffset = newOffset) } + +// see https://github.com/apache/spark/pull/42414 +private trait BloomFilterUpdater { + def update(bf: BloomFilter, v: Any): Boolean +} + +private object LongUpdater extends BloomFilterUpdater with Serializable { + override def update(bf: BloomFilter, v: Any): Boolean = + bf.putLong(v.asInstanceOf[Long]) +} + +private object IntUpdater extends BloomFilterUpdater with Serializable { + override def update(bf: BloomFilter, v: Any): Boolean = + bf.putLong(v.asInstanceOf[Int]) +} + +private object ShortUpdater extends BloomFilterUpdater with Serializable { + override def update(bf: BloomFilter, v: Any): Boolean = + bf.putLong(v.asInstanceOf[Short]) +} + +private object ByteUpdater extends BloomFilterUpdater with Serializable { + override def update(bf: BloomFilter, v: Any): Boolean = + bf.putLong(v.asInstanceOf[Byte]) +} + +private object BinaryUpdater extends BloomFilterUpdater with Serializable { + override def update(bf: BloomFilter, v: Any): Boolean = + bf.putBinary(v.asInstanceOf[UTF8String].getBytes) +} diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala index a04e7d68fba6..c05eb4a2fa68 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala @@ -115,24 +115,24 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with Logging { conf: SQLConf): RDD[CachedBatch] = { val localSchema = toStructType(schema) if (!validateSchema(localSchema)) { - // we can not use columnar cache here, as the `RowToColumnar` does not support this schema - return rowBasedCachedBatchSerializer.convertInternalRowToCachedBatch( + // we cannot use columnar cache here, as the `RowToColumnar` does not support this schema + rowBasedCachedBatchSerializer.convertInternalRowToCachedBatch( input, schema, storageLevel, conf) + } else { + val numRows = conf.columnBatchSize + val rddColumnarBatch = input.mapPartitions { + it => + RowToVeloxColumnarExec.toColumnarBatchIterator( + it, + localSchema, + numRows, + VeloxConfig.get.veloxPreferredBatchBytes) + } + convertColumnarBatchToCachedBatch(rddColumnarBatch, schema, storageLevel, conf) } - - val numRows = conf.columnBatchSize - val rddColumnarBatch = input.mapPartitions { - it => - RowToVeloxColumnarExec.toColumnarBatchIterator( - it, - localSchema, - numRows, - VeloxConfig.get.veloxPreferredBatchBytes) - } - convertColumnarBatchToCachedBatch(rddColumnarBatch, schema, storageLevel, conf) } override def convertCachedBatchToInternalRow( @@ -141,18 +141,18 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with Logging { selectedAttributes: Seq[Attribute], conf: SQLConf): RDD[InternalRow] = { if (!validateSchema(cacheAttributes)) { - // if we do not support this schema that means we are using row-based serializer, + // if we do not support this schema, that means we are using row-based serializer, // see `convertInternalRowToCachedBatch`, so fallback to vanilla Spark serializer - return rowBasedCachedBatchSerializer.convertCachedBatchToInternalRow( + rowBasedCachedBatchSerializer.convertCachedBatchToInternalRow( input, cacheAttributes, selectedAttributes, conf) + } else { + val rddColumnarBatch = + convertCachedBatchToColumnarBatch(input, cacheAttributes, selectedAttributes, conf) + rddColumnarBatch.mapPartitions(it => VeloxColumnarToRowExec.toRowIterator(it)) } - - val rddColumnarBatch = - convertCachedBatchToColumnarBatch(input, cacheAttributes, selectedAttributes, conf) - rddColumnarBatch.mapPartitions(it => VeloxColumnarToRowExec.toRowIterator(it)) } override def convertColumnarBatchToCachedBatch( @@ -190,58 +190,68 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with Logging { cacheAttributes: Seq[Attribute], selectedAttributes: Seq[Attribute], conf: SQLConf): RDD[ColumnarBatch] = { - // Find the ordinals and data types of the requested columns. - val requestedColumnIndices = selectedAttributes.map { - a => cacheAttributes.map(_.exprId).indexOf(a.exprId) - } - val shouldSelectAttributes = cacheAttributes != selectedAttributes - val localSchema = toStructType(cacheAttributes) - val timezoneId = SQLConf.get.sessionLocalTimeZone - input.mapPartitions { - it => - val runtime = Runtimes.contextInstance( - BackendsApiManager.getBackendName, - "ColumnarCachedBatchSerializer#read") - val jniWrapper = ColumnarBatchSerializerJniWrapper - .create(runtime) - val schema = SparkArrowUtil.toArrowSchema(localSchema, timezoneId) - val arrowAlloc = ArrowBufferAllocators.contextInstance() - val cSchema = ArrowSchema.allocateNew(arrowAlloc) - ArrowAbiUtil.exportSchema(arrowAlloc, schema, cSchema) - val deserializerHandle = jniWrapper - .init(cSchema.memoryAddress()) - cSchema.close() - - Iterators - .wrap(new Iterator[ColumnarBatch] { - override def hasNext: Boolean = it.hasNext - - override def next(): ColumnarBatch = { - val cachedBatch = it.next().asInstanceOf[CachedColumnarBatch] - val batchHandle = - jniWrapper - .deserialize(deserializerHandle, cachedBatch.bytes) - val batch = ColumnarBatches.create(batchHandle) - if (shouldSelectAttributes) { - try { - ColumnarBatches.select( - BackendsApiManager.getBackendName, - batch, - requestedColumnIndices.toArray) - } finally { - batch.close() + if (!validateSchema(cacheAttributes)) { + // if we do not support this schema, that means we are using row-based serializer, + // see `convertInternalRowToCachedBatch`, so fallback to vanilla Spark serializer + rowBasedCachedBatchSerializer.convertCachedBatchToColumnarBatch( + input, + cacheAttributes, + selectedAttributes, + conf) + } else { + // Find the ordinals and data types of the requested columns. + val requestedColumnIndices = selectedAttributes.map { + a => cacheAttributes.map(_.exprId).indexOf(a.exprId) + } + val shouldSelectAttributes = cacheAttributes != selectedAttributes + val localSchema = toStructType(cacheAttributes) + val timezoneId = SQLConf.get.sessionLocalTimeZone + input.mapPartitions { + it => + val runtime = Runtimes.contextInstance( + BackendsApiManager.getBackendName, + "ColumnarCachedBatchSerializer#read") + val jniWrapper = ColumnarBatchSerializerJniWrapper + .create(runtime) + val schema = SparkArrowUtil.toArrowSchema(localSchema, timezoneId) + val arrowAlloc = ArrowBufferAllocators.contextInstance() + val cSchema = ArrowSchema.allocateNew(arrowAlloc) + ArrowAbiUtil.exportSchema(arrowAlloc, schema, cSchema) + val deserializerHandle = jniWrapper + .init(cSchema.memoryAddress()) + cSchema.close() + + Iterators + .wrap(new Iterator[ColumnarBatch] { + override def hasNext: Boolean = it.hasNext + + override def next(): ColumnarBatch = { + val cachedBatch = it.next().asInstanceOf[CachedColumnarBatch] + val batchHandle = + jniWrapper + .deserialize(deserializerHandle, cachedBatch.bytes) + val batch = ColumnarBatches.create(batchHandle) + if (shouldSelectAttributes) { + try { + ColumnarBatches.select( + BackendsApiManager.getBackendName, + batch, + requestedColumnIndices.toArray) + } finally { + batch.close() + } + } else { + batch } - } else { - batch } + }) + .protectInvocationFlow() + .recycleIterator { + jniWrapper.close(deserializerHandle) } - }) - .protectInvocationFlow() - .recycleIterator { - jniWrapper.close(deserializerHandle) - } - .recyclePayload(_.close()) - .create() + .recyclePayload(_.close()) + .create() + } } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/expression/ExpressionConverter.scala b/gluten-substrait/src/main/scala/org/apache/gluten/expression/ExpressionConverter.scala index 418de8578f5c..a810a4ef1dbd 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/expression/ExpressionConverter.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/expression/ExpressionConverter.scala @@ -831,7 +831,14 @@ object ExpressionConverter extends SQLConfHelper with Logging { case t: TransformKeys => // default is `EXCEPTION` val mapKeyDedupPolicy = SQLConf.get.getConf(SQLConf.MAP_KEY_DEDUP_POLICY) - if (mapKeyDedupPolicy == SQLConf.MapKeyDedupPolicy.LAST_WIN.toString) { + + // Calling `.toString` on both sides ensures compatibility across all Spark versions. + // Starting from Spark 4.1, `SQLConf.get.getConf(SQLConf.MAP_KEY_DEDUP_POLICY)` returns + // an enum instead of a String. Without `.toString`, the comparison + // `mapKeyDedupPolicy == SQLConf.MapKeyDedupPolicy.LAST_WIN.toString` would silently fail + // in tests, producing only a "Comparing unrelated types" warning in IntelliJ IDEA, + // but no compile-time error. + if (mapKeyDedupPolicy.toString == SQLConf.MapKeyDedupPolicy.LAST_WIN.toString) { // TODO: Remove after fix ready for // https://github.com/facebookincubator/velox/issues/10219 throw new GlutenNotSupportException( diff --git a/gluten-ut/pom.xml b/gluten-ut/pom.xml index ec0158c41e82..58b4e6d65835 100644 --- a/gluten-ut/pom.xml +++ b/gluten-ut/pom.xml @@ -230,5 +230,11 @@ spark40 + + spark-4.1 + + spark41 + + diff --git a/gluten-ut/spark41/pom.xml b/gluten-ut/spark41/pom.xml new file mode 100644 index 000000000000..838649a21dc2 --- /dev/null +++ b/gluten-ut/spark41/pom.xml @@ -0,0 +1,306 @@ + + + 4.0.0 + + org.apache.gluten + gluten-ut + 1.6.0-SNAPSHOT + ../pom.xml + + + gluten-ut-spark41 + jar + Gluten Unit Test Spark41 + + + 1.16.0 + + + + + org.apache.gluten + gluten-ut-common + ${project.version} + test-jar + compile + + + org.apache.parquet + parquet-column + ${parquet.version} + tests + test + + + org.apache.parquet + parquet-common + ${parquet.version} + tests + test + + + org.apache.commons + commons-lang3 + ${commons-lang3.version} + + + org.apache.hive + hive-llap-common + 4.0.0 + test + + + org.apache.hive + hive-common + + + org.apache.hive + hive-serde + + + org.slf4j + slf4j-api + + + + + org.apache.hive + hive-llap-client + 2.3.10 + test + + + org.apache.hive + hive-common + + + org.apache.hive + hive-serde + + + org.apache.hive + hive-llap-common + + + org.apache.curator + curator-framework + + + org.apache.curator + apache-curator + + + org.apache.zookeeper + zookeeper + + + org.slf4j + slf4j-api + + + + + + + + + org.apache.maven.plugins + maven-resources-plugin + + + net.alchim31.maven + scala-maven-plugin + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.scalastyle + scalastyle-maven-plugin + + + com.diffplug.spotless + spotless-maven-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + org.scalatest + scalatest-maven-plugin + + . + + + + test + + test + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + prepare-test-jar + + test-jar + + test-compile + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + + + backends-clickhouse + + false + + + + org.apache.gluten + backends-clickhouse + ${project.version} + test + + + org.apache.celeborn + celeborn-client-spark-${spark.major.version}-shaded_${scala.binary.version} + ${celeborn.version} + test + + + org.apache.arrow + arrow-memory-core + ${arrow.version} + provided + + + io.netty + netty-common + + + io.netty + netty-buffer + + + + + org.apache.arrow + arrow-vector + ${arrow.version} + provided + + + io.netty + netty-common + + + io.netty + netty-buffer + + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-sources + + add-test-source + + generate-sources + + + src/test/backends-clickhouse + + + + + + + + + + backends-velox + + false + + + + org.junit.jupiter + junit-jupiter + 5.9.3 + test + + + net.aichler + jupiter-interface + 0.11.1 + test + + + org.apache.gluten + backends-velox + ${project.version} + test + + + + org.slf4j + slf4j-api + ${slf4j.version} + test + + + org.apache.logging.log4j + log4j-slf4j2-impl + ${log4j.version} + test + + + com.h2database + h2 + 2.2.220 + test + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-sources + + add-test-source + + generate-sources + + + src/test/backends-velox + + + + + + + + + + diff --git a/gluten-ut/spark41/src/test/backends-clickhouse/org/apache/gluten/GlutenColumnarWriteTestSupport.scala b/gluten-ut/spark41/src/test/backends-clickhouse/org/apache/gluten/GlutenColumnarWriteTestSupport.scala new file mode 100644 index 000000000000..4258cd891a5a --- /dev/null +++ b/gluten-ut/spark41/src/test/backends-clickhouse/org/apache/gluten/GlutenColumnarWriteTestSupport.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten + +import org.apache.spark.sql.execution.{ColumnarWriteFilesExec, SparkPlan} + +trait GlutenColumnarWriteTestSupport { + + def checkWriteFilesAndGetChild(sparkPlan: SparkPlan): SparkPlan = { + assert(sparkPlan.isInstanceOf[ColumnarWriteFilesExec]) + sparkPlan.asInstanceOf[ColumnarWriteFilesExec].child + } +} diff --git a/gluten-ut/spark41/src/test/backends-clickhouse/org/apache/gluten/execution/parquet/GlutenParquetV1FilterSuite2.scala b/gluten-ut/spark41/src/test/backends-clickhouse/org/apache/gluten/execution/parquet/GlutenParquetV1FilterSuite2.scala new file mode 100644 index 000000000000..5b84888376aa --- /dev/null +++ b/gluten-ut/spark41/src/test/backends-clickhouse/org/apache/gluten/execution/parquet/GlutenParquetV1FilterSuite2.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.execution.parquet + +import org.apache.spark.SparkConf +import org.apache.spark.sql.execution.datasources.parquet.GlutenParquetV1FilterSuite + + +/** testing use_local_format parquet reader. **/ +class GlutenParquetV1FilterSuite2 extends GlutenParquetV1FilterSuite { + override def sparkConf: SparkConf = + super.sparkConf + .set("spark.gluten.sql.columnar.backend.ch.runtime_config.use_local_format", "true") +} diff --git a/gluten-ut/spark41/src/test/backends-velox/org/apache/gluten/GlutenColumnarWriteTestSupport.scala b/gluten-ut/spark41/src/test/backends-velox/org/apache/gluten/GlutenColumnarWriteTestSupport.scala new file mode 100644 index 000000000000..68c0e1c932b5 --- /dev/null +++ b/gluten-ut/spark41/src/test/backends-velox/org/apache/gluten/GlutenColumnarWriteTestSupport.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten + +import org.apache.spark.sql.execution.{SparkPlan, ColumnarWriteFilesExec} + +trait GlutenColumnarWriteTestSupport { + + def checkWriteFilesAndGetChild(sparkPlan: SparkPlan): SparkPlan = { + assert(sparkPlan.isInstanceOf[ColumnarWriteFilesExec]) + sparkPlan.asInstanceOf[ColumnarWriteFilesExec].child + } +} diff --git a/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/group-by-ordinal.sql b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/group-by-ordinal.sql new file mode 100644 index 000000000000..b773396c050d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/group-by-ordinal.sql @@ -0,0 +1,96 @@ +-- group by ordinal positions + +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b); + +-- basic case +select a, sum(b) from data group by 1; + +-- constant case +select 1, 2, sum(b) from data group by 1, 2; + +-- duplicate group by column +select a, 1, sum(b) from data group by a, 1; +select a, 1, sum(b) from data group by 1, 2; + +-- group by a non-aggregate expression's ordinal +select a, b + 2, count(2) from data group by a, 2; + +-- with alias +select a as aa, b + 2 as bb, count(2) from data group by 1, 2; + +-- foldable non-literal: this should be the same as no grouping. +select sum(b) from data group by 1 + 0; + +-- negative cases: ordinal out of range +select a, b from data group by -1; +select a, b from data group by 0; +select a, b from data group by 3; + +-- negative case: position is an aggregate expression +select a, b, sum(b) from data group by 3; +select a, b, sum(b) + 2 from data group by 3; + +-- negative case: nondeterministic expression +select a, rand(0), sum(b) +from +(select /*+ REPARTITION(1) */ a, b from data) group by a, 2; + +-- negative case: star +select * from data group by a, b, 1; + +-- group by ordinal followed by order by +select a, count(a) from (select 1 as a) tmp group by 1 order by 1; + +-- group by ordinal followed by having +select count(a), a from (select 1 as a) tmp group by 2 having a > 0; + +-- mixed cases: group-by ordinals and aliases +select a, a AS k, count(b) from data group by k, 1; + +-- can use ordinal in CUBE +select a, b, count(1) from data group by cube(1, 2); + +-- mixed cases: can use ordinal in CUBE +select a, b, count(1) from data group by cube(1, b); + +-- can use ordinal with cube +select a, b, count(1) from data group by 1, 2 with cube; + +-- can use ordinal in ROLLUP +select a, b, count(1) from data group by rollup(1, 2); + +-- mixed cases: can use ordinal in ROLLUP +select a, b, count(1) from data group by rollup(1, b); + +-- can use ordinal with rollup +select a, b, count(1) from data group by 1, 2 with rollup; + +-- can use ordinal in GROUPING SETS +select a, b, count(1) from data group by grouping sets((1), (2), (1, 2)); + +-- mixed cases: can use ordinal in GROUPING SETS +select a, b, count(1) from data group by grouping sets((1), (b), (a, 2)); + +select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2)); + +-- range error +select a, b, count(1) from data group by a, -1; + +select a, b, count(1) from data group by a, 3; + +select a, b, count(1) from data group by cube(-1, 2); + +select a, b, count(1) from data group by cube(1, 3); + +-- turn off group by ordinal +set spark.sql.groupByOrdinal=false; + +-- can now group by negative literal +select sum(b) from data group by -1; diff --git a/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/group-by.sql b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/group-by.sql new file mode 100644 index 000000000000..c35cdb0de271 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/group-by.sql @@ -0,0 +1,266 @@ +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b); + +-- Aggregate with empty GroupBy expressions. +SELECT a, COUNT(b) FROM testData; +SELECT COUNT(a), COUNT(b) FROM testData; + +-- Aggregate with non-empty GroupBy expressions. +SELECT a, COUNT(b) FROM testData GROUP BY a; +SELECT a, COUNT(b) FROM testData GROUP BY b; +SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a; + +-- Aggregate grouped by literals. +SELECT 'foo', COUNT(a) FROM testData GROUP BY 1; + +-- Aggregate grouped by literals (whole stage code generation). +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate grouped by literals (hash aggregate). +SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate grouped by literals (sort aggregate). +SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate with complex GroupBy expressions. +SELECT a + b, COUNT(b) FROM testData GROUP BY a + b; +SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1; +SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1; + +-- struct() in group by +SELECT count(1) FROM testData GROUP BY struct(a + 0.1 AS aa); + +-- Aggregate with nulls. +SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) +FROM testData; + +-- Aggregate with foldable input and multiple distinct groups. +SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; + +-- Aliases in SELECT could be used in GROUP BY +SELECT a AS k, COUNT(b) FROM testData GROUP BY k; +SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1; + +-- GROUP BY alias with invalid col in SELECT list +SELECT a AS k, COUNT(non_existing) FROM testData GROUP BY k; + +-- Aggregate functions cannot be used in GROUP BY +SELECT COUNT(b) AS k FROM testData GROUP BY k; + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v); +SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a; + +-- turn off group by aliases +set spark.sql.groupByAliases=false; + +-- Check analysis exceptions +SELECT a AS k, COUNT(b) FROM testData GROUP BY k; + +-- Aggregate with empty input and non-empty GroupBy expressions. +SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a; + +-- Aggregate with empty input and empty GroupBy expressions. +SELECT COUNT(1) FROM testData WHERE false; +SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t; + +-- Aggregate with empty GroupBy expressions and filter on top +SELECT 1 from ( + SELECT 1 AS z, + MIN(a.x) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z; + +-- SPARK-24369 multiple distinct aggregations having the same argument set +SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y); + +-- SPARK-25708 HAVING without GROUP BY means global aggregate +SELECT 1 FROM range(10) HAVING true; + +SELECT 1 FROM range(10) HAVING MAX(id) > 0; + +SELECT id FROM range(10) HAVING id > 0; + +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=true; + +SELECT 1 FROM range(10) HAVING true; + +SELECT 1 FROM range(10) HAVING MAX(id) > 0; + +SELECT id FROM range(10) HAVING id > 0; + +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=false; + +-- Test data +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v); + +-- empty table +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0; + +-- all null values +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4; + +-- aggregates are null Filtering +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5; + +-- group by +SELECT k, every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k; + +-- having +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false; +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL; + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY k; + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY k; + +-- input type checking Int +SELECT every(1); + +-- input type checking Short +SELECT some(1S); + +-- input type checking Long +SELECT any(1L); + +-- input type checking String +SELECT every("true"); + +-- input type checking Decimal +SELECT bool_and(1.0); + +-- input type checking double +SELECT bool_or(1.0D); + +-- every/some/any aggregates/bool_and/bool_or are supported as windows expression. +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, bool_and(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, bool_or(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; + +-- Having referencing aggregate expressions is ok. +SELECT count(*) FROM test_agg HAVING count(*) > 1L; +SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true; + +-- Aggrgate expressions can be referenced through an alias +SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L; + +-- Error when aggregate expressions are in where clause directly +SELECT count(*) FROM test_agg WHERE count(*) > 1L; +SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L; +SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1; + +-- Aggregate with multiple distinct decimal columns +SELECT AVG(DISTINCT decimal_col), SUM(DISTINCT decimal_col) FROM VALUES (CAST(1 AS DECIMAL(9, 0))) t(decimal_col); + +-- SPARK-34581: Don't optimize out grouping expressions from aggregate expressions without aggregate function +SELECT not(a IS NULL), count(*) AS c +FROM testData +GROUP BY a IS NULL; + +SELECT if(not(a IS NULL), rand(0), 1), count(*) AS c +FROM testData +GROUP BY a IS NULL; + + +-- Histogram aggregates with different numeric input types +SELECT + histogram_numeric(col, 2) as histogram_2, + histogram_numeric(col, 3) as histogram_3, + histogram_numeric(col, 5) as histogram_5, + histogram_numeric(col, 10) as histogram_10 +FROM VALUES + (1), (2), (3), (4), (5), (6), (7), (8), (9), (10), + (11), (12), (13), (14), (15), (16), (17), (18), (19), (20), + (21), (22), (23), (24), (25), (26), (27), (28), (29), (30), + (31), (32), (33), (34), (35), (3), (37), (38), (39), (40), + (41), (42), (43), (44), (45), (46), (47), (48), (49), (50) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1), (2), (3) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1L), (2L), (3L) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1F), (2F), (3F) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1D), (2D), (3D) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1S), (2S), (3S) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BYTE)), (CAST(2 AS BYTE)), (CAST(3 AS BYTE)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS TINYINT)), (CAST(2 AS TINYINT)), (CAST(3 AS TINYINT)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS SMALLINT)), (CAST(2 AS SMALLINT)), (CAST(3 AS SMALLINT)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BIGINT)), (CAST(2 AS BIGINT)), (CAST(3 AS BIGINT)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (TIMESTAMP '2017-03-01 00:00:00'), + (TIMESTAMP '2017-04-01 00:00:00'), (TIMESTAMP '2017-05-01 00:00:00') AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '100-00' YEAR TO MONTH), + (INTERVAL '110-00' YEAR TO MONTH), (INTERVAL '120-00' YEAR TO MONTH) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '12 20:4:0' DAY TO SECOND), + (INTERVAL '12 21:4:0' DAY TO SECOND), (INTERVAL '12 22:4:0' DAY TO SECOND) AS tab(col); +SELECT histogram_numeric(col, 3) +FROM VALUES (NULL), (NULL), (NULL) AS tab(col); +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)) AS tab(col); +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS INT)), (CAST(NULL AS INT)), (CAST(NULL AS INT)) AS tab(col); + +-- SPARK-27974: Support ANSI Aggregate Function: array_agg +SELECT + collect_list(col), + array_agg(col) +FROM VALUES + (1), (2), (1) AS tab(col); +SELECT + a, + collect_list(b), + array_agg(b) +FROM VALUES + (1,4),(2,3),(1,4),(2,4) AS v(a,b) +GROUP BY a; + + +SELECT mode(a), mode(b) FROM testData; +SELECT a, mode(b) FROM testData GROUP BY a ORDER BY a; + + +-- SPARK-44846: PushFoldableIntoBranches in complex grouping expressions cause bindReference error +SELECT c * 2 AS d +FROM ( + SELECT if(b > 1, 1, b) AS c + FROM ( + SELECT if(a < 0, 0, a) AS b + FROM VALUES (-1), (1), (2) AS t1(a) + ) t2 + GROUP BY b + ) t3 +GROUP BY c; diff --git a/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/linear-regression.sql b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/linear-regression.sql new file mode 100644 index 000000000000..c7cb5bf1117a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/linear-regression.sql @@ -0,0 +1,52 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testRegression AS SELECT * FROM VALUES +(1, 10, null), (2, 10, 11), (2, 20, 22), (2, 25, null), (2, 30, 35) +AS testRegression(k, y, x); + +-- SPARK-37613: Support ANSI Aggregate Function: regr_count +SELECT regr_count(y, x) FROM testRegression; +SELECT regr_count(y, x) FROM testRegression WHERE x IS NOT NULL; +SELECT k, count(*), regr_count(y, x) FROM testRegression GROUP BY k; +SELECT k, count(*) FILTER (WHERE x IS NOT NULL), regr_count(y, x) FROM testRegression GROUP BY k; + +-- SPARK-37613: Support ANSI Aggregate Function: regr_r2 +SELECT regr_r2(y, x) FROM testRegression; +SELECT regr_r2(y, x) FROM testRegression WHERE x IS NOT NULL; +SELECT k, corr(y, x), regr_r2(y, x) FROM testRegression GROUP BY k; +SELECT k, corr(y, x) FILTER (WHERE x IS NOT NULL), regr_r2(y, x) FROM testRegression GROUP BY k; + +-- SPARK-37614: Support ANSI Aggregate Function: regr_avgx & regr_avgy +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression; +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, avg(x), avg(y), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k; +SELECT k, avg(x) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), avg(y) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k; + +-- SPARK-37672: Support ANSI Aggregate Function: regr_sxx +SELECT regr_sxx(y, x) FROM testRegression; +SELECT regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_sxx(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; + +-- SPARK-37681: Support ANSI Aggregate Function: regr_sxy +SELECT regr_sxy(y, x) FROM testRegression; +SELECT regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_sxy(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; + +-- SPARK-37702: Support ANSI Aggregate Function: regr_syy +SELECT regr_syy(y, x) FROM testRegression; +SELECT regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_syy(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; + +-- SPARK-39230: Support ANSI Aggregate Function: regr_slope +SELECT regr_slope(y, x) FROM testRegression; +SELECT regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_slope(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; + +-- SPARK-37623: Support ANSI Aggregate Function: regr_intercept +SELECT regr_intercept(y, x) FROM testRegression; +SELECT regr_intercept(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_intercept(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_intercept(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; diff --git a/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/misc-functions.sql b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/misc-functions.sql new file mode 100644 index 000000000000..907ff33000d8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/misc-functions.sql @@ -0,0 +1,22 @@ +-- test for misc functions + +-- typeof +select typeof(null); +select typeof(true); +select typeof(1Y), typeof(1S), typeof(1), typeof(1L); +select typeof(cast(1.0 as float)), typeof(1.0D), typeof(1.2); +select typeof(date '1986-05-23'), typeof(timestamp '1986-05-23'), typeof(interval '23 days'); +select typeof(x'ABCD'), typeof('SPARK'); +select typeof(array(1, 2)), typeof(map(1, 2)), typeof(named_struct('a', 1, 'b', 'spark')); + +-- Spark-32793: Rewrite AssertTrue with RaiseError +SELECT assert_true(true), assert_true(boolean(1)); +SELECT assert_true(false); +SELECT assert_true(boolean(0)); +SELECT assert_true(null); +SELECT assert_true(boolean(null)); +SELECT assert_true(false, 'custom error message'); + +CREATE TEMPORARY VIEW tbl_misc AS SELECT * FROM (VALUES (1), (8), (2)) AS T(v); +SELECT raise_error('error message'); +SELECT if(v > 5, raise_error('too big: ' || v), v + 1) FROM tbl_misc; diff --git a/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/random.sql b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/random.sql new file mode 100644 index 000000000000..a1aae7b8759d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/random.sql @@ -0,0 +1,17 @@ +-- rand with the seed 0 +SELECT rand(0); +SELECT rand(cast(3 / 7 AS int)); +SELECT rand(NULL); +SELECT rand(cast(NULL AS int)); + +-- rand unsupported data type +SELECT rand(1.0); + +-- randn with the seed 0 +SELECT randn(0L); +SELECT randn(cast(3 / 7 AS long)); +SELECT randn(NULL); +SELECT randn(cast(NULL AS long)); + +-- randn unsupported data type +SELECT rand('1') diff --git a/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/table-valued-functions.sql b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/table-valued-functions.sql new file mode 100644 index 000000000000..79d427bc2099 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/table-valued-functions.sql @@ -0,0 +1,126 @@ +-- unresolved function +select * from dummy(3); + +-- range call with end +select * from range(6 + cos(3)); + +-- range call with start and end +select * from range(5, 10); + +-- range call with step +select * from range(0, 10, 2); + +-- range call with numPartitions +select * from range(0, 10, 1, 200); + +-- range call with invalid number of arguments +select * from range(1, 1, 1, 1, 1); + +-- range call with null +select * from range(1, null); + +-- range call with incompatible type +select * from range(array(1, 2, 3)); + +-- range call with illegal step +select * from range(0, 5, 0); + +-- range call with a mixed-case function name +select * from RaNgE(2); + +-- range call with alias +select i from range(0, 2) t(i); + +-- explode +select * from explode(array(1, 2)); +select * from explode(map('a', 1, 'b', 2)); + +-- explode with empty values +select * from explode(array()); +select * from explode(map()); + +-- explode with column aliases +select * from explode(array(1, 2)) t(c1); +select * from explode(map('a', 1, 'b', 2)) t(k, v); + +-- explode with non-deterministic values +select * from explode(array(rand(0))); + +-- explode with erroneous input +select * from explode(null); +select * from explode(null) t(c1); +select * from explode(1); +select * from explode(1, 2); +select * from explode(explode(array(1))); +select * from explode(array(1, 2)) t(c1, c2); + +-- explode_outer +select * from explode_outer(array(1, 2)); +select * from explode_outer(map('a', 1, 'b', 2)); +select * from explode_outer(array()); +select * from explode_outer(map()); + +-- table-valued functions with join +select * from range(2) join explode(array(1, 2)); +select * from range(2) join explode_outer(array()); + +-- inline +select * from inline(array(struct(1, 'a'), struct(2, 'b'))); +select * from inline(array(struct(1, 'a'), struct(2, 'b'))) t(x, y); +select * from inline(array_remove(array(struct(1, 'a')), struct(1, 'a'))); + +-- inline with erroneous input +select * from inline(null); +select * from inline(array(struct(1, 2), struct(2, 3))) t(a, b, c); + +-- inline_outer +select * from inline_outer(array(struct(1, 'a'), struct(2, 'b'))); +select * from inline_outer(array_remove(array(struct(1, 'a')), struct(1, 'a'))); + +-- posexplode +select * from posexplode(array()); +select * from posexplode(array(1, 2)); +select * from posexplode(array(1, 2)) t(pos, x); +select * from posexplode(map()); +select * from posexplode(map('a', 1, 'b', 2)); +select * from posexplode(map('a', 1, 'b', 2)) t(pos, k, v); + +-- posexplode with erroneous input +select * from posexplode(1); +select * from posexplode(1, 2); +select * from posexplode(explode(array(1))); +select * from posexplode(array(1, 2)) t(x); + +-- posexplode +select * from posexplode_outer(array()); +select * from posexplode_outer(array(1, 2)); +select * from posexplode_outer(map()); +select * from posexplode_outer(map('a', 1, 'b', 2)); + +-- json_tuple +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b'); +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'c'); +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'a'); +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x, y); +select * from json_tuple('{"a": bad, "b": string}', 'a', 'b'); + +-- json_tuple with erroneous input +select * from json_tuple(); +select * from json_tuple('{"a": 1}'); +select * from json_tuple('{"a": 1}', 1); +select * from json_tuple('{"a": 1}', null); +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x); + +-- stack +select * from stack(1, 1, 2, 3); +select * from stack(2, 1, 2, 3); +select * from stack(3, 1, 2, 3) t(x); +select * from stack(4, 1, 2, 3) t(x); +select * from stack(2, 1, 1.1, 'a', 2, 2.2, 'b') t(a, b, c); +select * from stack(2, 1, 1.1, null, 2, null, 'b') t(a, b, c); + +-- stack with erroneous input +select * from stack(); +select * from stack(2, 1, 2, 3) t(a, b, c); +select * from stack(2, 1, '1.1', 'a', 2, 2.2, 'b'); +select * from stack(2, explode(array(1, 2, 3))); diff --git a/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/udf/udf-group-by.sql b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/udf/udf-group-by.sql new file mode 100644 index 000000000000..0cc57c97b020 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/inputs/udf/udf-group-by.sql @@ -0,0 +1,156 @@ +-- This test file was converted from group-by.sql. +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b); + +-- Aggregate with empty GroupBy expressions. +SELECT udf(a), udf(COUNT(b)) FROM testData; +SELECT COUNT(udf(a)), udf(COUNT(b)) FROM testData; + +-- Aggregate with non-empty GroupBy expressions. +SELECT udf(a), COUNT(udf(b)) FROM testData GROUP BY a; +SELECT udf(a), udf(COUNT(udf(b))) FROM testData GROUP BY b; +SELECT COUNT(udf(a)), COUNT(udf(b)) FROM testData GROUP BY udf(a); + +-- Aggregate grouped by literals. +SELECT 'foo', COUNT(udf(a)) FROM testData GROUP BY 1; + +-- Aggregate grouped by literals (whole stage code generation). +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY udf(1); + +-- Aggregate grouped by literals (hash aggregate). +SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1); + +-- Aggregate grouped by literals (sort aggregate). +SELECT 'foo', MAX(STRUCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1); + +-- Aggregate with complex GroupBy expressions. +SELECT udf(a + b), udf(COUNT(b)) FROM testData GROUP BY a + b; +SELECT udf(a + 2), udf(COUNT(b)) FROM testData GROUP BY a + 1; +SELECT udf(a + 1) + 1, udf(COUNT(b)) FROM testData GROUP BY udf(a + 1); + +-- Aggregate with nulls. +SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a)) +FROM testData; + +-- Aggregate with foldable input and multiple distinct groups. +SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY udf(a); + +-- Aliases in SELECT could be used in GROUP BY +SELECT udf(a) AS k, COUNT(udf(b)) FROM testData GROUP BY k; +SELECT a AS k, udf(COUNT(b)) FROM testData GROUP BY k HAVING k > 1; + +-- Aggregate functions cannot be used in GROUP BY +SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k; + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v); +SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY udf(a); + +-- turn off group by aliases +set spark.sql.groupByAliases=false; + +-- Check analysis exceptions +SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k; + +-- Aggregate with empty input and non-empty GroupBy expressions. +SELECT udf(a), COUNT(udf(1)) FROM testData WHERE false GROUP BY udf(a); + +-- Aggregate with empty input and empty GroupBy expressions. +SELECT udf(COUNT(1)) FROM testData WHERE false; +SELECT 1 FROM (SELECT udf(COUNT(1)) FROM testData WHERE false) t; + +-- Aggregate with empty GroupBy expressions and filter on top +SELECT 1 from ( + SELECT 1 AS z, + udf(MIN(a.x)) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z; + +-- SPARK-24369 multiple distinct aggregations having the same argument set +SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y); + +-- SPARK-25708 HAVING without GROUP BY means global aggregate +SELECT udf(1) FROM range(10) HAVING true; + +SELECT udf(udf(1)) FROM range(10) HAVING MAX(id) > 0; + +SELECT udf(id) FROM range(10) HAVING id > 0; + +-- Test data +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v); + +-- empty table +SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0; + +-- all null values +SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4; + +-- aggregates are null Filtering +SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5; + +-- group by +SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k); + +-- having +SELECT udf(k), every(v) FROM test_agg GROUP BY k HAVING every(v) = false; +SELECT udf(k), udf(every(v)) FROM test_agg GROUP BY udf(k) HAVING every(v) IS NULL; + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT udf(k), + udf(Every(v)) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY udf(k); + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT udf(udf(k)), + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY udf(udf(k)); + +-- input type checking Int +SELECT every(udf(1)); + +-- input type checking Short +SELECT some(udf(1S)); + +-- input type checking Long +SELECT any(udf(1L)); + +-- input type checking String +SELECT udf(every("true")); + +-- every/some/any aggregates are supported as windows expression. +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, udf(udf(v)), some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT udf(udf(k)), v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; + +-- Having referencing aggregate expressions is ok. +SELECT udf(count(*)) FROM test_agg HAVING count(*) > 1L; +SELECT k, udf(max(v)) FROM test_agg GROUP BY k HAVING max(v) = true; + +-- Aggrgate expressions can be referenced through an alias +SELECT * FROM (SELECT udf(COUNT(*)) AS cnt FROM test_agg) WHERE cnt > 1L; + +-- Error when aggregate expressions are in where clause directly +SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L; +SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L; +SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1; diff --git a/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/group-by-ordinal.sql.out b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/group-by-ordinal.sql.out new file mode 100644 index 000000000000..b968b4e09fac --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/group-by-ordinal.sql.out @@ -0,0 +1,524 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +select a, sum(b) from data group by 1 +-- !query schema +struct +-- !query output +1 3 +2 3 +3 3 + + +-- !query +select 1, 2, sum(b) from data group by 1, 2 +-- !query schema +struct<1:int,2:int,sum(b):bigint> +-- !query output +1 2 9 + + +-- !query +select a, 1, sum(b) from data group by a, 1 +-- !query schema +struct +-- !query output +1 1 3 +2 1 3 +3 1 3 + + +-- !query +select a, 1, sum(b) from data group by 1, 2 +-- !query schema +struct +-- !query output +1 1 3 +2 1 3 +3 1 3 + + +-- !query +select a, b + 2, count(2) from data group by a, 2 +-- !query schema +struct +-- !query output +1 3 1 +1 4 1 +2 3 1 +2 4 1 +3 3 1 +3 4 1 + + +-- !query +select a as aa, b + 2 as bb, count(2) from data group by 1, 2 +-- !query schema +struct +-- !query output +1 3 1 +1 4 1 +2 3 1 +2 4 1 +3 3 1 +3 4 1 + + +-- !query +select sum(b) from data group by 1 + 0 +-- !query schema +struct +-- !query output +9 + + +-- !query +select a, b from data group by -1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 33, + "fragment" : "-1" + } ] +} + + +-- !query +select a, b from data group by 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "0", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 32, + "fragment" : "0" + } ] +} + + +-- !query +select a, b from data group by 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "3", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 32, + "fragment" : "3" + } ] +} + + +-- !query +select a, b, sum(b) from data group by 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "sum(data.b) AS `sum(b)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 40, + "stopIndex" : 40, + "fragment" : "3" + } ] +} + + +-- !query +select a, b, sum(b) + 2 from data group by 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "(sum(data.b) + CAST(2 AS BIGINT)) AS `(sum(b) + 2)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 44, + "fragment" : "3" + } ] +} + + +-- !query +select a, rand(0), sum(b) +from +(select /*+ REPARTITION(1) */ a, b from data) group by a, 2 +-- !query schema +struct +-- !query output +1 0.5488135024422883 1 +1 0.7151893651681639 2 +2 0.5448831775801376 2 +2 0.6027633705776989 1 +3 0.4236547969336536 1 +3 0.6458941151817286 2 + + +-- !query +select * from data group by a, b, 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "STAR_GROUP_BY_POS", + "sqlState" : "0A000" +} + + +-- !query +select a, count(a) from (select 1 as a) tmp group by 1 order by 1 +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select count(a), a from (select 1 as a) tmp group by 2 having a > 0 +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select a, a AS k, count(b) from data group by k, 1 +-- !query schema +struct +-- !query output +1 1 2 +2 2 2 +3 3 2 + + +-- !query +select a, b, count(1) from data group by cube(1, 2) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by cube(1, b) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by 1, 2 with cube +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by rollup(1, 2) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by rollup(1, b) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by 1, 2 with rollup +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by grouping sets((1), (2), (1, 2)) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 + + +-- !query +select a, b, count(1) from data group by grouping sets((1), (b), (a, 2)) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 + + +-- !query +select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2)) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 + + +-- !query +select a, b, count(1) from data group by a, -1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 46, + "fragment" : "-1" + } ] +} + + +-- !query +select a, b, count(1) from data group by a, 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "count(1) AS `count(1)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 45, + "fragment" : "3" + } ] +} + + +-- !query +select a, b, count(1) from data group by cube(-1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 47, + "stopIndex" : 48, + "fragment" : "-1" + } ] +} + + +-- !query +select a, b, count(1) from data group by cube(1, 3) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "count(1) AS `count(1)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 50, + "stopIndex" : 50, + "fragment" : "3" + } ] +} + + +-- !query +set spark.sql.groupByOrdinal=false +-- !query schema +struct +-- !query output +spark.sql.groupByOrdinal false + + +-- !query +select sum(b) from data group by -1 +-- !query schema +struct +-- !query output +9 diff --git a/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/group-by.sql.out b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/group-by.sql.out new file mode 100644 index 000000000000..a4a3f76fa6a7 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/group-by.sql.out @@ -0,0 +1,1123 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT a, COUNT(b) FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 32, + "fragment" : "SELECT a, COUNT(b) FROM testData" + } ] +} + + +-- !query +SELECT COUNT(a), COUNT(b) FROM testData +-- !query schema +struct +-- !query output +7 7 + + +-- !query +SELECT a, COUNT(b) FROM testData GROUP BY a +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT a, COUNT(b) FROM testData GROUP BY b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a +-- !query schema +struct +-- !query output +0 1 +2 2 +2 2 +3 2 + + +-- !query +SELECT 'foo', COUNT(a) FROM testData GROUP BY 1 +-- !query schema +struct +-- !query output +foo 7 + + +-- !query +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct> +-- !query output + + + +-- !query +SELECT a + b, COUNT(b) FROM testData GROUP BY a + b +-- !query schema +struct<(a + b):int,count(b):bigint> +-- !query output +2 1 +3 2 +4 2 +5 1 +NULL 1 + + +-- !query +SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1 +-- !query schema +struct<((a + 1) + 1):int,count(b):bigint> +-- !query output +3 2 +4 2 +5 2 +NULL 1 + + +-- !query +SELECT count(1) FROM testData GROUP BY struct(a + 0.1 AS aa) +-- !query schema +struct +-- !query output +2 +2 +2 +3 + + +-- !query +SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) +FROM testData +-- !query schema +struct +-- !query output +-0.27238010581457284 -1.5069204152249138 1 3 2.142857142857143 0.8095238095238096 0.8997354108424375 15 7 + + +-- !query +SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT a AS k, COUNT(b) FROM testData GROUP BY k +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1 +-- !query schema +struct +-- !query output +2 2 +3 2 + + +-- !query +SELECT a AS k, COUNT(non_existing) FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`non_existing`", + "proposal" : "`testdata`.`a`, `testdata`.`b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 33, + "fragment" : "non_existing" + } ] +} + + +-- !query +SELECT COUNT(b) AS k FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "sqlExpr" : "count(testdata.b)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "COUNT(b)" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"k\"", + "expressionAnyValue" : "\"any_value(k)\"" + } +} + + +-- !query +set spark.sql.groupByAliases=false +-- !query schema +struct +-- !query output +spark.sql.groupByAliases false + + +-- !query +SELECT a AS k, COUNT(b) FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`k`", + "proposal" : "`testdata`.`a`, `testdata`.`b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 48, + "stopIndex" : 48, + "fragment" : "k" + } ] +} + + +-- !query +SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a +-- !query schema +struct +-- !query output + + + +-- !query +SELECT COUNT(1) FROM testData WHERE false +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT 1 from ( + SELECT 1 AS z, + MIN(a.x) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z +-- !query schema +struct<1:int> +-- !query output + + + +-- !query +SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) +-- !query schema +struct +-- !query output +0.9999999999999999 0.9999999999999999 3 + + +-- !query +SELECT 1 FROM range(10) HAVING true +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT 1 FROM range(10) HAVING MAX(id) > 0 +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT id FROM range(10) HAVING id > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 38, + "fragment" : "SELECT id FROM range(10) HAVING id > 0" + } ] +} + + +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=true +-- !query schema +struct +-- !query output +spark.sql.legacy.parser.havingWithoutGroupByAsWhere true + + +-- !query +SELECT 1 FROM range(10) HAVING true +-- !query schema +struct<1:int> +-- !query output +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + + +-- !query +SELECT 1 FROM range(10) HAVING MAX(id) > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(max(id) > 0)\"", + "expressionList" : "max(id)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 42, + "fragment" : "SELECT 1 FROM range(10) HAVING MAX(id) > 0" + } ] +} + + +-- !query +SELECT id FROM range(10) HAVING id > 0 +-- !query schema +struct +-- !query output +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=false +-- !query schema +struct +-- !query output +spark.sql.legacy.parser.havingWithoutGroupByAsWhere false + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0 +-- !query schema +struct +-- !query output +NULL NULL NULL NULL NULL + + +-- !query +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4 +-- !query schema +struct +-- !query output +NULL NULL NULL NULL NULL + + +-- !query +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5 +-- !query schema +struct +-- !query output +false true true false true + + +-- !query +SELECT k, every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k +-- !query schema +struct +-- !query output +1 false true true false true +2 true true true true true +3 false false false false false +4 NULL NULL NULL NULL NULL +5 false true true false true + + +-- !query +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false +-- !query schema +struct +-- !query output +1 false +3 false +5 false + + +-- !query +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL +-- !query schema +struct +-- !query output +4 NULL + + +-- !query +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY k +-- !query schema +struct +-- !query output +2 true + + +-- !query +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY k +-- !query schema +struct +-- !query output + + + +-- !query +SELECT every(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"every(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "every(1)" + } ] +} + + +-- !query +SELECT some(1S) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"some(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "some(1S)" + } ] +} + + +-- !query +SELECT any(1L) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"any(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 14, + "fragment" : "any(1L)" + } ] +} + + +-- !query +SELECT every("true") +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"true\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"every(true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "every(\"true\")" + } ] +} + + +-- !query +SELECT bool_and(1.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"bool_and(1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "bool_and(1.0)" + } ] +} + + +-- !query +SELECT bool_or(1.0D) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"bool_or(1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "bool_or(1.0D)" + } ] +} + + +-- !query +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true false +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true false + + +-- !query +SELECT k, v, some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT k, v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT k, v, bool_and(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true false +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true false + + +-- !query +SELECT k, v, bool_or(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT count(*) FROM test_agg HAVING count(*) > 1L +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true +-- !query schema +struct +-- !query output +1 true +2 true +5 true + + +-- !query +SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT count(*) FROM test_agg WHERE count(*) > 1L +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(count(1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 49, + "fragment" : "SELECT count(*) FROM test_agg WHERE count(*) > 1L" + } ] +} + + +-- !query +SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"((count(1) + 1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L" + } ] +} + + +-- !query +SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(((k = 1) OR (k = 2)) OR (((count(1) + 1) > 1) OR (max(k) > 1)))\"", + "expressionList" : "count(1), max(test_agg.k)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 86, + "fragment" : "SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1" + } ] +} + + +-- !query +SELECT AVG(DISTINCT decimal_col), SUM(DISTINCT decimal_col) FROM VALUES (CAST(1 AS DECIMAL(9, 0))) t(decimal_col) +-- !query schema +struct +-- !query output +1.0000 1 + + +-- !query +SELECT not(a IS NULL), count(*) AS c +FROM testData +GROUP BY a IS NULL +-- !query schema +struct<(NOT (a IS NULL)):boolean,c:bigint> +-- !query output +false 2 +true 7 + + +-- !query +SELECT if(not(a IS NULL), rand(0), 1), count(*) AS c +FROM testData +GROUP BY a IS NULL +-- !query schema +struct<(IF((NOT (a IS NULL)), rand(0), 1)):double,c:bigint> +-- !query output +0.5488135024422883 7 +1.0 2 + + +-- !query +SELECT + histogram_numeric(col, 2) as histogram_2, + histogram_numeric(col, 3) as histogram_3, + histogram_numeric(col, 5) as histogram_5, + histogram_numeric(col, 10) as histogram_10 +FROM VALUES + (1), (2), (3), (4), (5), (6), (7), (8), (9), (10), + (11), (12), (13), (14), (15), (16), (17), (18), (19), (20), + (21), (22), (23), (24), (25), (26), (27), (28), (29), (30), + (31), (32), (33), (34), (35), (3), (37), (38), (39), (40), + (41), (42), (43), (44), (45), (46), (47), (48), (49), (50) AS tab(col) +-- !query schema +struct>,histogram_3:array>,histogram_5:array>,histogram_10:array>> +-- !query output +[{"x":12,"y":26.0},{"x":38,"y":24.0}] [{"x":9,"y":20.0},{"x":25,"y":11.0},{"x":40,"y":19.0}] [{"x":5,"y":11.0},{"x":14,"y":8.0},{"x":22,"y":7.0},{"x":30,"y":10.0},{"x":43,"y":14.0}] [{"x":3,"y":6.0},{"x":8,"y":6.0},{"x":13,"y":4.0},{"x":17,"y":3.0},{"x":20,"y":4.0},{"x":25,"y":6.0},{"x":31,"y":7.0},{"x":39,"y":5.0},{"x":43,"y":4.0},{"x":48,"y":5.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1), (2), (3) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1L), (2L), (3L) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1F), (2F), (3F) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1.0,"y":1.0},{"x":2.0,"y":1.0},{"x":3.0,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1D), (2D), (3D) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1.0,"y":1.0},{"x":2.0,"y":1.0},{"x":3.0,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1S), (2S), (3S) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BYTE)), (CAST(2 AS BYTE)), (CAST(3 AS BYTE)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS TINYINT)), (CAST(2 AS TINYINT)), (CAST(3 AS TINYINT)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS SMALLINT)), (CAST(2 AS SMALLINT)), (CAST(3 AS SMALLINT)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BIGINT)), (CAST(2 AS BIGINT)), (CAST(3 AS BIGINT)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (TIMESTAMP '2017-03-01 00:00:00'), + (TIMESTAMP '2017-04-01 00:00:00'), (TIMESTAMP '2017-05-01 00:00:00') AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":2017-03-01 00:00:00,"y":1.0},{"x":2017-04-01 00:00:00,"y":1.0},{"x":2017-05-01 00:00:00,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '100-00' YEAR TO MONTH), + (INTERVAL '110-00' YEAR TO MONTH), (INTERVAL '120-00' YEAR TO MONTH) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":100-0,"y":1.0},{"x":110-0,"y":1.0},{"x":120-0,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '12 20:4:0' DAY TO SECOND), + (INTERVAL '12 21:4:0' DAY TO SECOND), (INTERVAL '12 22:4:0' DAY TO SECOND) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":12 20:04:00.000000000,"y":1.0},{"x":12 21:04:00.000000000,"y":1.0},{"x":12 22:04:00.000000000,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) +FROM VALUES (NULL), (NULL), (NULL) AS tab(col) +-- !query schema +struct>> +-- !query output +NULL + + +-- !query +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)) AS tab(col) +-- !query schema +struct>> +-- !query output +NULL + + +-- !query +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS INT)), (CAST(NULL AS INT)), (CAST(NULL AS INT)) AS tab(col) +-- !query schema +struct>> +-- !query output +NULL + + +-- !query +SELECT + collect_list(col), + array_agg(col) +FROM VALUES + (1), (2), (1) AS tab(col) +-- !query schema +struct,collect_list(col):array> +-- !query output +[1,2,1] [1,2,1] + + +-- !query +SELECT + a, + collect_list(b), + array_agg(b) +FROM VALUES + (1,4),(2,3),(1,4),(2,4) AS v(a,b) +GROUP BY a +-- !query schema +struct,collect_list(b):array> +-- !query output +1 [4,4] [4,4] +2 [3,4] [3,4] + + +-- !query +SELECT mode(a), mode(b) FROM testData +-- !query schema +struct +-- !query output +3 1 + + +-- !query +SELECT a, mode(b) FROM testData GROUP BY a ORDER BY a +-- !query schema +struct +-- !query output +NULL 1 +1 1 +2 1 +3 1 + + +-- !query +SELECT c * 2 AS d +FROM ( + SELECT if(b > 1, 1, b) AS c + FROM ( + SELECT if(a < 0, 0, a) AS b + FROM VALUES (-1), (1), (2) AS t1(a) + ) t2 + GROUP BY b + ) t3 +GROUP BY c +-- !query schema +struct +-- !query output +0 +2 diff --git a/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/linear-regression.sql.out b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/linear-regression.sql.out new file mode 100644 index 000000000000..a73347a2f7f5 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/linear-regression.sql.out @@ -0,0 +1,276 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testRegression AS SELECT * FROM VALUES +(1, 10, null), (2, 10, 11), (2, 20, 22), (2, 25, null), (2, 30, 35) +AS testRegression(k, y, x) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT regr_count(y, x) FROM testRegression +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT regr_count(y, x) FROM testRegression WHERE x IS NOT NULL +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT k, count(*), regr_count(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 1 0 +2 4 3 + + +-- !query +SELECT k, count(*) FILTER (WHERE x IS NOT NULL), regr_count(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 0 0 +2 3 3 + + +-- !query +SELECT regr_r2(y, x) FROM testRegression +-- !query schema +struct +-- !query output +0.9976905311778291 + + +-- !query +SELECT regr_r2(y, x) FROM testRegression WHERE x IS NOT NULL +-- !query schema +struct +-- !query output +0.9976905311778291 + + +-- !query +SELECT k, corr(y, x), regr_r2(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL NULL +2 0.9988445981121532 0.9976905311778291 + + +-- !query +SELECT k, corr(y, x) FILTER (WHERE x IS NOT NULL), regr_r2(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL NULL +2 0.9988445981121532 0.9976905311778291 + + +-- !query +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression +-- !query schema +struct +-- !query output +22.666666666666668 20.0 + + +-- !query +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +22.666666666666668 20.0 + + +-- !query +SELECT k, avg(x), avg(y), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL 10.0 NULL NULL +2 22.666666666666668 21.25 22.666666666666668 20.0 + + +-- !query +SELECT k, avg(x) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), avg(y) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL NULL NULL NULL +2 22.666666666666668 20.0 22.666666666666668 20.0 + + +-- !query +SELECT regr_sxx(y, x) FROM testRegression +-- !query schema +struct +-- !query output +288.66666666666663 + + +-- !query +SELECT regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +288.66666666666663 + + +-- !query +SELECT k, regr_sxx(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 288.66666666666663 + + +-- !query +SELECT k, regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 288.66666666666663 + + +-- !query +SELECT regr_sxy(y, x) FROM testRegression +-- !query schema +struct +-- !query output +240.0 + + +-- !query +SELECT regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +240.0 + + +-- !query +SELECT k, regr_sxy(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 240.0 + + +-- !query +SELECT k, regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 240.0 + + +-- !query +SELECT regr_syy(y, x) FROM testRegression +-- !query schema +struct +-- !query output +200.0 + + +-- !query +SELECT regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +200.0 + + +-- !query +SELECT k, regr_syy(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 200.0 + + +-- !query +SELECT k, regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 200.0 + + +-- !query +SELECT regr_slope(y, x) FROM testRegression +-- !query schema +struct +-- !query output +0.8314087759815244 + + +-- !query +SELECT regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +0.8314087759815244 + + +-- !query +SELECT k, regr_slope(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 0.8314087759815244 + + +-- !query +SELECT k, regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 0.8314087759815244 + + +-- !query +SELECT regr_intercept(y, x) FROM testRegression +-- !query schema +struct +-- !query output +1.1547344110854487 + + +-- !query +SELECT regr_intercept(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +1.1547344110854487 + + +-- !query +SELECT k, regr_intercept(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 1.1547344110854487 + + +-- !query +SELECT k, regr_intercept(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 1.1547344110854487 diff --git a/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/misc-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/misc-functions.sql.out new file mode 100644 index 000000000000..d6d1289a5adb --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/misc-functions.sql.out @@ -0,0 +1,134 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select typeof(null) +-- !query schema +struct +-- !query output +void + + +-- !query +select typeof(true) +-- !query schema +struct +-- !query output +boolean + + +-- !query +select typeof(1Y), typeof(1S), typeof(1), typeof(1L) +-- !query schema +struct +-- !query output +tinyint smallint int bigint + + +-- !query +select typeof(cast(1.0 as float)), typeof(1.0D), typeof(1.2) +-- !query schema +struct +-- !query output +float double decimal(2,1) + + +-- !query +select typeof(date '1986-05-23'), typeof(timestamp '1986-05-23'), typeof(interval '23 days') +-- !query schema +struct +-- !query output +date timestamp interval day + + +-- !query +select typeof(x'ABCD'), typeof('SPARK') +-- !query schema +struct +-- !query output +binary string + + +-- !query +select typeof(array(1, 2)), typeof(map(1, 2)), typeof(named_struct('a', 1, 'b', 'spark')) +-- !query schema +struct +-- !query output +array map struct + + +-- !query +SELECT assert_true(true), assert_true(boolean(1)) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +SELECT assert_true(false) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +'false' is not true! + + +-- !query +SELECT assert_true(boolean(0)) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +'cast(0 as boolean)' is not true! + + +-- !query +SELECT assert_true(null) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +'null' is not true! + + +-- !query +SELECT assert_true(boolean(null)) +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +'cast(null as boolean)' is not true! + + +-- !query +SELECT assert_true(false, 'custom error message') +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +custom error message + + +-- !query +CREATE TEMPORARY VIEW tbl_misc AS SELECT * FROM (VALUES (1), (8), (2)) AS T(v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT raise_error('error message') +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +error message + + +-- !query +SELECT if(v > 5, raise_error('too big: ' || v), v + 1) FROM tbl_misc +-- !query schema +struct<> +-- !query output +org.apache.gluten.exception.GlutenException +too big: 8 diff --git a/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/random.sql.out b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/random.sql.out new file mode 100644 index 000000000000..17e6f871b9c5 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/random.sql.out @@ -0,0 +1,115 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT rand(0) +-- !query schema +struct +-- !query output +0.5488135024422883 + + +-- !query +SELECT rand(cast(3 / 7 AS int)) +-- !query schema +struct +-- !query output +0.5488135024422883 + + +-- !query +SELECT rand(NULL) +-- !query schema +struct +-- !query output +0.5488135024422883 + + +-- !query +SELECT rand(cast(NULL AS int)) +-- !query schema +struct +-- !query output +0.5488135024422883 + + +-- !query +SELECT rand(1.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "1", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"rand(1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "rand(1.0)" + } ] +} + + +-- !query +SELECT randn(0L) +-- !query schema +struct +-- !query output +1.6034991609278433 + + +-- !query +SELECT randn(cast(3 / 7 AS long)) +-- !query schema +struct +-- !query output +1.6034991609278433 + + +-- !query +SELECT randn(NULL) +-- !query schema +struct +-- !query output +1.6034991609278433 + + +-- !query +SELECT randn(cast(NULL AS long)) +-- !query schema +struct +-- !query output +1.6034991609278433 + + +-- !query +SELECT rand('1') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"rand(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "rand('1')" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/table-valued-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/table-valued-functions.sql.out new file mode 100644 index 000000000000..0d5675fa6fde --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/table-valued-functions.sql.out @@ -0,0 +1,1017 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select * from dummy(3) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVABLE_TABLE_VALUED_FUNCTION", + "messageParameters" : { + "name" : "`dummy`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 22, + "fragment" : "dummy(3)" + } ] +} + + +-- !query +select * from range(6 + cos(3)) +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 + + +-- !query +select * from range(5, 10) +-- !query schema +struct +-- !query output +5 +6 +7 +8 +9 + + +-- !query +select * from range(0, 10, 2) +-- !query schema +struct +-- !query output +0 +2 +4 +6 +8 + + +-- !query +select * from range(0, 10, 1, 200) +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +select * from range(1, 1, 1, 1, 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "5", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[1, 2, 3, 4]", + "functionName" : "`range`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 34, + "fragment" : "range(1, 1, 1, 1, 1)" + } ] +} + + +-- !query +select * from range(1, null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`range`", + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "2", + "requiredType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 28, + "fragment" : "range(1, null)" + } ] +} + + +-- !query +select * from range(array(1, 2, 3)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`range`", + "inputSql" : "\"array(1, 2, 3)\"", + "inputType" : "\"ARRAY\"", + "paramIndex" : "2", + "requiredType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 35, + "fragment" : "range(array(1, 2, 3))" + } ] +} + + +-- !query +select * from range(0, 5, 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "FAILED_FUNCTION_CALL", + "sqlState" : "38000", + "messageParameters" : { + "funcName" : "`range`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 28, + "fragment" : "range(0, 5, 0)" + } ] +} + + +-- !query +select * from RaNgE(2) +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +select i from range(0, 2) t(i) +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +select * from explode(array(1, 2)) +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +select * from explode(map('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +a 1 +b 2 + + +-- !query +select * from explode(array()) +-- !query schema +struct +-- !query output + + + +-- !query +select * from explode(map()) +-- !query schema +struct +-- !query output + + + +-- !query +select * from explode(array(1, 2)) t(c1) +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +select * from explode(map('a', 1, 'b', 2)) t(k, v) +-- !query schema +struct +-- !query output +a 1 +b 2 + + +-- !query +select * from explode(array(rand(0))) +-- !query schema +struct +-- !query output +0.5488135024422883 + + +-- !query +select * from explode(null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "1", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"explode(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 27, + "fragment" : "explode(null)" + } ] +} + + +-- !query +select * from explode(null) t(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "1", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"explode(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 33, + "fragment" : "explode(null) t(c1)" + } ] +} + + +-- !query +select * from explode(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"explode(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 24, + "fragment" : "explode(1)" + } ] +} + + +-- !query +select * from explode(1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`explode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 27, + "fragment" : "explode(1, 2)" + } ] +} + + +-- !query +select * from explode(explode(array(1))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + "sqlState" : "0A000", + "messageParameters" : { + "expression" : "\"explode(explode(array(1)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 40, + "fragment" : "explode(explode(array(1)))" + } ] +} + + +-- !query +select * from explode(array(1, 2)) t(c1, c2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "messageParameters" : { + "aliasesNum" : "2", + "funcName" : "`explode`", + "outColsNum" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 44, + "fragment" : "explode(array(1, 2)) t(c1, c2)" + } ] +} + + +-- !query +select * from explode_outer(array(1, 2)) +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +select * from explode_outer(map('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +a 1 +b 2 + + +-- !query +select * from explode_outer(array()) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select * from explode_outer(map()) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +select * from range(2) join explode(array(1, 2)) +-- !query schema +struct +-- !query output +0 1 +0 2 +1 1 +1 2 + + +-- !query +select * from range(2) join explode_outer(array()) +-- !query schema +struct +-- !query output +0 NULL +1 NULL + + +-- !query +select * from inline(array(struct(1, 'a'), struct(2, 'b'))) +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +select * from inline(array(struct(1, 'a'), struct(2, 'b'))) t(x, y) +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +select * from inline(array_remove(array(struct(1, 'a')), struct(1, 'a'))) +-- !query schema +struct +-- !query output + + + +-- !query +select * from inline(null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "1", + "requiredType" : "\"ARRAY\"", + "sqlExpr" : "\"inline(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 26, + "fragment" : "inline(null)" + } ] +} + + +-- !query +select * from inline(array(struct(1, 2), struct(2, 3))) t(a, b, c) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "messageParameters" : { + "aliasesNum" : "3", + "funcName" : "`inline`", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 66, + "fragment" : "inline(array(struct(1, 2), struct(2, 3))) t(a, b, c)" + } ] +} + + +-- !query +select * from inline_outer(array(struct(1, 'a'), struct(2, 'b'))) +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +select * from inline_outer(array_remove(array(struct(1, 'a')), struct(1, 'a'))) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +select * from posexplode(array()) +-- !query schema +struct +-- !query output + + + +-- !query +select * from posexplode(array(1, 2)) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +select * from posexplode(array(1, 2)) t(pos, x) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +select * from posexplode(map()) +-- !query schema +struct +-- !query output + + + +-- !query +select * from posexplode(map('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +0 a 1 +1 b 2 + + +-- !query +select * from posexplode(map('a', 1, 'b', 2)) t(pos, k, v) +-- !query schema +struct +-- !query output +0 a 1 +1 b 2 + + +-- !query +select * from posexplode(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"posexplode(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 27, + "fragment" : "posexplode(1)" + } ] +} + + +-- !query +select * from posexplode(1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`posexplode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 30, + "fragment" : "posexplode(1, 2)" + } ] +} + + +-- !query +select * from posexplode(explode(array(1))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + "sqlState" : "0A000", + "messageParameters" : { + "expression" : "\"posexplode(explode(array(1)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 43, + "fragment" : "posexplode(explode(array(1)))" + } ] +} + + +-- !query +select * from posexplode(array(1, 2)) t(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "messageParameters" : { + "aliasesNum" : "1", + "funcName" : "`posexplode`", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 42, + "fragment" : "posexplode(array(1, 2)) t(x)" + } ] +} + + +-- !query +select * from posexplode_outer(array()) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +select * from posexplode_outer(array(1, 2)) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +select * from posexplode_outer(map()) +-- !query schema +struct +-- !query output +NULL NULL NULL + + +-- !query +select * from posexplode_outer(map('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +0 a 1 +1 b 2 + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') +-- !query schema +struct +-- !query output +1 2 + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'c') +-- !query schema +struct +-- !query output +1 NULL + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'a') +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x, y) +-- !query schema +struct +-- !query output +1 2 + + +-- !query +select * from json_tuple('{"a": bad, "b": string}', 'a', 'b') +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +select * from json_tuple() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 1", + "functionName" : "`json_tuple`" + } +} + + +-- !query +select * from json_tuple('{"a": 1}') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 1", + "functionName" : "`json_tuple`" + } +} + + +-- !query +select * from json_tuple('{"a": 1}', 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_STRING_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "funcName" : "`json_tuple`", + "sqlExpr" : "\"json_tuple({\"a\": 1}, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 39, + "fragment" : "json_tuple('{\"a\": 1}', 1)" + } ] +} + + +-- !query +select * from json_tuple('{"a": 1}', null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_STRING_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "funcName" : "`json_tuple`", + "sqlExpr" : "\"json_tuple({\"a\": 1}, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 42, + "fragment" : "json_tuple('{\"a\": 1}', null)" + } ] +} + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "messageParameters" : { + "aliasesNum" : "1", + "funcName" : "`json_tuple`", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 62, + "fragment" : "json_tuple('{\"a\": 1, \"b\": 2}', 'a', 'b') AS t(x)" + } ] +} + + +-- !query +select * from stack(1, 1, 2, 3) +-- !query schema +struct +-- !query output +1 2 3 + + +-- !query +select * from stack(2, 1, 2, 3) +-- !query schema +struct +-- !query output +1 2 +3 NULL + + +-- !query +select * from stack(3, 1, 2, 3) t(x) +-- !query schema +struct +-- !query output +1 +2 +3 + + +-- !query +select * from stack(4, 1, 2, 3) t(x) +-- !query schema +struct +-- !query output +1 +2 +3 +NULL + + +-- !query +select * from stack(2, 1, 1.1, 'a', 2, 2.2, 'b') t(a, b, c) +-- !query schema +struct +-- !query output +1 1.1 a +2 2.2 b + + +-- !query +select * from stack(2, 1, 1.1, null, 2, null, 'b') t(a, b, c) +-- !query schema +struct +-- !query output +1 1.1 NULL +2 NULL b + + +-- !query +select * from stack() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 1", + "functionName" : "`stack`" + } +} + + +-- !query +select * from stack(2, 1, 2, 3) t(a, b, c) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "messageParameters" : { + "aliasesNum" : "3", + "funcName" : "`stack`", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 42, + "fragment" : "stack(2, 1, 2, 3) t(a, b, c)" + } ] +} + + +-- !query +select * from stack(2, 1, '1.1', 'a', 2, 2.2, 'b') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.STACK_COLUMN_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "columnIndex" : "1", + "leftParamIndex" : "2", + "leftType" : "\"STRING\"", + "rightParamIndex" : "5", + "rightType" : "\"DECIMAL(2,1)\"", + "sqlExpr" : "\"stack(2, 1, 1.1, a, 2, 2.2, b)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 50, + "fragment" : "stack(2, 1, '1.1', 'a', 2, 2.2, 'b')" + } ] +} + + +-- !query +select * from stack(2, explode(array(1, 2, 3))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + "sqlState" : "0A000", + "messageParameters" : { + "expression" : "\"stack(2, explode(array(1, 2, 3)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 47, + "fragment" : "stack(2, explode(array(1, 2, 3)))" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/udf/udf-group-by.sql.out b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/udf/udf-group-by.sql.out new file mode 100644 index 000000000000..d3735acf0f08 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-clickhouse/sql-tests/results/udf/udf-group-by.sql.out @@ -0,0 +1,689 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(a), udf(COUNT(b)) FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 42, + "fragment" : "SELECT udf(a), udf(COUNT(b)) FROM testData" + } ] +} + + +-- !query +SELECT COUNT(udf(a)), udf(COUNT(b)) FROM testData +-- !query schema +struct +-- !query output +7 7 + + +-- !query +SELECT udf(a), COUNT(udf(b)) FROM testData GROUP BY a +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT udf(a), udf(COUNT(udf(b))) FROM testData GROUP BY b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT COUNT(udf(a)), COUNT(udf(b)) FROM testData GROUP BY udf(a) +-- !query schema +struct +-- !query output +0 1 +2 2 +2 2 +3 2 + + +-- !query +SELECT 'foo', COUNT(udf(a)) FROM testData GROUP BY 1 +-- !query schema +struct +-- !query output +foo 7 + + +-- !query +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY udf(1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', MAX(STRUCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1) +-- !query schema +struct> +-- !query output + + + +-- !query +SELECT udf(a + b), udf(COUNT(b)) FROM testData GROUP BY a + b +-- !query schema +struct +-- !query output +2 1 +3 2 +4 2 +5 1 +NULL 1 + + +-- !query +SELECT udf(a + 2), udf(COUNT(b)) FROM testData GROUP BY a + 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT udf(a + 1) + 1, udf(COUNT(b)) FROM testData GROUP BY udf(a + 1) +-- !query schema +struct<(udf((a + 1)) + 1):int,udf(count(b)):bigint> +-- !query output +3 2 +4 2 +5 2 +NULL 1 + + +-- !query +SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a)) +FROM testData +-- !query schema +struct +-- !query output +-0.27238010581457284 -1.5069204152249138 1 3 2.142857142857143 0.8095238095238096 0.8997354108424375 15 7 + + +-- !query +SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY udf(a) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT udf(a) AS k, COUNT(udf(b)) FROM testData GROUP BY k +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT a AS k, udf(COUNT(b)) FROM testData GROUP BY k HAVING k > 1 +-- !query schema +struct +-- !query output +2 2 +3 2 + + +-- !query +SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "sqlExpr" : "CAST(udf(cast(count(b) as string)) AS BIGINT)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "udf(COUNT(b))" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY udf(a) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"k\"", + "expressionAnyValue" : "\"any_value(k)\"" + } +} + + +-- !query +set spark.sql.groupByAliases=false +-- !query schema +struct +-- !query output +spark.sql.groupByAliases false + + +-- !query +SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`k`", + "proposal" : "`testdata`.`a`, `testdata`.`b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 58, + "stopIndex" : 58, + "fragment" : "k" + } ] +} + + +-- !query +SELECT udf(a), COUNT(udf(1)) FROM testData WHERE false GROUP BY udf(a) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(COUNT(1)) FROM testData WHERE false +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT 1 FROM (SELECT udf(COUNT(1)) FROM testData WHERE false) t +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT 1 from ( + SELECT 1 AS z, + udf(MIN(a.x)) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z +-- !query schema +struct<1:int> +-- !query output + + + +-- !query +SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) +-- !query schema +struct +-- !query output +0.9999999999999999 0.9999999999999999 3 + + +-- !query +SELECT udf(1) FROM range(10) HAVING true +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT udf(udf(1)) FROM range(10) HAVING MAX(id) > 0 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT udf(id) FROM range(10) HAVING id > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 43, + "fragment" : "SELECT udf(id) FROM range(10) HAVING id > 0" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0 +-- !query schema +struct +-- !query output +NULL NULL NULL + + +-- !query +SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4 +-- !query schema +struct +-- !query output +NULL NULL NULL + + +-- !query +SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5 +-- !query schema +struct +-- !query output +false true true + + +-- !query +SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k) +-- !query schema +struct +-- !query output +1 false true true +2 true true true +3 false false false +4 NULL NULL NULL +5 false true true + + +-- !query +SELECT udf(k), every(v) FROM test_agg GROUP BY k HAVING every(v) = false +-- !query schema +struct +-- !query output +1 false +3 false +5 false + + +-- !query +SELECT udf(k), udf(every(v)) FROM test_agg GROUP BY udf(k) HAVING every(v) IS NULL +-- !query schema +struct +-- !query output +4 NULL + + +-- !query +SELECT udf(k), + udf(Every(v)) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY udf(k) +-- !query schema +struct +-- !query output +2 true + + +-- !query +SELECT udf(udf(k)), + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY udf(udf(k)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT every(udf(1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"udf(1)\"", + "inputType" : "\"INT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"every(udf(1))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "every(udf(1))" + } ] +} + + +-- !query +SELECT some(udf(1S)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"udf(1)\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"some(udf(1))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "some(udf(1S))" + } ] +} + + +-- !query +SELECT any(udf(1L)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"udf(1)\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"any(udf(1))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "any(udf(1L))" + } ] +} + + +-- !query +SELECT udf(every("true")) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"true\"", + "inputType" : "\"STRING\"", + "paramIndex" : "1", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"every(true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 24, + "fragment" : "every(\"true\")" + } ] +} + + +-- !query +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true false +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true false + + +-- !query +SELECT k, udf(udf(v)), some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT udf(udf(k)), v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT udf(count(*)) FROM test_agg HAVING count(*) > 1L +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT k, udf(max(v)) FROM test_agg GROUP BY k HAVING max(v) = true +-- !query schema +struct +-- !query output +1 true +2 true +5 true + + +-- !query +SELECT * FROM (SELECT udf(COUNT(*)) AS cnt FROM test_agg) WHERE cnt > 1L +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(count(1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L" + } ] +} + + +-- !query +SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"((count(1) + 1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 59, + "fragment" : "SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L" + } ] +} + + +-- !query +SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(((k = 1) OR (k = 2)) OR (((count(1) + 1) > 1) OR (max(k) > 1)))\"", + "expressionList" : "count(1), max(test_agg.k)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 91, + "fragment" : "SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/array.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/array.sql new file mode 100644 index 000000000000..865dc8bac4ea --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/array.sql @@ -0,0 +1,183 @@ +-- test cases for array functions + +create temporary view data as select * from values + ("one", array(11, 12, 13), array(array(111, 112, 113), array(121, 122, 123))), + ("two", array(21, 22, 23), array(array(211, 212, 213), array(221, 222, 223))) + as data(a, b, c); + +select * from data; + +-- index into array +select a, b[0], b[0] + b[1] from data; + +-- index into array of arrays +select a, c[0][0] + c[0][0 + 1] from data; + + +create temporary view primitive_arrays as select * from values ( + array(true), + array(2Y, 1Y), + array(2S, 1S), + array(2, 1), + array(2L, 1L), + array(9223372036854775809, 9223372036854775808), + array(2.0D, 1.0D), + array(float(2.0), float(1.0)), + array(date '2016-03-14', date '2016-03-13'), + array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000') +) as primitive_arrays( + boolean_array, + tinyint_array, + smallint_array, + int_array, + bigint_array, + decimal_array, + double_array, + float_array, + date_array, + timestamp_array +); + +select * from primitive_arrays; + +-- array_contains on all primitive types: result should alternate between true and false +select + array_contains(boolean_array, true), array_contains(boolean_array, false), + array_contains(tinyint_array, 2Y), array_contains(tinyint_array, 0Y), + array_contains(smallint_array, 2S), array_contains(smallint_array, 0S), + array_contains(int_array, 2), array_contains(int_array, 0), + array_contains(bigint_array, 2L), array_contains(bigint_array, 0L), + array_contains(decimal_array, 9223372036854775809), array_contains(decimal_array, 1), + array_contains(double_array, 2.0D), array_contains(double_array, 0.0D), + array_contains(float_array, float(2.0)), array_contains(float_array, float(0.0)), + array_contains(date_array, date '2016-03-14'), array_contains(date_array, date '2016-01-01'), + array_contains(timestamp_array, timestamp '2016-11-15 20:54:00.000'), array_contains(timestamp_array, timestamp '2016-01-01 20:54:00.000') +from primitive_arrays; + +-- array_contains on nested arrays +select array_contains(b, 11), array_contains(c, array(111, 112, 113)) from data; + +-- sort_array +select + sort_array(boolean_array), + sort_array(tinyint_array), + sort_array(smallint_array), + sort_array(int_array), + sort_array(bigint_array), + sort_array(decimal_array), + sort_array(double_array), + sort_array(float_array), + sort_array(date_array), + sort_array(timestamp_array) +from primitive_arrays; + +-- sort_array with an invalid string literal for the argument of sort order. +select sort_array(array('b', 'd'), '1'); + +-- sort_array with an invalid null literal casted as boolean for the argument of sort order. +select sort_array(array('b', 'd'), cast(NULL as boolean)); + +-- size +select + size(boolean_array), + size(tinyint_array), + size(smallint_array), + size(int_array), + size(bigint_array), + size(decimal_array), + size(double_array), + size(float_array), + size(date_array), + size(timestamp_array) +from primitive_arrays; + +-- index out of range for array elements +select element_at(array(1, 2, 3), 5); +select element_at(array(1, 2, 3), -5); +select element_at(array(1, 2, 3), 0); + +select elt(4, '123', '456'); +select elt(0, '123', '456'); +select elt(-1, '123', '456'); +select elt(null, '123', '456'); +select elt(null, '123', null); +select elt(1, '123', null); +select elt(2, '123', null); + +select array(1, 2, 3)[5]; +select array(1, 2, 3)[-1]; + +-- array_size +select array_size(array()); +select array_size(array(true)); +select array_size(array(2, 1)); +select array_size(NULL); +select array_size(map('a', 1, 'b', 2)); + +-- size(arrays_zip) +select size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10))); +select size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10))); +select size(arrays_zip(array(1, 2, 3), array(4), null, array(7, 8, 9, 10))); + +-- isnotnull(arrays_zip) +select isnotnull(arrays_zip(array(), array(4), array(7, 8, 9, 10))); +select isnotnull(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10))); +select isnotnull(arrays_zip(array(1, 2, 3), NULL, array(4), array(7, 8, 9, 10))); + +-- function get() +select get(array(1, 2, 3), 0); +select get(array(1, 2, 3), 3); +select get(array(1, 2, 3), null); +select get(array(1, 2, 3), -1); + +-- function array_insert() +select array_insert(array(1, 2, 3), 3, 4); +select array_insert(array(2, 3, 4), 0, 1); +select array_insert(array(2, 3, 4), 1, 1); +select array_insert(array(1, 3, 4), -2, 2); +select array_insert(array(1, 2, 3), 3, "4"); +select array_insert(cast(NULL as ARRAY), 1, 1); +select array_insert(array(1, 2, 3, NULL), cast(NULL as INT), 4); +select array_insert(array(1, 2, 3, NULL), 4, cast(NULL as INT)); +select array_insert(array(2, 3, NULL, 4), 5, 5); +select array_insert(array(2, 3, NULL, 4), -5, 1); +select array_insert(array(1), 2, cast(2 as tinyint)); + +set spark.sql.legacy.negativeIndexInArrayInsert=true; +select array_insert(array(1, 3, 4), -2, 2); +select array_insert(array(2, 3, NULL, 4), -5, 1); +set spark.sql.legacy.negativeIndexInArrayInsert=false; + +-- function array_compact +select array_compact(id) from values (1) as t(id); +select array_compact(array("1", null, "2", null)); +select array_compact(array("a", "b", "c")); +select array_compact(array(1D, null, 2D, null)); +select array_compact(array(array(1, 2, 3, null), null, array(4, null, 6))); +select array_compact(array(null)); + +-- function array_append +select array_append(array(1, 2, 3), 4); +select array_append(array('a', 'b', 'c'), 'd'); +select array_append(array(1, 2, 3, NULL), NULL); +select array_append(array('a', 'b', 'c', NULL), NULL); +select array_append(CAST(null AS ARRAY), 'a'); +select array_append(CAST(null AS ARRAY), CAST(null as String)); +select array_append(array(), 1); +select array_append(CAST(array() AS ARRAY), CAST(NULL AS String)); +select array_append(array(CAST(NULL AS String)), CAST(NULL AS String)); + +-- function array_prepend +select array_prepend(array(1, 2, 3), 4); +select array_prepend(array('a', 'b', 'c'), 'd'); +select array_prepend(array(1, 2, 3, NULL), NULL); +select array_prepend(array('a', 'b', 'c', NULL), NULL); +select array_prepend(CAST(null AS ARRAY), 'a'); +select array_prepend(CAST(null AS ARRAY), CAST(null as String)); +select array_prepend(array(), 1); +select array_prepend(CAST(array() AS ARRAY), CAST(NULL AS String)); +select array_prepend(array(CAST(NULL AS String)), CAST(NULL AS String)); + +-- SPARK-45599: Confirm 0.0, -0.0, and NaN are handled appropriately. +select array_union(array(0.0, -0.0, DOUBLE("NaN")), array(0.0, -0.0, DOUBLE("NaN"))); +select array_distinct(array(0.0, -0.0, -0.0, DOUBLE("NaN"), DOUBLE("NaN"))); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/binary.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/binary.sql new file mode 100644 index 000000000000..8da97e466341 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/binary.sql @@ -0,0 +1,8 @@ +--SET spark.sql.binaryOutputStyle=UTF-8 + +SELECT X''; +SELECT X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333'; +SELECT CAST('Spark' as BINARY); +SELECT array( X'', X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333', CAST('Spark' as BINARY)); +SELECT to_csv(named_struct('n', 1, 'info', X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333')); +select to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop'))); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/binary_base64.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/binary_base64.sql new file mode 100644 index 000000000000..853eedd51773 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/binary_base64.sql @@ -0,0 +1,3 @@ +--IMPORT binary.sql + +--SET spark.sql.binaryOutputStyle=BASE64 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/binary_basic.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/binary_basic.sql new file mode 100644 index 000000000000..1a5b64bdf7e0 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/binary_basic.sql @@ -0,0 +1,4 @@ +--IMPORT binary.sql + +--SET spark.sql.binaryOutputStyle=BASIC + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/binary_hex.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/binary_hex.sql new file mode 100644 index 000000000000..7863da737a72 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/binary_hex.sql @@ -0,0 +1,3 @@ +--IMPORT binary.sql + +--SET spark.sql.binaryOutputStyle=HEX diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/binary_hex_discrete.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/binary_hex_discrete.sql new file mode 100644 index 000000000000..282a7634cbc5 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/binary_hex_discrete.sql @@ -0,0 +1,3 @@ +--IMPORT binary.sql + +--SET spark.sql.binaryOutputStyle=HEX_DISCRETE diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/bitwise.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/bitwise.sql new file mode 100644 index 000000000000..e080fdd32a4a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/bitwise.sql @@ -0,0 +1,93 @@ +-- test cases for bitwise functions + +-- null +select bit_count(null); + +-- boolean +select bit_count(true); +select bit_count(false); + +-- byte/tinyint +select bit_count(cast(1 as tinyint)); +select bit_count(cast(2 as tinyint)); +select bit_count(cast(3 as tinyint)); + +-- short/smallint +select bit_count(1S); +select bit_count(2S); +select bit_count(3S); + +-- int +select bit_count(1); +select bit_count(2); +select bit_count(3); + +-- long/bigint +select bit_count(1L); +select bit_count(2L); +select bit_count(3L); + +-- negative num +select bit_count(-1L); + +-- edge value +select bit_count(9223372036854775807L); +select bit_count(-9223372036854775808L); + +-- other illegal arguments +select bit_count("bit count"); +select bit_count('a'); + +-- test for bit_xor +-- +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (2, 3, 4, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4); + +-- empty case +SELECT BIT_XOR(b3) AS n1 FROM bitwise_test where 1 = 0; + +-- null case +SELECT BIT_XOR(b4) AS n1 FROM bitwise_test where b4 is null; + +-- the suffix numbers show the expected answer +SELECT + BIT_XOR(cast(b1 as tinyint)) AS a4, + BIT_XOR(cast(b2 as smallint)) AS b5, + BIT_XOR(b3) AS c2, + BIT_XOR(b4) AS d2, + BIT_XOR(distinct b4) AS e2 +FROM bitwise_test; + +-- group by +SELECT bit_xor(b3) FROM bitwise_test GROUP BY b1 & 1; + +--having +SELECT b1, bit_xor(b2) FROM bitwise_test GROUP BY b1 HAVING bit_and(b2) < 7; + +-- window +SELECT b1, b2, bit_xor(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test; + +-- getbit +select getbit(11L, 3), getbit(11L, 2), getbit(11L, 1), getbit(11L, 0); +select getbit(11L, 2 + 1), getbit(11L, 3 - 1), getbit(10L + 1, 1 * 1), getbit(cast(11L / 1 AS long), 1 - 1); +select getbit(11L, 63); +select getbit(11L, -1); +select getbit(11L, 64); + +SELECT 20181117 >> 2; +SELECT 20181117 << 2; +SELECT 20181117 >>> 2; +SELECT 20181117 > > 2; +SELECT 20181117 < < 2; +SELECT 20181117 > >> 2; +SELECT 20181117 <<< 2; +SELECT 20181117 >>>> 2; +select cast(null as array>), 20181117 >> 2; +select cast(null as array>), 20181117 >>> 2; +select cast(null as map>), 20181117 >> 2; + +select 1 << 1 + 2 as plus_over_shift; -- if correct, the result is 8. otherwise, 4 +select 2 >> 1 << 1 as left_to_right; -- if correct, the result is 2. otherwise, 0 +select 1 & 2 >> 1 as shift_over_ampersand; -- if correct, the result is 1. otherwise, 0 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cast.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cast.sql new file mode 100644 index 000000000000..2bf53f9730b3 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cast.sql @@ -0,0 +1,179 @@ +-- cast string representing a valid fractional number to integral should truncate the number +SELECT CAST('1.23' AS int); +SELECT CAST('1.23' AS long); +SELECT CAST('-4.56' AS int); +SELECT CAST('-4.56' AS long); + +-- cast string which are not numbers to numeric types +SELECT CAST('abc' AS int); +SELECT CAST('abc' AS long); +SELECT CAST('abc' AS float); +SELECT CAST('abc' AS double); + +-- cast string representing a very large number to integral should return null +SELECT CAST('1234567890123' AS int); +SELECT CAST('12345678901234567890123' AS long); + +-- cast empty string to integral should return null +SELECT CAST('' AS int); +SELECT CAST('' AS long); +SELECT CAST('' AS float); +SELECT CAST('' AS double); + +-- cast null to integral should return null +SELECT CAST(NULL AS int); +SELECT CAST(NULL AS long); + +-- cast invalid decimal string to numeric types +SELECT CAST('123.a' AS int); +SELECT CAST('123.a' AS long); +SELECT CAST('123.a' AS float); +SELECT CAST('123.a' AS double); + +-- '-2147483648' is the smallest int value +SELECT CAST('-2147483648' AS int); +SELECT CAST('-2147483649' AS int); + +-- '2147483647' is the largest int value +SELECT CAST('2147483647' AS int); +SELECT CAST('2147483648' AS int); + +-- '-9223372036854775808' is the smallest long value +SELECT CAST('-9223372036854775808' AS long); +SELECT CAST('-9223372036854775809' AS long); + +-- '9223372036854775807' is the largest long value +SELECT CAST('9223372036854775807' AS long); +SELECT CAST('9223372036854775808' AS long); + +-- cast string to its binary representation +SELECT HEX(CAST('abc' AS binary)); + +-- cast integral values to their corresponding binary representation +SELECT HEX(CAST(CAST(123 AS byte) AS binary)); +SELECT HEX(CAST(CAST(-123 AS byte) AS binary)); +SELECT HEX(CAST(123S AS binary)); +SELECT HEX(CAST(-123S AS binary)); +SELECT HEX(CAST(123 AS binary)); +SELECT HEX(CAST(-123 AS binary)); +SELECT HEX(CAST(123L AS binary)); +SELECT HEX(CAST(-123L AS binary)); + +DESC FUNCTION boolean; +DESC FUNCTION EXTENDED boolean; +-- TODO: migrate all cast tests here. + +-- cast string to interval and interval to string +SELECT CAST('interval 3 month 1 hour' AS interval); +SELECT CAST("interval '3-1' year to month" AS interval year to month); +SELECT CAST("interval '3 00:00:01' day to second" AS interval day to second); +SELECT CAST(interval 3 month 1 hour AS string); +SELECT CAST(interval 3 year 1 month AS string); +SELECT CAST(interval 3 day 1 second AS string); + +-- trim string before cast to numeric +select cast(' 1' as tinyint); +select cast(' 1\t' as tinyint); +select cast(' 1' as smallint); +select cast(' 1' as INT); +select cast(' 1' as bigint); +select cast(' 1' as float); +select cast(' 1 ' as DOUBLE); +select cast('1.0 ' as DEC); +select cast('1中文' as tinyint); +select cast('1中文' as smallint); +select cast('1中文' as INT); +select cast('中文1' as bigint); +select cast('1中文' as bigint); + +-- trim string before cast to boolean +select cast('\t\t true \n\r ' as boolean); +select cast('\t\n false \t\r' as boolean); +select cast('\t\n xyz \t\r' as boolean); + +select cast('23.45' as decimal(4, 2)); +select cast('123.45' as decimal(4, 2)); +select cast('xyz' as decimal(4, 2)); + +select cast('2022-01-01' as date); +select cast('a' as date); +select cast('2022-01-01 00:00:00' as timestamp); +select cast('a' as timestamp); +select cast('2022-01-01 00:00:00' as timestamp_ntz); +select cast('a' as timestamp_ntz); + +select cast(cast('inf' as double) as timestamp); +select cast(cast('inf' as float) as timestamp); + +-- cast ANSI intervals to integrals +select cast(interval '1' year as tinyint); +select cast(interval '-10-2' year to month as smallint); +select cast(interval '1000' month as int); +select cast(interval -'10.123456' second as tinyint); +select cast(interval '23:59:59' hour to second as smallint); +select cast(interval -'1 02:03:04.123' day to second as int); +select cast(interval '10' day as bigint); + +select cast(interval '-1000' month as tinyint); +select cast(interval '1000000' second as smallint); + +-- cast integrals to ANSI intervals +select cast(1Y as interval year); +select cast(-122S as interval year to month); +select cast(ym as interval year to month) from values(-122S) as t(ym); +select cast(1000 as interval month); +select cast(-10L as interval second); +select cast(100Y as interval hour to second); +select cast(dt as interval hour to second) from values(100Y) as t(dt); +select cast(-1000S as interval day to second); +select cast(10 as interval day); + +select cast(2147483647 as interval year); +select cast(-9223372036854775808L as interval day); + +-- cast ANSI intervals to decimals +select cast(interval '-1' year as decimal(10, 0)); +select cast(interval '1.000001' second as decimal(10, 6)); +select cast(interval '08:11:10.001' hour to second as decimal(10, 4)); +select cast(interval '1 01:02:03.1' day to second as decimal(8, 1)); +select cast(interval '10.123' second as decimal(4, 2)); +select cast(interval '10.005' second as decimal(4, 2)); +select cast(interval '10.123' second as decimal(5, 2)); +select cast(interval '10.123' second as decimal(1, 0)); + +-- cast decimals to ANSI intervals +select cast(10.123456BD as interval day to second); +select cast(80.654321BD as interval hour to minute); +select cast(-10.123456BD as interval year to month); +select cast(10.654321BD as interval month); + +-- cast double colon syntax tests +SELECT '1.23' :: int; +SELECT 'abc' :: int; +SELECT '12345678901234567890123' :: long; +SELECT '' :: int; +SELECT NULL :: int; +SELECT '123.a' :: int; +SELECT '-2147483648' :: int; +SELECT HEX('abc' :: binary); +SELECT HEX((123 :: byte) :: binary); +SELECT 'interval 3 month 1 hour' :: interval; +SELECT interval 3 day 1 second :: string; +select ' 1 ' :: DOUBLE; +select '1.0 ' :: DEC; +select '\t\t true \n\r ' :: boolean; +select '2022-01-01 00:00:00' :: timestamp; +select interval '-10-2' year to month :: smallint; +select -10L :: interval second; +select interval '08:11:10.001' hour to second :: decimal(10, 4); +select 10.123456BD :: interval day to second; + +SELECT '1.23' :: int :: long; +SELECT '2147483648' :: long :: int; +SELECT CAST('2147483648' :: long AS int); +SELECT map(1, '123', 2, '456')[1] :: int; + +-- cast double colon syntax negative tests +SELECT '2147483648' :: BINT; +SELECT '2147483648' :: SELECT; +SELECT FALSE IS NOT NULL :: string; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/ceil-floor-with-scale-param.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/ceil-floor-with-scale-param.sql new file mode 100644 index 000000000000..c05429b3ef77 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/ceil-floor-with-scale-param.sql @@ -0,0 +1,31 @@ +-- Tests different scenarios of ceil and floor functions with scale parameters +SELECT CEIL(2.5, 0); +SELECT CEIL(3.5, 0); +SELECT CEIL(-2.5, 0); +SELECT CEIL(-3.5, 0); +SELECT CEIL(-0.35, 1); +SELECT CEIL(-35, -1); +SELECT CEIL(-0.1, 0); +SELECT CEIL(5, 0); +SELECT CEIL(3.14115, -3); +SELECT CEIL(9.9, 0); +SELECT CEIL(CAST(99 AS DECIMAL(2, 0)), -1); +SELECT CEIL(2.5, null); +SELECT CEIL(2.5, 'a'); +SELECT CEIL(2.5, 0, 0); + +-- Same inputs with floor function +SELECT FLOOR(2.5, 0); +SELECT FLOOR(3.5, 0); +SELECT FLOOR(-2.5, 0); +SELECT FLOOR(-3.5, 0); +SELECT FLOOR(-0.35, 1); +SELECT FLOOR(-35, -1); +SELECT FLOOR(-0.1, 0); +SELECT FLOOR(5, 0); +SELECT FLOOR(3.14115, -3); +SELECT FLOOR(-9.9, 0); +SELECT FLOOR(CAST(-99 AS DECIMAL(2, 0)), -1); +SELECT FLOOR(2.5, null); +SELECT FLOOR(2.5, 'a'); +SELECT FLOOR(2.5, 0, 0); \ No newline at end of file diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/change-column.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/change-column.sql new file mode 100644 index 000000000000..2b57891cfcbc --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/change-column.sql @@ -0,0 +1,49 @@ +-- Create the origin table +CREATE TABLE test_change(a INT, b STRING, c INT) using parquet; +DESC test_change; + +-- ALTER TABLE CHANGE COLUMN must change either type or comment +ALTER TABLE test_change CHANGE a; +DESC test_change; + +-- Change column name (not supported on v1 table) +ALTER TABLE test_change RENAME COLUMN a TO a1; +DESC test_change; + +-- Change column dataType (not supported yet) +ALTER TABLE test_change CHANGE a TYPE STRING; +DESC test_change; + +-- Change column position (not supported yet) +ALTER TABLE test_change CHANGE a AFTER b; +ALTER TABLE test_change CHANGE b FIRST; +DESC test_change; + +-- Change column comment +ALTER TABLE test_change CHANGE a COMMENT 'this is column a'; +ALTER TABLE test_change CHANGE b COMMENT '#*02?`'; +ALTER TABLE test_change CHANGE c COMMENT ''; +DESC test_change; + +-- Don't change anything. +ALTER TABLE test_change CHANGE a TYPE INT; +ALTER TABLE test_change CHANGE a COMMENT 'this is column a'; +DESC test_change; + +-- Change a invalid column +ALTER TABLE test_change CHANGE invalid_col TYPE INT; +DESC test_change; + +-- Check case insensitivity. +ALTER TABLE test_change CHANGE A COMMENT 'case insensitivity'; +DESC test_change; + +-- Change column can't apply to a temporary/global_temporary view +CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one"; +ALTER TABLE temp_view CHANGE a TYPE INT; +CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one"; +ALTER TABLE global_temp.global_temp_view CHANGE a TYPE INT; + +-- DROP TEST TABLE +DROP TABLE test_change; +DROP VIEW global_temp.global_temp_view; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/charvarchar.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/charvarchar.sql new file mode 100644 index 000000000000..f5b20f99c8a0 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/charvarchar.sql @@ -0,0 +1,128 @@ +create table char_tbl(c char(5), v varchar(6)) using parquet; +desc formatted char_tbl; +desc formatted char_tbl c; +show create table char_tbl; + +create table char_tbl2 using parquet as select * from char_tbl; +show create table char_tbl2; +desc formatted char_tbl2; +desc formatted char_tbl2 c; + +create table char_tbl3 like char_tbl; +desc formatted char_tbl3; +desc formatted char_tbl3 c; +show create table char_tbl3; + +create view char_view as select * from char_tbl; +desc formatted char_view; +desc formatted char_view c; +show create table char_view; + +alter table char_tbl rename to char_tbl1; +desc formatted char_tbl1; + +alter table char_tbl1 change column c type char(6); +alter table char_tbl1 change column c type char(5); +desc formatted char_tbl1; + +alter table char_tbl1 add columns (d char(5)); +desc formatted char_tbl1; + +alter view char_view as select * from char_tbl2; +desc formatted char_view; + +alter table char_tbl1 SET TBLPROPERTIES('yes'='no'); +desc formatted char_tbl1; + +alter view char_view SET TBLPROPERTIES('yes'='no'); +desc formatted char_view; + +alter table char_tbl1 UNSET TBLPROPERTIES('yes'); +desc formatted char_tbl1; + +alter view char_view UNSET TBLPROPERTIES('yes'); +desc formatted char_view; + +alter table char_tbl1 SET SERDEPROPERTIES('yes'='no'); +desc formatted char_tbl1; + +create table char_part(c1 char(5), c2 char(2), v1 varchar(6), v2 varchar(2)) using parquet partitioned by (v2, c2); +desc formatted char_part; + +alter table char_part change column c1 comment 'char comment'; +alter table char_part change column v1 comment 'varchar comment'; +alter table char_part add partition (v2='ke', c2='nt') location 'loc1'; +desc formatted char_part; + +alter table char_part partition (v2='ke') rename to partition (v2='nt'); +desc formatted char_part; + +alter table char_part partition (v2='ke', c2='nt') set location 'loc2'; +desc formatted char_part; + +MSCK REPAIR TABLE char_part; +desc formatted char_part; + +create temporary view str_view as select c, v from values + (null, null), + (null, 'S'), + ('N', 'N '), + ('Ne', 'Sp'), + ('Net ', 'Spa '), + ('NetE', 'Spar'), + ('NetEa ', 'Spark '), + ('NetEas ', 'Spark'), + ('NetEase', 'Spark-') t(c, v); + +create table char_tbl4(c7 char(7), c8 char(8), v varchar(6), s string) using parquet; +insert into char_tbl4 select c, c, v, c from str_view; + +select c7, c8, v, s from char_tbl4; +select c7, c8, v, s from char_tbl4 where c7 = c8; +select c7, c8, v, s from char_tbl4 where c7 = v; +select c7, c8, v, s from char_tbl4 where c7 = s; +select c7, c8, v, s from char_tbl4 where c7 = 'NetEase '; +select c7, c8, v, s from char_tbl4 where v = 'Spark '; +select c7, c8, v, s from char_tbl4 order by c7; +select c7, c8, v, s from char_tbl4 order by v; + +select ascii(c7), ascii(c8), ascii(v), ascii(s) from char_tbl4; +select base64(c7), base64(c8), base64(v), ascii(s) from char_tbl4; +select bit_length(c7), bit_length(c8), bit_length(v), bit_length(s) from char_tbl4; +select char_length(c7), char_length(c8), char_length(v), char_length(s) from char_tbl4; +select octet_length(c7), octet_length(c8), octet_length(v), octet_length(s) from char_tbl4; +select concat_ws('|', c7, c8), concat_ws('|', c7, v), concat_ws('|', c7, s), concat_ws('|', v, s) from char_tbl4; +select concat(c7, c8), concat(c7, v), concat(c7, s), concat(v, s) from char_tbl4; +select like(c7, 'Ne _'), like(c8, 'Ne _') from char_tbl4; +select like(v, 'Spark_') from char_tbl4; +select c7 = c8, upper(c7) = upper(c8), lower(c7) = lower(c8) from char_tbl4 where s = 'NetEase'; +select c7 = s, upper(c7) = upper(s), lower(c7) = lower(s) from char_tbl4 where s = 'NetEase'; +select c7 = 'NetEase', upper(c7) = upper('NetEase'), lower(c7) = lower('NetEase') from char_tbl4 where s = 'NetEase'; +select printf('Hey, %s%s%s%s', c7, c8, v, s) from char_tbl4; +select repeat(c7, 2), repeat(c8, 2), repeat(v, 2), repeat(s, 2) from char_tbl4; +select replace(c7, 'Net', 'Apache'), replace(c8, 'Net', 'Apache'), replace(v, 'Spark', 'Kyuubi'), replace(s, 'Net', 'Apache') from char_tbl4; +select rpad(c7, 10), rpad(c8, 5), rpad(v, 5), rpad(s, 5) from char_tbl4; +select rtrim(c7), rtrim(c8), rtrim(v), rtrim(s) from char_tbl4; +select split(c7, 'e'), split(c8, 'e'), split(v, 'a'), split(s, 'e') from char_tbl4; +select substring(c7, 2), substring(c8, 2), substring(v, 3), substring(s, 2) from char_tbl4; +select left(c7, 2), left(c8, 2), left(v, 3), left(s, 2) from char_tbl4; +select right(c7, 2), right(c8, 2), right(v, 3), right(s, 2) from char_tbl4; +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; +select typeof(c7), typeof(c8), typeof(v), typeof(s) from char_tbl4 limit 1; +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; +select cast(c7 as char(1)), cast(c8 as char(10)), cast(v as char(1)), cast(v as varchar(1)), cast(s as char(5)) from char_tbl4; + +-- char_tbl has renamed to char_tbl1 +drop table char_tbl1; +drop table char_tbl2; +drop table char_tbl3; +drop table char_tbl4; + +-- ascii value for Latin-1 Supplement characters +select ascii('§'), ascii('÷'), ascii('×10'); +select chr(167), chr(247), chr(215); + +-- to_varchar is an alias for to_char +SELECT to_varchar(78.12, '$99.99'); +SELECT to_varchar(111.11, '99.9'); +SELECT to_varchar(12454.8, '99,999.9S'); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/collations.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/collations.sql new file mode 100644 index 000000000000..17815ed5dde6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/collations.sql @@ -0,0 +1,527 @@ +-- test cases for collation support + +-- Create a test table with data +create table t1(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet; +insert into t1 values('aaa', 'aaa'); +insert into t1 values('AAA', 'AAA'); +insert into t1 values('bbb', 'bbb'); +insert into t1 values('BBB', 'BBB'); + +-- describe +describe table t1; + +-- group by and count utf8_binary +select count(*) from t1 group by utf8_binary; + +-- group by and count utf8_lcase +select count(*) from t1 group by utf8_lcase; + +-- filter equal utf8_binary +select * from t1 where utf8_binary = 'aaa'; + +-- filter equal utf8_lcase +select * from t1 where utf8_lcase = 'aaa' collate utf8_lcase; + +-- filter less then utf8_binary +select * from t1 where utf8_binary < 'bbb'; + +-- filter less then utf8_lcase +select * from t1 where utf8_lcase < 'bbb' collate utf8_lcase; + +-- inner join +select l.utf8_binary, r.utf8_lcase from t1 l join t1 r on l.utf8_lcase = r.utf8_lcase; + +-- create second table for anti-join +create table t2(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet; +insert into t2 values('aaa', 'aaa'); +insert into t2 values('bbb', 'bbb'); + +-- anti-join on lcase +select * from t1 anti join t2 on t1.utf8_lcase = t2.utf8_lcase; + +drop table t2; +drop table t1; + +-- set operations +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate utf8_lcase from values ('aaa'), ('bbb'); +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate utf8_lcase from values ('aaa'), ('bbb'); +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate utf8_lcase from values ('aaa'), ('bbb'); +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate utf8_lcase from values ('aaa'), ('bbb'); +select col1 collate utf8_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate utf8_lcase from values ('aaa'), ('bbb'); + +-- set operations with conflicting collations +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate unicode_ci from values ('aaa'), ('bbb'); +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate unicode_ci from values ('aaa'), ('bbb'); +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate unicode_ci from values ('aaa'), ('bbb'); +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate unicode_ci from values ('aaa'), ('bbb'); +select col1 collate utf8_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate unicode_ci from values ('aaa'), ('bbb'); + +-- create table with struct field +create table t1 (c1 struct) USING PARQUET; + +insert into t1 values (named_struct('utf8_binary', 'aaa', 'utf8_lcase', 'aaa')); +insert into t1 values (named_struct('utf8_binary', 'AAA', 'utf8_lcase', 'AAA')); + +-- aggregate against nested field utf8_binary +select count(*) from t1 group by c1.utf8_binary; + +-- aggregate against nested field utf8_lcase +select count(*) from t1 group by c1.utf8_lcase; + +drop table t1; + +-- array function tests +select array_contains(ARRAY('aaa' collate utf8_lcase),'AAA' collate utf8_lcase); +select array_position(ARRAY('aaa' collate utf8_lcase, 'bbb' collate utf8_lcase),'BBB' collate utf8_lcase); + +-- utility +select nullif('aaa' COLLATE utf8_lcase, 'AAA' COLLATE utf8_lcase); +select least('aaa' COLLATE utf8_lcase, 'AAA' collate utf8_lcase, 'a' collate utf8_lcase); + +-- array operations +select arrays_overlap(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)); +select array_distinct(array('aaa' collate utf8_lcase, 'AAA' collate utf8_lcase)); +select array_union(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)); +select array_intersect(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)); +select array_except(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)); + +-- ICU collations (all statements return true) +select 'a' collate unicode < 'A'; +select 'a' collate unicode_ci = 'A'; +select 'a' collate unicode_ai = 'å'; +select 'a' collate unicode_ci_ai = 'Å'; +select 'a' collate en < 'A'; +select 'a' collate en_ci = 'A'; +select 'a' collate en_ai = 'å'; +select 'a' collate en_ci_ai = 'Å'; +select 'Kypper' collate sv < 'Köpfe'; +select 'Kypper' collate de > 'Köpfe'; +select 'I' collate tr_ci = 'ı'; + +-- create table for str_to_map +create table t4 (text string collate utf8_binary, pairDelim string collate utf8_lcase, keyValueDelim string collate utf8_binary) using parquet; + +insert into t4 values('a:1,b:2,c:3', ',', ':'); + +select str_to_map(text, pairDelim, keyValueDelim) from t4; +select str_to_map(text collate utf8_binary, pairDelim collate utf8_lcase, keyValueDelim collate utf8_binary) from t4; +select str_to_map(text collate utf8_binary, pairDelim collate utf8_binary, keyValueDelim collate utf8_binary) from t4; +select str_to_map(text collate unicode_ai, pairDelim collate unicode_ai, keyValueDelim collate unicode_ai) from t4; + +drop table t4; + +create table t5(s string, utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet; +insert into t5 values ('Spark', 'Spark', 'SQL'); +insert into t5 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaAAaA'); +insert into t5 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaA'); +insert into t5 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaAaaAaaAaAaaAaaAaA'); +insert into t5 values ('bbAbaAbA', 'bbAbAAbA', 'a'); +insert into t5 values ('İo', 'İo', 'İo'); +insert into t5 values ('İo', 'İo', 'İo '); +insert into t5 values ('İo', 'İo ', 'İo'); +insert into t5 values ('İo', 'İo', 'i̇o'); +insert into t5 values ('efd2', 'efd2', 'efd2'); +insert into t5 values ('Hello, world! Nice day.', 'Hello, world! Nice day.', 'Hello, world! Nice day.'); +insert into t5 values ('Something else. Nothing here.', 'Something else. Nothing here.', 'Something else. Nothing here.'); +insert into t5 values ('kitten', 'kitten', 'sitTing'); +insert into t5 values ('abc', 'abc', 'abc'); +insert into t5 values ('abcdcba', 'abcdcba', 'aBcDCbA'); + +create table t6(ascii long) using parquet; +insert into t6 values (97); +insert into t6 values (66); + +create table t7(ascii double) using parquet; +insert into t7 values (97.52143); +insert into t7 values (66.421); + +create table t8(format string collate utf8_binary, utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet; +insert into t8 values ('%s%s', 'abCdE', 'abCdE'); + +create table t9(num long) using parquet; +insert into t9 values (97); +insert into t9 values (66); + +create table t10(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet; +insert into t10 values ('aaAaAAaA', 'aaAaaAaA'); +insert into t10 values ('efd2', 'efd2'); + +-- ConcatWs +select concat_ws(' ', utf8_lcase, utf8_lcase) from t5; +select concat_ws(' ', utf8_binary, utf8_lcase) from t5; +select concat_ws(' ' collate utf8_binary, utf8_binary, 'SQL' collate utf8_lcase) from t5; +select concat_ws(' ' collate utf8_lcase, utf8_binary, 'SQL' collate utf8_lcase) from t5; +select concat_ws(',', utf8_lcase, 'word'), concat_ws(',', utf8_binary, 'word') from t5; +select concat_ws(',', utf8_lcase, 'word' collate utf8_binary), concat_ws(',', utf8_binary, 'word' collate utf8_lcase) from t5; + +-- Elt +select elt(2, s, utf8_binary) from t5; +select elt(2, utf8_binary, utf8_lcase, s) from t5; +select elt(1, utf8_binary collate utf8_binary, utf8_lcase collate utf8_lcase) from t5; +select elt(1, utf8_binary collate utf8_binary, utf8_lcase collate utf8_binary) from t5; +select elt(1, utf8_binary collate utf8_binary, utf8_lcase) from t5; +select elt(1, utf8_binary, 'word'), elt(1, utf8_lcase, 'word') from t5; +select elt(1, utf8_binary, 'word' collate utf8_lcase), elt(1, utf8_lcase, 'word' collate utf8_binary) from t5; + +-- SplitPart +select split_part(utf8_binary, utf8_lcase, 3) from t5; +select split_part(s, utf8_binary, 1) from t5; +select split_part(utf8_binary collate utf8_binary, s collate utf8_lcase, 1) from t5; +select split_part(utf8_binary, utf8_lcase collate utf8_binary, 2) from t5; +select split_part(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t5; +select split_part(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 2) from t5; +select split_part(utf8_binary, 'a', 3), split_part(utf8_lcase, 'a', 3) from t5; +select split_part(utf8_binary, 'a' collate utf8_lcase, 3), split_part(utf8_lcase, 'a' collate utf8_binary, 3) from t5; +select split_part(utf8_binary, 'a ' collate utf8_lcase_rtrim, 3), split_part(utf8_lcase, 'a' collate utf8_binary, 3) from t5; + +-- Contains +select contains(utf8_binary, utf8_lcase) from t5; +select contains(s, utf8_binary) from t5; +select contains(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select contains(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select contains(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select contains(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5; +select contains(utf8_binary, 'a'), contains(utf8_lcase, 'a') from t5; +select contains(utf8_binary, 'AaAA' collate utf8_lcase), contains(utf8_lcase, 'AAa' collate utf8_binary) from t5; +select contains(utf8_binary, 'AaAA ' collate utf8_lcase_rtrim), contains(utf8_lcase, 'AAa ' collate utf8_binary_rtrim) from t5; + +-- SubstringIndex +select substring_index(utf8_binary, utf8_lcase, 2) from t5; +select substring_index(s, utf8_binary,1) from t5; +select substring_index(utf8_binary collate utf8_binary, s collate utf8_lcase, 3) from t5; +select substring_index(utf8_binary, utf8_lcase collate utf8_binary, 2) from t5; +select substring_index(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t5; +select substring_index(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 2) from t5; +select substring_index(utf8_binary, 'a', 2), substring_index(utf8_lcase, 'a', 2) from t5; +select substring_index(utf8_binary, 'AaAA' collate utf8_lcase, 2), substring_index(utf8_lcase, 'AAa' collate utf8_binary, 2) from t5; +select substring_index(utf8_binary, 'AaAA ' collate utf8_lcase_rtrim, 2), substring_index(utf8_lcase, 'AAa' collate utf8_binary, 2) from t5; + +-- StringInStr +select instr(utf8_binary, utf8_lcase) from t5; +select instr(s, utf8_binary) from t5; +select instr(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select instr(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select instr(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select instr(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5; +select instr(utf8_binary, 'a'), instr(utf8_lcase, 'a') from t5; +select instr(utf8_binary, 'AaAA' collate utf8_lcase), instr(utf8_lcase, 'AAa' collate utf8_binary) from t5; + +-- FindInSet +select find_in_set(utf8_binary, utf8_lcase) from t5; +select find_in_set(s, utf8_binary) from t5; +select find_in_set(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select find_in_set(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select find_in_set(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select find_in_set(utf8_binary, 'aaAaaAaA,i̇o'), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o') from t5; +select find_in_set(utf8_binary, 'aaAaaAaA,i̇o' collate utf8_lcase), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o' collate utf8_binary) from t5; +select find_in_set(utf8_binary, 'aaAaaAaA,i̇o ' collate utf8_lcase_rtrim), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o' collate utf8_binary) from t5; +-- StartsWith +select startswith(utf8_binary, utf8_lcase) from t5; +select startswith(s, utf8_binary) from t5; +select startswith(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select startswith(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select startswith(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select startswith(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5; +select startswith(utf8_binary, 'aaAaaAaA'), startswith(utf8_lcase, 'aaAaaAaA') from t5; +select startswith(utf8_binary, 'aaAaaAaA' collate utf8_lcase), startswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t5; +select startswith(utf8_binary, 'aaAaaAaA ' collate utf8_lcase_rtrim), startswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t5; + +-- StringTranslate +select translate(utf8_lcase, utf8_lcase, '12345') from t5; +select translate(utf8_binary, utf8_lcase, '12345') from t5; +select translate(utf8_binary, 'aBc' collate utf8_lcase, '12345' collate utf8_binary) from t5; +select translate(utf8_binary, 'SQL' collate utf8_lcase, '12345' collate utf8_lcase) from t5; +select translate(utf8_binary, 'SQL' collate unicode_ai, '12345' collate unicode_ai) from t5; +select translate(utf8_lcase, 'aaAaaAaA', '12345'), translate(utf8_binary, 'aaAaaAaA', '12345') from t5; +select translate(utf8_lcase, 'aBc' collate utf8_binary, '12345'), translate(utf8_binary, 'aBc' collate utf8_lcase, '12345') from t5; +select translate(utf8_lcase, 'aBc ' collate utf8_binary_rtrim, '12345'), translate(utf8_binary, 'aBc' collate utf8_lcase, '12345') from t5; + +-- Replace +select replace(utf8_binary, utf8_lcase, 'abc') from t5; +select replace(s, utf8_binary, 'abc') from t5; +select replace(utf8_binary collate utf8_binary, s collate utf8_lcase, 'abc') from t5; +select replace(utf8_binary, utf8_lcase collate utf8_binary, 'abc') from t5; +select replace(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 'abc') from t5; +select replace(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 'abc') from t5; +select replace(utf8_binary, 'aaAaaAaA', 'abc'), replace(utf8_lcase, 'aaAaaAaA', 'abc') from t5; +select replace(utf8_binary, 'aaAaaAaA' collate utf8_lcase, 'abc'), replace(utf8_lcase, 'aaAaaAaA' collate utf8_binary, 'abc') from t5; +select replace(utf8_binary, 'aaAaaAaA ' collate utf8_lcase_rtrim, 'abc'), replace(utf8_lcase, 'aaAaaAaA' collate utf8_binary, 'abc') from t5; + +-- EndsWith +select endswith(utf8_binary, utf8_lcase) from t5; +select endswith(s, utf8_binary) from t5; +select endswith(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select endswith(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select endswith(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select endswith(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5; +select endswith(utf8_binary, 'aaAaaAaA'), endswith(utf8_lcase, 'aaAaaAaA') from t5; +select endswith(utf8_binary, 'aaAaaAaA' collate utf8_lcase), endswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t5; +select endswith(utf8_binary, 'aaAaaAaA ' collate utf8_lcase_rtrim), endswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t5; + +-- StringRepeat +select repeat(utf8_binary, 3), repeat(utf8_lcase, 2) from t5; +select repeat(utf8_binary collate utf8_lcase, 3), repeat(utf8_lcase collate utf8_binary, 2) from t5; + +-- Ascii & UnBase64 string expressions +select ascii(utf8_binary), ascii(utf8_lcase) from t5; +select ascii(utf8_binary collate utf8_lcase), ascii(utf8_lcase collate utf8_binary) from t5; +select unbase64(utf8_binary), unbase64(utf8_lcase) from t10; +select unbase64(utf8_binary collate utf8_lcase), unbase64(utf8_lcase collate utf8_binary) from t10; + +-- Chr +select chr(ascii) from t6; + +-- Base64, Decode +select base64(utf8_binary), base64(utf8_lcase) from t5; +select base64(utf8_binary collate utf8_lcase), base64(utf8_lcase collate utf8_binary) from t5; +select decode(encode(utf8_binary, 'utf-8'), 'utf-8'), decode(encode(utf8_lcase, 'utf-8'), 'utf-8') from t5; +select decode(encode(utf8_binary collate utf8_lcase, 'utf-8'), 'utf-8'), decode(encode(utf8_lcase collate utf8_binary, 'utf-8'), 'utf-8') from t5; + +-- FormatNumber +select format_number(ascii, '###.###') from t7; +select format_number(ascii, '###.###' collate utf8_lcase) from t7; + +-- Encode, ToBinary +select encode(utf8_binary, 'utf-8'), encode(utf8_lcase, 'utf-8') from t5; +select encode(utf8_binary collate utf8_lcase, 'utf-8'), encode(utf8_lcase collate utf8_binary, 'utf-8') from t5; +select to_binary(utf8_binary, 'utf-8'), to_binary(utf8_lcase, 'utf-8') from t5; +select to_binary(utf8_binary collate utf8_lcase, 'utf-8'), to_binary(utf8_lcase collate utf8_binary, 'utf-8') from t5; + +-- Sentences +select sentences(utf8_binary), sentences(utf8_lcase) from t5; +select sentences(utf8_binary collate utf8_lcase), sentences(utf8_lcase collate utf8_binary) from t5; + +-- Upper +select upper(utf8_binary), upper(utf8_lcase) from t5; +select upper(utf8_binary collate utf8_lcase), upper(utf8_lcase collate utf8_binary) from t5; + +-- Lower +select lower(utf8_binary), lower(utf8_lcase) from t5; +select lower(utf8_binary collate utf8_lcase), lower(utf8_lcase collate utf8_binary) from t5; + +-- InitCap +select initcap(utf8_binary), initcap(utf8_lcase) from t5; +select initcap(utf8_binary collate utf8_lcase), initcap(utf8_lcase collate utf8_binary) from t5; + +-- Overlay +select overlay(utf8_binary, utf8_lcase, 2) from t5; +select overlay(s, utf8_binary,1) from t5; +select overlay(utf8_binary collate utf8_binary, s collate utf8_lcase, 3) from t5; +select overlay(utf8_binary, utf8_lcase collate utf8_binary, 2) from t5; +select overlay(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t5; +select overlay(utf8_binary, 'a', 2), overlay(utf8_lcase, 'a', 2) from t5; +select overlay(utf8_binary, 'AaAA' collate utf8_lcase, 2), overlay(utf8_lcase, 'AAa' collate utf8_binary, 2) from t5; + +-- FormatString +select format_string(format, utf8_binary, utf8_lcase) from t8; +select format_string(format collate utf8_lcase, utf8_lcase, utf8_binary collate utf8_lcase, 3), format_string(format, utf8_lcase collate utf8_binary, utf8_binary) from t8; +select format_string(format, utf8_binary, utf8_lcase) from t8; + +-- SoundEx +select soundex(utf8_binary), soundex(utf8_lcase) from t5; +select soundex(utf8_binary collate utf8_lcase), soundex(utf8_lcase collate utf8_binary) from t5; + +-- Length, BitLength & OctetLength +select length(utf8_binary), length(utf8_lcase) from t5; +select length(utf8_binary collate utf8_lcase), length(utf8_lcase collate utf8_binary) from t5; +select bit_length(utf8_binary), bit_length(utf8_lcase) from t5; +select bit_length(utf8_binary collate utf8_lcase), bit_length(utf8_lcase collate utf8_binary) from t5; +select octet_length(utf8_binary), octet_length(utf8_lcase) from t5; +select octet_length(utf8_binary collate utf8_lcase), octet_length(utf8_lcase collate utf8_binary) from t5; +select octet_length(utf8_binary collate utf8_lcase_rtrim), octet_length(utf8_lcase collate utf8_binary_rtrim) from t5; + +-- Luhncheck +select luhn_check(num) from t9; + +-- Levenshtein +select levenshtein(utf8_binary, utf8_lcase) from t5; +select levenshtein(s, utf8_binary) from t5; +select levenshtein(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select levenshtein(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select levenshtein(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select levenshtein(utf8_binary, 'a'), levenshtein(utf8_lcase, 'a') from t5; +select levenshtein(utf8_binary, 'AaAA' collate utf8_lcase, 3), levenshtein(utf8_lcase, 'AAa' collate utf8_binary, 4) from t5; + +-- IsValidUTF8 +select is_valid_utf8(utf8_binary), is_valid_utf8(utf8_lcase) from t5; +select is_valid_utf8(utf8_binary collate utf8_lcase), is_valid_utf8(utf8_lcase collate utf8_binary) from t5; +select is_valid_utf8(utf8_binary collate utf8_lcase_rtrim), is_valid_utf8(utf8_lcase collate utf8_binary_rtrim) from t5; + +-- MakeValidUTF8 +select make_valid_utf8(utf8_binary), make_valid_utf8(utf8_lcase) from t5; +select make_valid_utf8(utf8_binary collate utf8_lcase), make_valid_utf8(utf8_lcase collate utf8_binary) from t5; +select make_valid_utf8(utf8_binary collate utf8_lcase_rtrim), make_valid_utf8(utf8_lcase collate utf8_binary_rtrim) from t5; + +-- ValidateUTF8 +select validate_utf8(utf8_binary), validate_utf8(utf8_lcase) from t5; +select validate_utf8(utf8_binary collate utf8_lcase), validate_utf8(utf8_lcase collate utf8_binary) from t5; +select validate_utf8(utf8_binary collate utf8_lcase_rtrim), validate_utf8(utf8_lcase collate utf8_binary_rtrim) from t5; + +-- TryValidateUTF8 +select try_validate_utf8(utf8_binary), try_validate_utf8(utf8_lcase) from t5; +select try_validate_utf8(utf8_binary collate utf8_lcase), try_validate_utf8(utf8_lcase collate utf8_binary) from t5; +select try_validate_utf8(utf8_binary collate utf8_lcase_rtrim), try_validate_utf8(utf8_lcase collate utf8_binary_rtrim) from t5; + +-- Left/Right/Substr +select substr(utf8_binary, 2, 2), substr(utf8_lcase, 2, 2) from t5; +select substr(utf8_binary collate utf8_lcase, 2, 2), substr(utf8_lcase collate utf8_binary, 2, 2) from t5; +select right(utf8_binary, 2), right(utf8_lcase, 2) from t5; +select right(utf8_binary collate utf8_lcase, 2), right(utf8_lcase collate utf8_binary, 2) from t5; +select left(utf8_binary, '2' collate utf8_lcase), left(utf8_lcase, 2) from t5; +select left(utf8_binary collate utf8_lcase, 2), left(utf8_lcase collate utf8_binary, 2) from t5; + +-- StringRPad +select rpad(utf8_binary, 8, utf8_lcase) from t5; +select rpad(s, 8, utf8_binary) from t5; +select rpad(utf8_binary collate utf8_binary, 8, s collate utf8_lcase) from t5; +select rpad(utf8_binary, 8, utf8_lcase collate utf8_binary) from t5; +select rpad(utf8_binary collate utf8_lcase, 8, utf8_lcase collate utf8_lcase) from t5; +select lpad(utf8_binary collate utf8_binary_rtrim, 8, utf8_lcase collate utf8_binary_rtrim) from t5; +select rpad(utf8_binary, 8, 'a'), rpad(utf8_lcase, 8, 'a') from t5; +select rpad(utf8_binary, 8, 'AaAA' collate utf8_lcase), rpad(utf8_lcase, 8, 'AAa' collate utf8_binary) from t5; + +-- StringLPad +select lpad(utf8_binary, 8, utf8_lcase) from t5; +select lpad(s, 8, utf8_binary) from t5; +select lpad(utf8_binary collate utf8_binary, 8, s collate utf8_lcase) from t5; +select lpad(utf8_binary, 8, utf8_lcase collate utf8_binary) from t5; +select lpad(utf8_binary collate utf8_lcase, 8, utf8_lcase collate utf8_lcase) from t5; +select lpad(utf8_binary collate utf8_binary_rtrim, 8, utf8_lcase collate utf8_binary_rtrim) from t5; +select lpad(utf8_binary, 8, 'a'), lpad(utf8_lcase, 8, 'a') from t5; +select lpad(utf8_binary, 8, 'AaAA' collate utf8_lcase), lpad(utf8_lcase, 8, 'AAa' collate utf8_binary) from t5; + +-- Locate +select locate(utf8_binary, utf8_lcase) from t5; +select locate(s, utf8_binary) from t5; +select locate(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select locate(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select locate(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 3) from t5; +select locate(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 3) from t5; +select locate(utf8_binary, 'a'), locate(utf8_lcase, 'a') from t5; +select locate(utf8_binary, 'AaAA' collate utf8_lcase, 4), locate(utf8_lcase, 'AAa' collate utf8_binary, 4) from t5; +select locate(utf8_binary, 'AaAA ' collate utf8_binary_rtrim, 4), locate(utf8_lcase, 'AAa ' collate utf8_binary, 4) from t5; + +-- StringTrim +select TRIM(utf8_binary, utf8_lcase) from t5; +select TRIM(s, utf8_binary) from t5; +select TRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select TRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select TRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select TRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5; +select TRIM(utf8_binary collate utf8_binary_rtrim, utf8_lcase collate utf8_binary_rtrim) from t5; +select TRIM('ABc', utf8_binary), TRIM('ABc', utf8_lcase) from t5; +select TRIM('ABc' collate utf8_lcase, utf8_binary), TRIM('AAa' collate utf8_binary, utf8_lcase) from t5; +-- StringTrimBoth +select BTRIM(utf8_binary, utf8_lcase) from t5; +select BTRIM(s, utf8_binary) from t5; +select BTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select BTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select BTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select BTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5; +select BTRIM(utf8_binary collate utf8_binary_rtrim, utf8_lcase collate utf8_binary_rtrim) from t5; +select BTRIM('ABc', utf8_binary), BTRIM('ABc', utf8_lcase) from t5; +select BTRIM('ABc' collate utf8_lcase, utf8_binary), BTRIM('AAa' collate utf8_binary, utf8_lcase) from t5; +-- StringTrimLeft +select LTRIM(utf8_binary, utf8_lcase) from t5; +select LTRIM(s, utf8_binary) from t5; +select LTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select LTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select LTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select LTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5; +select LTRIM(utf8_binary collate utf8_binary_rtrim, utf8_lcase collate utf8_binary_rtrim) from t5; +select LTRIM('ABc', utf8_binary), LTRIM('ABc', utf8_lcase) from t5; +select LTRIM('ABc' collate utf8_lcase, utf8_binary), LTRIM('AAa' collate utf8_binary, utf8_lcase) from t5; +-- StringTrimRight +select RTRIM(utf8_binary, utf8_lcase) from t5; +select RTRIM(s, utf8_binary) from t5; +select RTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5; +select RTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5; +select RTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5; +select RTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5; +select RTRIM(utf8_binary collate utf8_binary_rtrim, utf8_lcase collate utf8_binary_rtrim) from t5; +select RTRIM('ABc', utf8_binary), RTRIM('ABc', utf8_lcase) from t5; +select RTRIM('ABc' collate utf8_lcase, utf8_binary), RTRIM('AAa' collate utf8_binary, utf8_lcase) from t5; + +-- Implicit aliases to collated expression trees are correctly generated + +-- Simple select +select concat_ws(' ', utf8_lcase, utf8_lcase) from t5; + +-- Select by implicit alias +select `concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)` from ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 +); + +-- Select by star +select * from ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 +); + +-- Select by qualified star +select subq1.* from ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 +) AS subq1; + +-- Implicit alias in CTE output +with cte as ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 +) +select * from cte; + +-- Implicit alias in EXISTS subquery output +select * from values (1) where exists ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 +); + +-- Implicit alias in scalar subquery output +select ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 limit 1 +); + +-- Scalar subquery with CTE with implicit alias +select ( + with cte as ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 + ) + select * from cte limit 1 +); + +-- Outer reference to implicit alias +select * from ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 limit 1 +) +where ( + `concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)` == 'aaa' +); + +-- Implicit alias reference in Sort +select lower(`concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)`) from ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 + group by 1 + order by 1 +); + +-- Implciit alias from aggregate in Sort +select lower(`concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)`) from ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 + group by 1 + order by max(concat_ws(' ', utf8_lcase, utf8_lcase)) +); + +-- Implicit alias in view schema +create temporary view v1 as ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 +); + +select * from v1; + +select `concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)` from v1; + +drop view v1; + +drop table t5; +drop table t6; +drop table t7; +drop table t8; +drop table t9; +drop table t10; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/column-resolution-aggregate.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/column-resolution-aggregate.sql new file mode 100644 index 000000000000..4f879fc809d9 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/column-resolution-aggregate.sql @@ -0,0 +1,33 @@ +-- Tests covering column resolution priority in Aggregate. + +CREATE TEMPORARY VIEW v1 AS VALUES (1, 1, 1), (2, 2, 1) AS t(a, b, k); +CREATE TEMPORARY VIEW v2 AS VALUES (1, 1, 1), (2, 2, 1) AS t(x, y, all); + +-- Relation output columns have higher priority than lateral column alias. This query +-- should fail as `b` is not in GROUP BY. +SELECT max(a) AS b, b FROM v1 GROUP BY k; + +-- Lateral column alias has higher priority than outer reference. +SELECT a FROM v1 WHERE (12, 13) IN (SELECT max(x + 10) AS a, a + 1 FROM v2); + +-- Relation output columns have higher priority than GROUP BY alias. This query should +-- fail as `a` is not in GROUP BY. +SELECT a AS k FROM v1 GROUP BY k; + +-- Relation output columns have higher priority than GROUP BY ALL. This query should +-- fail as `x` is not in GROUP BY. +SELECT x FROM v2 GROUP BY all; + +-- GROUP BY alias has higher priority than GROUP BY ALL, this query fails as `b` is not in GROUP BY. +SELECT a AS all, b FROM v1 GROUP BY all; + +-- GROUP BY alias/ALL does not support lateral column alias. +SELECT k AS lca, lca + 1 AS col FROM v1 GROUP BY k, col; +SELECT k AS lca, lca + 1 AS col FROM v1 GROUP BY all; + +-- GROUP BY alias still works if it does not directly reference lateral column alias. +SELECT k AS lca, lca + 1 AS col FROM v1 GROUP BY lca; + +-- GROUP BY ALL has higher priority than outer reference. This query should run as `a` and `b` are +-- in GROUP BY due to the GROUP BY ALL resolution. +SELECT * FROM v2 WHERE EXISTS (SELECT a, b FROM v1 GROUP BY all); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/column-resolution-sort.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/column-resolution-sort.sql new file mode 100644 index 000000000000..2c5b9f9e9dfc --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/column-resolution-sort.sql @@ -0,0 +1,20 @@ +--SET spark.sql.leafNodeDefaultParallelism=1 +-- Tests covering column resolution priority in Sort. + +CREATE TEMPORARY VIEW v1 AS VALUES (1, 2, 2), (2, 1, 1) AS t(a, b, k); +CREATE TEMPORARY VIEW v2 AS VALUES (1, 2, 2), (2, 1, 1) AS t(a, b, all); + +-- Relation output columns have higher priority than missing reference. +-- Query will fail if we order by the column `v1.b`, as it's not in GROUP BY. +-- Actually results are [1, 2] as we order by `max(a) AS b`. +SELECT max(a) AS b FROM v1 GROUP BY k ORDER BY b; + +-- Missing reference has higher priority than ORDER BY ALL. +-- Results will be [1, 2] if we order by `max(a)`. +-- Actually results are [2, 1] as we order by the grouping column `v2.all`. +SELECT max(a) FROM v2 GROUP BY all ORDER BY all; + +-- ORDER BY ALL has higher priority than outer reference. +-- Results will be [1, 1] if we order by outer reference 'v2.all'. +-- Actually results are [2, 2] as we order by column `v1.b` +SELECT (SELECT b FROM v1 ORDER BY all LIMIT 1) FROM v2; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/columnresolution-negative.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/columnresolution-negative.sql new file mode 100644 index 000000000000..d100023b4ee1 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/columnresolution-negative.sql @@ -0,0 +1,50 @@ +-- Negative testcases for column resolution +CREATE DATABASE mydb1; +USE mydb1; +CREATE TABLE t1 USING parquet AS SELECT 1 AS i1; + +CREATE DATABASE mydb2; +USE mydb2; +CREATE TABLE t1 USING parquet AS SELECT 20 AS i1; + +-- Negative tests: column resolution scenarios with ambiguous cases in join queries +SET spark.sql.crossJoin.enabled = true; +USE mydb1; +SELECT i1 FROM t1, mydb1.t1; +SELECT t1.i1 FROM t1, mydb1.t1; +SELECT mydb1.t1.i1 FROM t1, mydb1.t1; +SELECT i1 FROM t1, mydb2.t1; +SELECT t1.i1 FROM t1, mydb2.t1; +USE mydb2; +SELECT i1 FROM t1, mydb1.t1; +SELECT t1.i1 FROM t1, mydb1.t1; +SELECT i1 FROM t1, mydb2.t1; +SELECT t1.i1 FROM t1, mydb2.t1; +SELECT db1.t1.i1 FROM t1, mydb2.t1; +SET spark.sql.crossJoin.enabled = false; + +-- Negative tests +USE mydb1; +SELECT mydb1.t1 FROM t1; +SELECT t1.x.y.* FROM t1; +SELECT t1 FROM mydb1.t1; +USE mydb2; +SELECT mydb1.t1.i1 FROM t1; + +-- Negative tests: view cannot resolve column after incompatible schema change +USE mydb1; +CREATE VIEW v1 AS SELECT * FROM t1; +DROP TABLE t1; +CREATE TABLE t1 USING parquet AS SELECT 1 AS i2; +SELECT * FROM v1; + +-- Negative tests: temp view cannot resolve column after incompatible schema change +USE mydb2; +CREATE TEMP VIEW v2 AS SELECT * FROM t1; +DROP TABLE t1; +CREATE TABLE t1 USING parquet AS SELECT 1 AS i2; +SELECT * FROM v2; + +-- reset +DROP DATABASE mydb1 CASCADE; +DROP DATABASE mydb2 CASCADE; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/columnresolution-views.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/columnresolution-views.sql new file mode 100644 index 000000000000..83c32a5bf243 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/columnresolution-views.sql @@ -0,0 +1,23 @@ +-- Tests for qualified column names for the view code-path +-- Test scenario with Temporary view +CREATE OR REPLACE TEMPORARY VIEW view1 AS SELECT 2 AS i1; +SELECT view1.* FROM view1; +SELECT * FROM view1; +SELECT view1.i1 FROM view1; +SELECT i1 FROM view1; +SELECT a.i1 FROM view1 AS a; +SELECT i1 FROM view1 AS a; +-- cleanup +DROP VIEW view1; + +-- Test scenario with Global Temp view +CREATE OR REPLACE GLOBAL TEMPORARY VIEW view1 as SELECT 1 as i1; +SELECT * FROM global_temp.view1; +SELECT global_temp.view1.* FROM global_temp.view1; +SELECT i1 FROM global_temp.view1; +SELECT global_temp.view1.i1 FROM global_temp.view1; +SELECT view1.i1 FROM global_temp.view1; +SELECT a.i1 FROM global_temp.view1 AS a; +SELECT i1 FROM global_temp.view1 AS a; +-- cleanup +DROP VIEW global_temp.view1; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/columnresolution.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/columnresolution.sql new file mode 100644 index 000000000000..d001185a7393 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/columnresolution.sql @@ -0,0 +1,78 @@ +-- Tests covering different scenarios with qualified column names +-- Scenario: column resolution scenarios with datasource table +CREATE DATABASE mydb1; +USE mydb1; +CREATE TABLE t1 USING parquet AS SELECT 1 AS i1; + +CREATE DATABASE mydb2; +USE mydb2; +CREATE TABLE t1 USING parquet AS SELECT 20 AS i1; + +USE mydb1; +SELECT i1 FROM t1; +SELECT i1 FROM mydb1.t1; +SELECT t1.i1 FROM t1; +SELECT t1.i1 FROM mydb1.t1; + +SELECT mydb1.t1.i1 FROM t1; +SELECT mydb1.t1.i1 FROM mydb1.t1; + +USE mydb2; +SELECT i1 FROM t1; +SELECT i1 FROM mydb1.t1; +SELECT t1.i1 FROM t1; +SELECT t1.i1 FROM mydb1.t1; +SELECT mydb1.t1.i1 FROM mydb1.t1; + +-- Scenario: resolve fully qualified table name in star expansion +USE mydb1; +SELECT t1.* FROM t1; +SELECT mydb1.t1.* FROM mydb1.t1; +SELECT t1.* FROM mydb1.t1; +USE mydb2; +SELECT t1.* FROM t1; +SELECT mydb1.t1.* FROM mydb1.t1; +SELECT t1.* FROM mydb1.t1; +SELECT a.* FROM mydb1.t1 AS a; + +-- Scenario: resolve in case of subquery + +USE mydb1; +CREATE TABLE t3 USING parquet AS SELECT * FROM VALUES (4,1), (3,1) AS t3(c1, c2); +CREATE TABLE t4 USING parquet AS SELECT * FROM VALUES (4,1), (2,1) AS t4(c2, c3); + +SELECT * FROM t3 WHERE c1 IN (SELECT c2 FROM t4 WHERE t4.c3 = t3.c2); + +SELECT * FROM mydb1.t3 WHERE c1 IN + (SELECT mydb1.t4.c2 FROM mydb1.t4 WHERE mydb1.t4.c3 = mydb1.t3.c2); + +-- Scenario: column resolution scenarios in join queries +SET spark.sql.crossJoin.enabled = true; + +SELECT mydb1.t1.i1 FROM t1, mydb2.t1; + +SELECT mydb1.t1.i1 FROM mydb1.t1, mydb2.t1; + +USE mydb2; +SELECT mydb1.t1.i1 FROM t1, mydb1.t1; +SET spark.sql.crossJoin.enabled = false; + +-- Scenario: Table with struct column +USE mydb1; +CREATE TABLE t5(i1 INT, t5 STRUCT) USING parquet; +INSERT INTO t5 VALUES(1, (2, 3)); +SELECT t5.i1 FROM t5; +SELECT t5.t5.i1 FROM t5; +SELECT t5.t5.i1 FROM mydb1.t5; +SELECT t5.i1 FROM mydb1.t5; +SELECT t5.* FROM mydb1.t5; +SELECT t5.t5.* FROM mydb1.t5; +SELECT mydb1.t5.t5.i1 FROM mydb1.t5; +SELECT mydb1.t5.t5.i2 FROM mydb1.t5; +SELECT mydb1.t5.* FROM mydb1.t5; +SELECT mydb1.t5.* FROM t5; + +-- Cleanup and Reset +USE default; +DROP DATABASE mydb1 CASCADE; +DROP DATABASE mydb2 CASCADE; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/comments.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/comments.sql new file mode 100644 index 000000000000..da5e57a94292 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/comments.sql @@ -0,0 +1,119 @@ +-- Test comments. + +-- the first case of bracketed comment +--QUERY-DELIMITER-START +/* This is the first example of bracketed comment. +SELECT 'ommented out content' AS first; +*/ +SELECT 'selected content' AS first; +--QUERY-DELIMITER-END + +-- the second case of bracketed comment +--QUERY-DELIMITER-START +/* This is the second example of bracketed comment. +SELECT '/', 'ommented out content' AS second; +*/ +SELECT '/', 'selected content' AS second; +--QUERY-DELIMITER-END + +-- the third case of bracketed comment +--QUERY-DELIMITER-START +/* This is the third example of bracketed comment. + *SELECT '*', 'ommented out content' AS third; + */ +SELECT '*', 'selected content' AS third; +--QUERY-DELIMITER-END + +-- the first case of empty bracketed comment +--QUERY-DELIMITER-START +/**/ +SELECT 'selected content' AS fourth; +--QUERY-DELIMITER-END + +-- the first case of nested bracketed comment +--QUERY-DELIMITER-START +/* This is the first example of nested bracketed comment. +/* I am a nested bracketed comment.*/ +*/ +SELECT 'selected content' AS fifth; +--QUERY-DELIMITER-END + +-- the second case of nested bracketed comment +--QUERY-DELIMITER-START +/* This is the second example of nested bracketed comment. +/* I am a nested bracketed comment. + */ + */ +SELECT 'selected content' AS sixth; +--QUERY-DELIMITER-END + +-- the third case of nested bracketed comment +--QUERY-DELIMITER-START +/* + * This is the third example of nested bracketed comment. + /* + * I am a nested bracketed comment. + */ + */ +SELECT 'selected content' AS seventh; +--QUERY-DELIMITER-END + +-- the fourth case of nested bracketed comment +--QUERY-DELIMITER-START +/* + * This is the fourth example of nested bracketed comment. +SELECT /* I am a nested bracketed comment.*/ * FROM testData; + */ +SELECT 'selected content' AS eighth; +--QUERY-DELIMITER-END + +-- the fifth case of nested bracketed comment +--QUERY-DELIMITER-START +SELECT /* + * This is the fifth example of nested bracketed comment. +/* I am a second level of nested bracketed comment. +/* I am a third level of nested bracketed comment. +Other information of third level. +SELECT 'ommented out content' AS ninth; +*/ +Other information of second level. +*/ +Other information of first level. +*/ +'selected content' AS ninth; +--QUERY-DELIMITER-END + +-- the first case of empty nested bracketed comment +--QUERY-DELIMITER-START +/*/**/*/ +SELECT 'selected content' AS tenth; +--QUERY-DELIMITER-END + +-- the first case of unclosed bracketed comment +--QUERY-DELIMITER-START +/*abc*/ +select 1 as a +/* + +2 as b +/*abc*/ +, 3 as c + +/**/ +; +--QUERY-DELIMITER-END + +-- the second case of unclosed bracketed comment +--QUERY-DELIMITER-START +/*abc*/ +select 1 as a +/* + +2 as b +/*abc*/ +, 3 as c + +/**/ +select 4 as d +; +--QUERY-DELIMITER-END diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/comparator.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/comparator.sql new file mode 100644 index 000000000000..70af4f75ac43 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/comparator.sql @@ -0,0 +1,13 @@ +-- binary type +select x'00' < x'0f'; +select x'00' < x'ff'; + +-- trim string to numeric +select '1 ' = 1Y; +select '\t1 ' = 1Y; +select '1 ' = 1S; +select '1 ' = 1; +select ' 1' = 1L; +select ' 1' = cast(1.0 as float); +select ' 1.0 ' = 1.0D; +select ' 1.0 ' = 1.0BD; \ No newline at end of file diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/conditional-functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/conditional-functions.sql new file mode 100644 index 000000000000..c7a4b055f024 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/conditional-functions.sql @@ -0,0 +1,38 @@ +-- Tests for conditional functions + +CREATE TABLE conditional_t USING PARQUET AS SELECT c1, c2 FROM VALUES(1d, 0),(2d, 1),(null, 1),(CAST('NaN' AS DOUBLE), 0) AS t(c1, c2); + +SELECT nanvl(c2, c1/c2 + c1/c2) FROM conditional_t; +SELECT nanvl(c2, 1/0) FROM conditional_t; +SELECT nanvl(1-0, 1/0) FROM conditional_t; + +SELECT if(c2 >= 0, 1-0, 1/0) from conditional_t; +SELECT if(1 == 1, 1, 1/0); +SELECT if(1 != 1, 1/0, 1); + +SELECT coalesce(c2, 1/0) from conditional_t; +SELECT coalesce(1, 1/0); +SELECT coalesce(null, 1, 1/0); + +SELECT case when c2 >= 0 then 1 else 1/0 end from conditional_t; +SELECT case when 1 < 2 then 1 else 1/0 end; +SELECT case when 1 > 2 then 1/0 else 1 end; + +SELECT nullifzero(0), + nullifzero(cast(0 as tinyint)), + nullifzero(cast(0 as bigint)), + nullifzero('0'), + nullifzero(0.0), + nullifzero(1), + nullifzero(null); + +SELECT nullifzero('abc'); + +SELECT zeroifnull(null), + zeroifnull(1), + zeroifnull(cast(1 as tinyint)), + zeroifnull(cast(1 as bigint)); + +SELECT zeroifnull('abc'); + +DROP TABLE conditional_t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/count.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/count.sql new file mode 100644 index 000000000000..644e808cc04c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/count.sql @@ -0,0 +1,55 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) +AS testData(a, b); + +-- count with single expression +SELECT + count(*), count(1), count(null), count(a), count(b), count(a + b), count((a, b)) +FROM testData; + +-- distinct count with single expression +SELECT + count(DISTINCT 1), + count(DISTINCT null), + count(DISTINCT a), + count(DISTINCT b), + count(DISTINCT (a + b)), + count(DISTINCT (a, b)) +FROM testData; + +-- count with multiple expressions +SELECT count(a, b), count(b, a), count(testData.*, testData.*) FROM testData; + +-- distinct count with multiple expressions +SELECT + count(DISTINCT a, b), count(DISTINCT b, a), count(DISTINCT *), count(DISTINCT testData.*, testData.*) +FROM testData; + +-- distinct count with multiple literals +SELECT count(DISTINCT 3,2); +SELECT count(DISTINCT 2), count(DISTINCT 2,3); +SELECT count(DISTINCT 2), count(DISTINCT 3,2); +SELECT count(DISTINCT a), count(DISTINCT 2,3) FROM testData; +SELECT count(DISTINCT a), count(DISTINCT 3,2) FROM testData; +SELECT count(DISTINCT a), count(DISTINCT 2), count(DISTINCT 2,3) FROM testData; +SELECT count(DISTINCT a), count(DISTINCT 2), count(DISTINCT 3,2) FROM testData; +SELECT count(distinct 0.8), percentile_approx(distinct a, 0.8) FROM testData; + +-- legacy behavior: allow calling function count without parameters +set spark.sql.legacy.allowParameterlessCount=true; +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding; +SELECT count() FROM testData; + +-- count without expressions +set spark.sql.legacy.allowParameterlessCount=false; +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; +SELECT count() FROM testData; + +-- legacy behavior: allow count(testData.*) +set spark.sql.legacy.allowStarWithSingleTableIdentifierInCount=true; +SELECT count(testData.*) FROM testData; + +-- count with a single tblName.* as parameter +set spark.sql.legacy.allowStarWithSingleTableIdentifierInCount=false; +SELECT count(testData.*) FROM testData; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cross-join.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cross-join.sql new file mode 100644 index 000000000000..b64197e2bc70 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cross-join.sql @@ -0,0 +1,36 @@ +-- Cross join detection and error checking is done in JoinSuite since explain output is +-- used in the error message and the ids are not stable. Only positive cases are checked here. + +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1); + +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2); + +-- Cross joins with and without predicates +SELECT * FROM nt1 cross join nt2; +SELECT * FROM nt1 cross join nt2 where nt1.k = nt2.k; +SELECT * FROM nt1 cross join nt2 on (nt1.k = nt2.k); +SELECT * FROM nt1 cross join nt2 where nt1.v1 = 1 and nt2.v2 = 22; + +SELECT a.key, b.key FROM +(SELECT k key FROM nt1 WHERE v1 < 2) a +CROSS JOIN +(SELECT k key FROM nt2 WHERE v2 = 22) b; + +-- Join reordering +create temporary view A(a, va) as select * from nt1; +create temporary view B(b, vb) as select * from nt1; +create temporary view C(c, vc) as select * from nt1; +create temporary view D(d, vd) as select * from nt1; + +-- Allowed since cross join with C is explicit +select * from ((A join B on (a = b)) cross join C) join D on (a = d); +-- Cross joins with non-equal predicates +SELECT * FROM nt1 CROSS JOIN nt2 ON (nt1.k > nt2.k); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/csv-functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/csv-functions.sql new file mode 100644 index 000000000000..01d436534a10 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/csv-functions.sql @@ -0,0 +1,24 @@ +-- from_csv +select from_csv('1, 3.14', 'a INT, f FLOAT'); +select from_csv('26/08/2015', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')); +-- Check if errors handled +select from_csv('1', 1); +select from_csv('1', 'a InvalidType'); +select from_csv('1', 'Array'); +select from_csv('1', 'a INT', named_struct('mode', 'PERMISSIVE')); +select from_csv('1', 'a INT', map('mode', 1)); +select from_csv(); +-- infer schema of json literal +select from_csv('1,abc', schema_of_csv('1,abc')); +select schema_of_csv('1|abc', map('delimiter', '|')); +select schema_of_csv(null); +CREATE TEMPORARY VIEW csvTable(csvField, a) AS SELECT * FROM VALUES ('1,abc', 'a'); +SELECT schema_of_csv(csvField) FROM csvTable; +-- Clean up +DROP VIEW IF EXISTS csvTable; +-- to_csv +select to_csv(named_struct('a', 1, 'b', 2)); +select to_csv(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy')); +-- Check if errors handled +select to_csv(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE')); +select to_csv(named_struct('a', 1, 'b', 2), map('mode', 1)); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cte-command.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cte-command.sql new file mode 100644 index 000000000000..ee90c2de49eb --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cte-command.sql @@ -0,0 +1,33 @@ +-- WITH inside CTE +CREATE TABLE cte_tbl USING csv AS WITH s AS (SELECT 42 AS col) SELECT * FROM s; + +SELECT * FROM cte_tbl; + +-- WITH inside CREATE VIEW +CREATE TEMPORARY VIEW cte_view AS WITH s AS (SELECT 42 AS col) SELECT * FROM s; + +SELECT * FROM cte_view; + +-- INSERT inside WITH +WITH s AS (SELECT 43 AS col) +INSERT INTO cte_tbl SELECT * FROM S; + +SELECT * FROM cte_tbl; + +-- WITH inside INSERT +INSERT INTO cte_tbl WITH s AS (SELECT 44 AS col) SELECT * FROM s; + +SELECT * FROM cte_tbl; + +CREATE TABLE cte_tbl2 (col INT) USING csv; +-- Multi-INSERT +WITH s AS (SELECT 45 AS col) +FROM s +INSERT INTO cte_tbl SELECT col +INSERT INTO cte_tbl2 SELECT col; + +SELECT * FROM cte_tbl; +SELECT * FROM cte_tbl2; + +DROP TABLE cte_tbl; +DROP TABLE cte_tbl2; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cte-legacy.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cte-legacy.sql new file mode 100644 index 000000000000..29dee1a3afd3 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cte-legacy.sql @@ -0,0 +1,2 @@ +--SET spark.sql.legacy.ctePrecedencePolicy = legacy +--IMPORT cte-nested.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cte-nested.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cte-nested.sql new file mode 100644 index 000000000000..3b2ba1fcdd66 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cte-nested.sql @@ -0,0 +1,228 @@ +-- CTE in CTE definition +WITH t as ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 +) +SELECT * FROM t; + +-- CTE in subquery +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 1) + SELECT * FROM t +); + +-- CTE in subquery expression +SELECT ( + WITH t AS (SELECT 1) + SELECT * FROM t +); + +-- un-referenced CTE in subquery expression: outer reference in CTE relation +SELECT ( + WITH unreferenced AS (SELECT id) + SELECT 1 +) FROM range(1); + +-- un-referenced CTE in subquery expression: outer reference in CTE main query +SELECT ( + WITH unreferenced AS (SELECT 1) + SELECT id +) FROM range(1); + +-- Make sure CTE in subquery is scoped to that subquery rather than global +-- the 2nd half of the union should fail because the cte is scoped to the first half +SELECT * FROM + ( + WITH cte AS (SELECT * FROM range(10)) + SELECT * FROM cte WHERE id = 8 + ) a +UNION +SELECT * FROM cte; + +-- CTE in CTE definition shadows outer +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +SELECT * FROM t2; + +-- CTE in CTE definition shadows outer 2 +WITH + t(c) AS (SELECT 1), + t2 AS ( + SELECT ( + SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) + ) + ) +SELECT * FROM t2; + +-- CTE in CTE definition shadows outer 3 +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2), + t2 AS ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) + SELECT * FROM t2 + ) +SELECT * FROM t2; + +-- CTE in subquery shadows outer +WITH t(c) AS (SELECT 1) +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +); + +-- CTE in subquery shadows outer 2 +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) +); + +-- CTE in subquery shadows outer 3 +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 3) + SELECT * FROM t + ) +); + +-- CTE in subquery expression shadows outer +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t +); + +-- CTE in subquery expression shadows outer 2 +WITH t AS (SELECT 1) +SELECT ( + SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +); + +-- CTE in subquery expression shadows outer 3 +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) +); + +-- CTE in subquery expression shadows outer 4 +WITH t(c) AS (SELECT 1) +SELECT * FROM t +WHERE c IN ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +); + +-- forward name conflict is not a real conflict +WITH + t AS ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 + ), + t2 AS (SELECT 2) +SELECT * FROM t; + +-- case insensitive name conflicts: in other CTE relations +WITH + abc AS (SELECT 1), + t AS ( + WITH aBc AS (SELECT 2) + SELECT * FROM aBC + ) +SELECT * FROM t; + +-- case insensitive name conflicts: in subquery expressions +WITH abc AS (SELECT 1) +SELECT ( + WITH aBc AS (SELECT 2) + SELECT * FROM aBC +); + +-- SPARK-38404: CTE in CTE definition references outer +WITH + t1 AS (SELECT 1), + t2 AS ( + WITH t3 AS ( + SELECT * FROM t1 + ) + SELECT * FROM t3 + ) +SELECT * FROM t2; + +-- CTE nested in CTE main body FROM clause references outer CTE def +WITH cte_outer AS ( + SELECT 1 +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_outer + ) + SELECT * FROM cte_inner +); + +-- CTE double nested in CTE main body FROM clause references outer CTE def +WITH cte_outer AS ( + SELECT 1 +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM ( + WITH cte_inner_inner AS ( + SELECT * FROM cte_outer + ) + SELECT * FROM cte_inner_inner + ) + ) + SELECT * FROM cte_inner +); + +-- Invalid reference to invisible CTE def nested CTE def +WITH cte_outer AS ( + WITH cte_invisible_inner AS ( + SELECT 1 + ) + SELECT * FROM cte_invisible_inner +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_invisible_inner + ) + SELECT * FROM cte_inner +); + +-- Invalid reference to invisible CTE def nested CTE def (in FROM) +WITH cte_outer AS ( + SELECT * FROM ( + WITH cte_invisible_inner AS ( + SELECT 1 + ) + SELECT * FROM cte_invisible_inner + ) +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_invisible_inner + ) + SELECT * FROM cte_inner +); \ No newline at end of file diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cte-nonlegacy.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cte-nonlegacy.sql new file mode 100644 index 000000000000..3316f6740281 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cte-nonlegacy.sql @@ -0,0 +1,2 @@ +--SET spark.sql.legacy.ctePrecedencePolicy = EXCEPTION +--IMPORT cte-nested.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cte.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cte.sql new file mode 100644 index 000000000000..ecf760ecac70 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/cte.sql @@ -0,0 +1,184 @@ +create temporary view t as select * from values 0, 1, 2 as t(id); +create temporary view t2 as select * from values 0, 1 as t(id); +create temporary view t3 as select * from t; + +-- WITH clause should not fall into infinite loop by referencing self +WITH s AS (SELECT 1 FROM s) SELECT * FROM s; + +WITH r AS (SELECT (SELECT * FROM r)) +SELECT * FROM r; + +-- WITH clause should reference the base table +WITH t AS (SELECT 1 FROM t) SELECT * FROM t; + +-- Table `t` referenced by a view should take precedence over the top CTE `t` +WITH t AS (SELECT 1) SELECT * FROM t3; + +-- WITH clause should not allow cross reference +WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2; + +-- WITH clause should reference the previous CTE +WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1 cross join t2; + +-- SPARK-18609 CTE with self-join +WITH CTE1 AS ( + SELECT b.id AS id + FROM T2 a + CROSS JOIN (SELECT id AS id FROM T2) b +) +SELECT t1.id AS c1, + t2.id AS c2 +FROM CTE1 t1 + CROSS JOIN CTE1 t2; + +-- CTE with column alias +WITH t(x) AS (SELECT 1) +SELECT * FROM t WHERE x = 1; + +-- CTE with multiple column aliases +WITH t(x, y) AS (SELECT 1, 2) +SELECT * FROM t WHERE x = 1 AND y = 2; + +-- CTE with duplicate column aliases +WITH t(x, x) AS (SELECT 1, 2) +SELECT * FROM t; + +-- CTE with empty column alias list is not allowed +WITH t() AS (SELECT 1) +SELECT * FROM t; + +-- CTEs with duplicate names are not allowed +WITH + t(x) AS (SELECT 1), + t(x) AS (SELECT 2) +SELECT * FROM t; + +-- invalid CTE relation should fail the query even if it's not referenced +WITH t AS (SELECT 1 FROM non_existing_table) +SELECT 2; + +-- The following tests are ported from Postgres +-- Multiple uses are evaluated only once +SELECT count(*) FROM ( + WITH q1(x) AS (SELECT random() FROM range(1, 5)) + SELECT * FROM q1 + UNION + SELECT * FROM q1 +) ss; + +-- Deeply nested +WITH w1(c1) AS + (WITH w2(c2) AS + (WITH w3(c3) AS + (WITH w4(c4) AS + (WITH w5(c5) AS + (WITH w6(c6) AS + (WITH w7(c7) AS + (WITH w8(c8) AS + (SELECT 1) + SELECT * FROM w8) + SELECT * FROM w7) + SELECT * FROM w6) + SELECT * FROM w5) + SELECT * FROM w4) + SELECT * FROM w3) + SELECT * FROM w2) +SELECT * FROM w1; + +-- CTE referencing an outer-level variable, should fail +SELECT ( WITH cte(foo) AS ( VALUES(id) ) + SELECT (SELECT foo FROM cte) ) +FROM t; + +-- CTE name collision with subquery name +WITH same_name AS (SELECT 42) +SELECT * FROM same_name, (SELECT 10) AS same_name; + +-- CTE name collision with subquery name, should fail +WITH same_name(x) AS (SELECT 42) +SELECT same_name.x FROM (SELECT 10) AS same_name(x), same_name; + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; +-- Test behavior with an unknown-type literal in the WITH +WITH q AS (SELECT 'foo' AS x) +SELECT x, typeof(x) FROM q; + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; +-- The following tests are ported from ZetaSQL +-- Alias inside the with hides the underlying column name, should fail +with cte as (select id as id_alias from t) +select id from cte; + +-- Reference of later WITH, should fail. +with r1 as (select * from r2), + r2 as (select 1) +select 2; + +-- WITH in a table subquery +SELECT * FROM + (WITH q AS (select 1 x) SELECT x+1 AS y FROM q); + +-- WITH in an expression subquery +select (with q as (select 1 x) select * from q); + +-- WITH in an IN subquery +select 1 in (with q as (select 1) select * from q); + +-- WITH alias referenced outside its scope, should fail +SELECT * FROM + (WITH q AS (select 1 x) SELECT x+1 AS y FROM q), + q; + +-- References to CTEs of the same name should be resolved properly +WITH T1 as (select 1 a) +select * +from + T1 x, + (WITH T1 as (select 2 b) select * from T1) y, + T1 z; + +-- References to CTEs of the same name should be resolved properly +WITH TTtt as (select 1 a), + `tTTt_2` as (select 2 a) +select * +from + (WITH TtTt as (select 3 c) select * from ttTT, `tttT_2`); + +-- Correlated CTE subquery +select + (WITH q AS (select T.x) select * from q) +from (select 1 x, 2 y) T; + +-- The main query inside WITH can be correlated. +select + (WITH q AS (select 3 z) select x + t.y + z from q) +from (select 1 x, 2 y) T; + +-- A WITH subquery alias is visible inside a WITH clause subquery. +WITH q1 as (select 1 x) +select * from + (with q2 as (select * from q1) select * from q2); + +-- A WITH subquery alias is visible inside a WITH clause subquery, and they have the same name. +WITH q1 as (select 1 x) +select * from + (with q1 as (select x+1 from q1) select * from q1); + +-- The following tests are ported from DuckDB +-- Duplicate CTE alias, should fail +with cte1 as (select 42), cte1 as (select 42) select * FROM cte1; + +-- Refer to CTE in subquery +with cte1 as (Select id as j from t) +select * from cte1 where j = (select max(j) from cte1 as cte2); + +-- Nested CTE views that re-use CTE aliases +with cte AS (SELECT * FROM va) SELECT * FROM cte; + +-- Self-refer to non-existent cte, should fail. +with cte as (select * from cte) select * from cte; + +-- Clean up +DROP VIEW IF EXISTS t; +DROP VIEW IF EXISTS t2; +DROP VIEW IF EXISTS t3; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/current_database_catalog.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/current_database_catalog.sql new file mode 100644 index 000000000000..d7aed6afaa17 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/current_database_catalog.sql @@ -0,0 +1,2 @@ +-- get current_database/current_schema and current_catalog +select current_database(), current_schema(), current_catalog(); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/date.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/date.sql new file mode 100644 index 000000000000..c1ae432d6316 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/date.sql @@ -0,0 +1,168 @@ +-- date literals, functions and operations + +create temporary view date_view as select '2011-11-11' date_str, '1' int_str; + +select date '2019-01-01\t'; +select date '2020-01-01中文'; + +select make_date(2019, 1, 1), make_date(12, 12, 12); +-- invalid month +select make_date(2000, 13, 1); +-- invalid day +select make_date(2000, 1, 33); + +-- invalid: year field must have at least 4 digits +select date'015'; +-- invalid: month field can have at most 2 digits +select date'2021-4294967297-11'; + +select current_date = current_date; +-- under ANSI mode, `current_date` can't be a function name. +select current_date() = current_date(); +select curdate(1); + +-- conversions between date and unix_date (number of days from epoch) +select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null); +select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(null); + +select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd'); + +-- `dayofweek` accepts both date and timestamp ltz/ntz inputs. +select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), + dayofweek('1582-10-15 13:10:15'), dayofweek(timestamp_ltz'1582-10-15 13:10:15'), dayofweek(timestamp_ntz'1582-10-15 13:10:15'); + +-- `weekday` accepts both date and timestamp ltz/ntz inputs. +select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null), + weekday('1582-10-15 13:10:15'), weekday(timestamp_ltz'1582-10-15 13:10:15'), weekday(timestamp_ntz'1582-10-15 13:10:15'); + +-- `year` accepts both date and timestamp ltz/ntz inputs. +select year('1500-01-01'), year('1582-10-15 13:10:15'), year(timestamp_ltz'1582-10-15 13:10:15'), year(timestamp_ntz'1582-10-15 13:10:15'); + +-- `month` accepts both date and timestamp ltz/ntz inputs. +select month('1500-01-01'), month('1582-10-15 13:10:15'), month(timestamp_ltz'1582-10-15 13:10:15'), month(timestamp_ntz'1582-10-15 13:10:15'); + +-- `dayOfYear` accepts both date and timestamp ltz/ntz inputs. +select dayOfYear('1500-01-01'), dayOfYear('1582-10-15 13:10:15'), dayOfYear(timestamp_ltz'1582-10-15 13:10:15'), dayOfYear(timestamp_ntz'1582-10-15 13:10:15'); + +-- next_day +select next_day("2015-07-23", "Mon"); +select next_day("2015-07-23", "xx"); +select next_day("2015-07-23 12:12:12", "Mon"); +-- next_date does not accept timestamp lzt/ntz input +select next_day(timestamp_ltz"2015-07-23 12:12:12", "Mon"); +select next_day(timestamp_ntz"2015-07-23 12:12:12", "Mon"); +select next_day("xx", "Mon"); +select next_day(null, "Mon"); +select next_day(null, "xx"); + +-- date add +select date_add(date'2011-11-11', 1); +select date_add('2011-11-11', 1); +select date_add('2011-11-11', 1Y); +select date_add('2011-11-11', 1S); +-- invalid cases: the second parameter can only be byte/short/int +select date_add('2011-11-11', 1L); +select date_add('2011-11-11', 1.0); +select date_add('2011-11-11', 1E1); +-- the second parameter can be a string literal if it can be parsed to int +select date_add('2011-11-11', '1'); +select date_add('2011-11-11', '1.2'); +-- null input leads to null result. +select date_add(null, 1); +select date_add(date'2011-11-11', null); +-- `date_add` accepts both date and timestamp ltz/ntz inputs (non-ANSI mode). +select date_add(timestamp_ltz'2011-11-11 12:12:12', 1); +select date_add(timestamp_ntz'2011-11-11 12:12:12', 1); + +-- date sub +select date_sub(date'2011-11-11', 1); +select date_sub('2011-11-11', 1); +select date_sub('2011-11-11', 1Y); +select date_sub('2011-11-11', 1S); +-- invalid cases: the second parameter can only be byte/short/int +select date_sub('2011-11-11', 1L); +select date_sub('2011-11-11', 1.0); +select date_sub('2011-11-11', 1E1); +-- the second parameter can be a string literal if it can be parsed to int +select date_sub(date'2011-11-11', '1'); +select date_sub(date'2011-11-11', '1.2'); +-- null input leads to null result. +select date_sub(null, 1); +select date_sub(date'2011-11-11', null); +-- `date_sub` accepts both date and timestamp ltz/ntz inputs (non-ANSI mode). +select date_sub(timestamp_ltz'2011-11-11 12:12:12', 1); +select date_sub(timestamp_ntz'2011-11-11 12:12:12', 1); + +-- date add/sub with non-literal string column +select date_add('2011-11-11', int_str) from date_view; +select date_sub('2011-11-11', int_str) from date_view; +select date_add(date_str, 1) from date_view; +select date_sub(date_str, 1) from date_view; + +-- date +/- number +select date '2011-11-11' + 1E1; +select date '2001-09-28' + 7Y; +select 7S + date '2001-09-28'; +select date '2001-10-01' - 7; + +-- date - date +select date '2001-10-01' - date '2001-09-28'; +-- if one side is string/null literal, promote it to date type. +select date '2001-10-01' - '2001-09-28'; +select '2001-10-01' - date '2001-09-28'; +select date '2001-09-28' - null; +select null - date '2019-10-06'; +-- invalid: non-literal string column +select date_str - date '2001-09-28' from date_view; +select date '2001-09-28' - date_str from date_view; + +-- invalid: date + string literal +select date'2011-11-11' + '1'; +select '1' + date'2011-11-11'; + +-- null result: date + null +select date'2011-11-11' + null; +select null + date'2011-11-11'; + +-- date +/- interval and interval + date +select date '2012-01-01' - interval '2-2' year to month, + date '2011-11-11' - interval '2' day, + date '2012-01-01' + interval '-2-2' year to month, + date '2011-11-11' + interval '-2' month, + - interval '2-2' year to month + date '2012-01-01', + interval '-2' day + date '2011-11-11'; + +-- Unsupported narrow text style +select to_date('26/October/2015', 'dd/MMMMM/yyyy'); +select from_json('{"d":"26/October/2015"}', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')); +select from_csv('26/October/2015', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')); + +-- Add a number of units to a timestamp or a date +select dateadd(MICROSECOND, 1001, timestamp'2022-02-25 01:02:03.123'); +select date_add(MILLISECOND, -1, timestamp'2022-02-25 01:02:03.456'); +select dateadd(SECOND, 58, timestamp'2022-02-25 01:02:03'); +select date_add(MINUTE, -100, date'2022-02-25'); +select dateadd(HOUR, -1, timestamp'2022-02-25 01:02:03'); +select date_add(DAY, 367, date'2022-02-25'); +select dateadd(WEEK, -4, timestamp'2022-02-25 01:02:03'); +select date_add(MONTH, -1, timestamp'2022-02-25 01:02:03'); +select dateadd(QUARTER, 5, date'2022-02-25'); +select date_add(YEAR, 1, date'2022-02-25'); + +select dateadd('MICROSECOND', 1001, timestamp'2022-02-25 01:02:03.123'); +select date_add('QUARTER', 5, date'2022-02-25'); + +-- Get the difference between timestamps or dates in the specified units +select datediff(MICROSECOND, timestamp'2022-02-25 01:02:03.123', timestamp'2022-02-25 01:02:03.124001'); +select date_diff(MILLISECOND, timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455'); +select datediff(SECOND, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 01:03:01'); +select date_diff(MINUTE, date'2022-02-25', timestamp'2022-02-24 22:20:00'); +select datediff(HOUR, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 00:02:03'); +select date_diff(DAY, date'2022-02-25', timestamp'2023-02-27 00:00:00'); +select datediff(WEEK, timestamp'2022-02-25 01:02:03', timestamp'2022-01-28 01:02:03'); +select date_diff(MONTH, timestamp'2022-02-25 01:02:03', timestamp'2022-01-25 01:02:03'); +select datediff(QUARTER, date'2022-02-25', date'2023-05-25'); +select date_diff(YEAR, date'2022-02-25', date'2023-02-25'); + +select date_diff('MILLISECOND', timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455'); +select datediff('YEAR', date'2022-02-25', date'2023-02-25'); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-formatting-invalid.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-formatting-invalid.sql new file mode 100644 index 000000000000..11bba00e91ab --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-formatting-invalid.sql @@ -0,0 +1,53 @@ +--- TESTS FOR DATETIME FORMATTING FUNCTIONS WITH INVALID PATTERNS --- + +-- separating this from datetime-formatting.sql, because the text form +-- for patterns with 5 letters in SimpleDateFormat varies from different JDKs +select date_format('2018-11-17 13:33:33.333', 'GGGGG'); +-- pattern letter count can not be greater than 6 +select date_format('2018-11-17 13:33:33.333', 'yyyyyyy'); +-- q/L in JDK 8 will fail when the count is more than 2 +select date_format('2018-11-17 13:33:33.333', 'qqqqq'); +select date_format('2018-11-17 13:33:33.333', 'QQQQQ'); +select date_format('2018-11-17 13:33:33.333', 'MMMMM'); +select date_format('2018-11-17 13:33:33.333', 'LLLLL'); + +select date_format('2018-11-17 13:33:33.333', 'EEEEE'); +select date_format('2018-11-17 13:33:33.333', 'FF'); +select date_format('2018-11-17 13:33:33.333', 'ddd'); +-- DD is invalid if the day-of-year exceeds 100, but it becomes valid in Java 11 +-- select date_format('2018-11-17 13:33:33.333', 'DD'); +select date_format('2018-11-17 13:33:33.333', 'DDDD'); +select date_format('2018-11-17 13:33:33.333', 'HHH'); +select date_format('2018-11-17 13:33:33.333', 'hhh'); +select date_format('2018-11-17 13:33:33.333', 'kkk'); +select date_format('2018-11-17 13:33:33.333', 'KKK'); +select date_format('2018-11-17 13:33:33.333', 'mmm'); +select date_format('2018-11-17 13:33:33.333', 'sss'); +select date_format('2018-11-17 13:33:33.333', 'SSSSSSSSSS'); +select date_format('2018-11-17 13:33:33.333', 'aa'); +select date_format('2018-11-17 13:33:33.333', 'V'); +select date_format('2018-11-17 13:33:33.333', 'zzzzz'); +select date_format('2018-11-17 13:33:33.333', 'XXXXXX'); +select date_format('2018-11-17 13:33:33.333', 'ZZZZZZ'); +select date_format('2018-11-17 13:33:33.333', 'OO'); +select date_format('2018-11-17 13:33:33.333', 'xxxxxx'); + +select date_format('2018-11-17 13:33:33.333', 'A'); +select date_format('2018-11-17 13:33:33.333', 'n'); +select date_format('2018-11-17 13:33:33.333', 'N'); +select date_format('2018-11-17 13:33:33.333', 'p'); + +-- disabled week-based patterns +select date_format('2018-11-17 13:33:33.333', 'Y'); +select date_format('2018-11-17 13:33:33.333', 'w'); +select date_format('2018-11-17 13:33:33.333', 'W'); +select date_format('2018-11-17 13:33:33.333', 'u'); +select date_format('2018-11-17 13:33:33.333', 'e'); +select date_format('2018-11-17 13:33:33.333', 'c'); + +-- others +select date_format('2018-11-17 13:33:33.333', 'B'); +select date_format('2018-11-17 13:33:33.333', 'C'); +select date_format('2018-11-17 13:33:33.333', 'I'); + + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-formatting-legacy.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-formatting-legacy.sql new file mode 100644 index 000000000000..19cab61a7ee5 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-formatting-legacy.sql @@ -0,0 +1,2 @@ +--SET spark.sql.legacy.timeParserPolicy=LEGACY +--IMPORT datetime-formatting.sql \ No newline at end of file diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-formatting.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-formatting.sql new file mode 100644 index 000000000000..3618bb5c399f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-formatting.sql @@ -0,0 +1,73 @@ +--- TESTS FOR DATETIME FORMATTING FUNCTIONS --- + +create temporary view v as select col from values + (timestamp '1582-06-01 11:33:33.123UTC+080000'), + (timestamp '1970-01-01 00:00:00.000Europe/Paris'), + (timestamp '1970-12-31 23:59:59.999Asia/Srednekolymsk'), + (timestamp '1996-04-01 00:33:33.123Australia/Darwin'), + (timestamp '2018-11-17 13:33:33.123Z'), + (timestamp '2020-01-01 01:33:33.123Asia/Shanghai'), + (timestamp '2100-01-01 01:33:33.123America/Los_Angeles') t(col); + +select col, date_format(col, 'G GG GGG GGGG'), to_char(col, 'G GG GGG GGGG'), to_varchar(col, 'G GG GGG GGGG') from v; + +select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy'), to_char(col, 'y yy yyy yyyy yyyyy yyyyyy'), to_varchar(col, 'y yy yyy yyyy yyyyy yyyyyy') from v; + +select col, date_format(col, 'q qq'), to_char(col, 'q qq'), to_varchar(col, 'q qq') from v; + +select col, date_format(col, 'Q QQ QQQ QQQQ'), to_char(col, 'Q QQ QQQ QQQQ'), to_varchar(col, 'Q QQ QQQ QQQQ') from v; + +select col, date_format(col, 'M MM MMM MMMM'), to_char(col, 'M MM MMM MMMM'), to_varchar(col, 'M MM MMM MMMM') from v; + +select col, date_format(col, 'L LL'), to_char(col, 'L LL'), to_varchar(col, 'L LL') from v; + +select col, date_format(col, 'E EE EEE EEEE'), to_char(col, 'E EE EEE EEEE'), to_varchar(col, 'E EE EEE EEEE') from v; + +select col, date_format(col, 'F'), to_char(col, 'F'), to_varchar(col, 'F') from v; + +select col, date_format(col, 'd dd'), to_char(col, 'd dd'), to_varchar(col, 'd dd') from v; + +select col, date_format(col, 'DD'), to_char(col, 'DD'), to_varchar(col, 'DD') from v where col = timestamp '2100-01-01 01:33:33.123America/Los_Angeles'; +select col, date_format(col, 'D DDD'), to_char(col, 'D DDD'), to_varchar(col, 'D DDD') from v; + +select col, date_format(col, 'H HH'), to_char(col, 'H HH'), to_varchar(col, 'H HH') from v; + +select col, date_format(col, 'h hh'), to_char(col, 'h hh'), to_varchar(col, 'h hh') from v; + +select col, date_format(col, 'k kk'), to_char(col, 'k kk'), to_varchar(col, 'k kk') from v; + +select col, date_format(col, 'K KK'), to_char(col, 'K KK'), to_varchar(col, 'K KK') from v; + +select col, date_format(col, 'm mm'), to_char(col, 'm mm'), to_varchar(col, 'm mm') from v; + +select col, date_format(col, 's ss'), to_char(col, 's ss'), to_varchar(col, 's ss') from v; + +select col, date_format(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS'), to_char(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS'), to_varchar(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS') from v; + +select col, date_format(col, 'a'), to_char(col, 'a'), to_varchar(col, 'a') from v; + +select col, date_format(col, 'VV'), to_char(col, 'VV'), to_varchar(col, 'VV') from v; + +select col, date_format(col, 'z zz zzz zzzz'), to_char(col, 'z zz zzz zzzz'), to_varchar(col, 'z zz zzz zzzz') from v; + +select col, date_format(col, 'X XX XXX'), to_char(col, 'X XX XXX'), to_varchar(col, 'X XX XXX') from v; +select col, date_format(col, 'XXXX XXXXX'), to_char(col, 'XXXX XXXXX'), to_varchar(col, 'XXXX XXXXX') from v; + +select col, date_format(col, 'Z ZZ ZZZ ZZZZ ZZZZZ'), to_char(col, 'Z ZZ ZZZ ZZZZ ZZZZZ'), to_varchar(col, 'Z ZZ ZZZ ZZZZ ZZZZZ') from v; + +select col, date_format(col, 'O OOOO'), to_char(col, 'O OOOO'), to_varchar(col, 'O OOOO') from v; + +select col, date_format(col, 'x xx xxx xxxx xxxx xxxxx'), to_char(col, 'x xx xxx xxxx xxxx xxxxx'), to_varchar(col, 'x xx xxx xxxx xxxx xxxxx') from v; + +-- optional pattern, but the results won't be optional for formatting +select col, date_format(col, '[yyyy-MM-dd HH:mm:ss]'), to_char(col, '[yyyy-MM-dd HH:mm:ss]'), to_varchar(col, '[yyyy-MM-dd HH:mm:ss]') from v; + +-- literals +select col, date_format(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'"), to_char(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'"), to_varchar(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'") from v; +select col, date_format(col, "''"), to_char(col, "''"), to_varchar(col, "''") from v; +select col, date_format(col, ''), to_char(col, ''), to_varchar(col, '') from v; + +-- different datetime types +select date_format(date'2023-08-18', 'yyyy-MM-dd'), to_char(date'2023-08-18', 'yyyy-MM-dd'), to_varchar(date'2023-08-18', 'yyyy-MM-dd'); +select date_format(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'), to_char(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'), to_varchar(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'); +select date_format(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS'), to_char(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS'), to_varchar(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS'); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-legacy.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-legacy.sql new file mode 100644 index 000000000000..22b29f3451ff --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-legacy.sql @@ -0,0 +1,3 @@ +--SET spark.sql.legacy.timeParserPolicy=LEGACY +--IMPORT date.sql +--IMPORT timestamp.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-parsing-invalid.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-parsing-invalid.sql new file mode 100644 index 000000000000..1f5a9b117ca2 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-parsing-invalid.sql @@ -0,0 +1,39 @@ +--- TESTS FOR DATETIME PARSING FUNCTIONS WITH INVALID VALUES --- + +-- parsing invalid value with pattern 'y' +select to_timestamp('294248', 'y'); -- out of year value range [0, 294247] +select to_timestamp('1', 'yy'); -- the number of digits must be 2 for 'yy'. +select to_timestamp('-12', 'yy'); -- out of year value range [0, 99] for reduced two digit form +select to_timestamp('123', 'yy'); -- the number of digits must be 2 for 'yy'. +select to_timestamp('1', 'yyy'); -- the number of digits must be in [3, 6] for 'yyy' + +select to_timestamp('1234567', 'yyyyyyy'); -- the length of 'y' pattern must be less than 7 + +-- parsing invalid values with pattern 'D' +select to_timestamp('366', 'D'); +select to_timestamp('9', 'DD'); +select to_timestamp('366', 'DD'); +select to_timestamp('9', 'DDD'); +select to_timestamp('99', 'DDD'); +select to_timestamp('30-365', 'dd-DDD'); +select to_timestamp('11-365', 'MM-DDD'); +select to_timestamp('2019-366', 'yyyy-DDD'); +select to_timestamp('12-30-365', 'MM-dd-DDD'); +select to_timestamp('2020-01-365', 'yyyy-dd-DDD'); +select to_timestamp('2020-10-350', 'yyyy-MM-DDD'); +select to_timestamp('2020-11-31-366', 'yyyy-MM-dd-DDD'); +-- add a special case to test csv, because the legacy formatter it uses is lenient then Spark should +-- throw SparkUpgradeException +select from_csv('2018-366', 'date Date', map('dateFormat', 'yyyy-DDD')); + +-- Datetime types parse error +select to_date("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS"); +select to_date("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS"); +select to_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS"); +select to_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS"); +select unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS"); +select unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS"); +select to_unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS"); +select to_unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS"); +select cast("Unparseable" as timestamp); +select cast("Unparseable" as date); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-parsing-legacy.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-parsing-legacy.sql new file mode 100644 index 000000000000..ee1afe502ab7 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-parsing-legacy.sql @@ -0,0 +1,2 @@ +--SET spark.sql.legacy.timeParserPolicy=LEGACY +--IMPORT datetime-parsing.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-parsing.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-parsing.sql new file mode 100644 index 000000000000..f87ae556867b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-parsing.sql @@ -0,0 +1,45 @@ +--- TESTS FOR DATETIME PARSING FUNCTIONS --- + +-- parsing with pattern 'y'. +-- the range of valid year is [-290307, 294247], +-- but particularly, some thrift client use java.sql.Timestamp to parse timestamp, which allows +-- only positive year values less or equal than 9999. So the cases bellow only use [1, 9999] to pass +-- ThriftServerQueryTestSuite +select to_timestamp('1', 'y'); +select to_timestamp('009999', 'y'); + +-- reduced two digit form is used, the range of valid year is 20-[01, 99] +select to_timestamp('00', 'yy'); +select to_timestamp('99', 'yy'); + +-- the range of valid year is [-290307, 294247], the number of digits must be in [3, 6] for 'yyy' +select to_timestamp('001', 'yyy'); +select to_timestamp('009999', 'yyy'); + +-- the range of valid year is [-9999, 9999], the number of digits must be 4 for 'yyyy'. +select to_timestamp('0001', 'yyyy'); +select to_timestamp('9999', 'yyyy'); + +-- the range of valid year is [-99999, 99999], the number of digits must be 5 for 'yyyyy'. +select to_timestamp('00001', 'yyyyy'); +select to_timestamp('09999', 'yyyyy'); + +-- the range of valid year is [-290307, 294247], the number of digits must be 6 for 'yyyyyy'. +select to_timestamp('000001', 'yyyyyy'); +select to_timestamp('009999', 'yyyyyy'); + +-- parsing with pattern 'D' +select to_timestamp('9', 'D'); +select to_timestamp('300', 'D'); +select to_timestamp('09', 'DD'); +select to_timestamp('99', 'DD'); +select to_timestamp('100', 'DD'); +select to_timestamp('009', 'DDD'); +select to_timestamp('365', 'DDD'); +select to_timestamp('31-365', 'dd-DDD'); +select to_timestamp('12-365', 'MM-DDD'); +select to_timestamp('2020-365', 'yyyy-DDD'); +select to_timestamp('12-31-365', 'MM-dd-DDD'); +select to_timestamp('2020-30-365', 'yyyy-dd-DDD'); +select to_timestamp('2020-12-350', 'yyyy-MM-DDD'); +select to_timestamp('2020-12-31-366', 'yyyy-MM-dd-DDD'); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-special.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-special.sql new file mode 100644 index 000000000000..2699274bb7c9 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/datetime-special.sql @@ -0,0 +1,10 @@ +-- special date and timestamp values that are not allowed in the SQL standard +-- these tests are put in this separated file because they don't work in JDBC environment + +-- date with year outside [0000-9999] +select date'999999-03-18', date'-0001-1-28', date'0015'; +select make_date(999999, 3, 18), make_date(-1, 1, 28); + +-- timestamp with year outside [0000-9999] +select timestamp'-1969-12-31 16:00:00', timestamp'-0015-03-18 16:00:00', timestamp'-000001', timestamp'99999-03-18T12:03:17'; +select make_timestamp(-1969, 12, 31, 16, 0, 0.0), make_timestamp(-15, 3, 18, 16, 0, 0.0), make_timestamp(99999, 3, 18, 12, 3, 17.0); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/decimalArithmeticOperations.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/decimalArithmeticOperations.sql new file mode 100644 index 000000000000..13bbd9d81b79 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/decimalArithmeticOperations.sql @@ -0,0 +1,125 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +CREATE TEMPORARY VIEW t AS SELECT 1.0 as a, 0.0 as b; + +-- division, remainder and pmod by 0 return NULL +select a / b from t; +select a % b from t; +select pmod(a, b) from t; + +-- tests for decimals handling in operations +create table decimals_test(id int, a decimal(38,18), b decimal(38,18)) using parquet; + +insert into decimals_test values(1, 100.0, 999.0), (2, 12345.123, 12345.123), + (3, 0.1234567891011, 1234.1), (4, 123456789123456789.0, 1.123456789123456789); + +-- test decimal operations +select id, a+b, a-b, a*b, a/b from decimals_test order by id; + +-- test operations between decimals and constants +select id, a*10, b/10 from decimals_test order by id; + +-- test operations on constants +select 10.3 * 3.0; +select 10.3000 * 3.0; +select 10.30000 * 30.0; +select 10.300000000000000000 * 3.000000000000000000; +select 10.300000000000000000 * 3.0000000000000000000; +select 2.35E10 * 1.0; + +-- arithmetic operations causing an overflow return NULL +select (5e36BD + 0.1) + 5e36BD; +select (-4e36BD - 0.1) - 7e36BD; +select 12345678901234567890.0 * 12345678901234567890.0; +select 1e35BD / 0.1; +select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD; + +-- arithmetic operations causing a precision loss are truncated +select 12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345; +select 123456789123456789.1234567890 * 1.123456789123456789; +select 12345678912345.123456789123 / 0.000000012345678; + +-- union decimal type + +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) + CAST(90 AS DECIMAL(3, 1)); + +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) - CAST(-90 AS DECIMAL(3, 1)); + +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) * CAST(10 AS DECIMAL(3, 1)); + +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) / CAST(10 AS DECIMAL(3, 1)); + +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(10, 2)) % CAST(3 AS DECIMAL(5, 1)); + +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT pmod(CAST(10 AS DECIMAL(10, 2)), CAST(3 AS DECIMAL(5, 1))); + +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(10, 2)) div CAST(3 AS DECIMAL(5, 1)); + +-- return NULL instead of rounding, according to old Spark versions' behavior +set spark.sql.decimalOperations.allowPrecisionLoss=false; + +-- test decimal operations +select /*+ COALESCE(1) */ id, a+b, a-b, a*b, a/b from decimals_test order by id; + +-- test operations between decimals and constants +select id, a*10, b/10 from decimals_test order by id; + +-- test operations on constants +select 10.3 * 3.0; +select 10.3000 * 3.0; +select 10.30000 * 30.0; +select 10.300000000000000000 * 3.000000000000000000; +select 10.300000000000000000 * 3.0000000000000000000; +select 2.35E10 * 1.0; + +-- arithmetic operations causing an overflow return NULL +select (5e36BD + 0.1) + 5e36BD; +select (-4e36BD - 0.1) - 7e36BD; +select 12345678901234567890.0 * 12345678901234567890.0; +select 1e35BD / 0.1; +select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD; + +-- arithmetic operations causing a precision loss return NULL +select 12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345; +select 123456789123456789.1234567890 * 1.123456789123456789; +select 12345678912345.123456789123 / 0.000000012345678; + +select 1.0123456789012345678901234567890123456e36BD / 0.1; +select 1.0123456789012345678901234567890123456e35BD / 1.0; +select 1.0123456789012345678901234567890123456e34BD / 1.0; +select 1.0123456789012345678901234567890123456e33BD / 1.0; +select 1.0123456789012345678901234567890123456e32BD / 1.0; +select 1.0123456789012345678901234567890123456e31BD / 1.0; +select 1.0123456789012345678901234567890123456e31BD / 0.1; +select 1.0123456789012345678901234567890123456e31BD / 10.0; + +drop table decimals_test; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/describe-part-after-analyze.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/describe-part-after-analyze.sql new file mode 100644 index 000000000000..f4239da90627 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/describe-part-after-analyze.sql @@ -0,0 +1,34 @@ +CREATE TABLE t (key STRING, value STRING, ds STRING, hr INT) USING parquet + PARTITIONED BY (ds, hr); + +INSERT INTO TABLE t PARTITION (ds='2017-08-01', hr=10) +VALUES ('k1', 100), ('k2', 200), ('k3', 300); + +INSERT INTO TABLE t PARTITION (ds='2017-08-01', hr=11) +VALUES ('k1', 101), ('k2', 201), ('k3', 301), ('k4', 401); + +INSERT INTO TABLE t PARTITION (ds='2017-09-01', hr=5) +VALUES ('k1', 102), ('k2', 202); + +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10); + +-- Collect stats for a single partition +ANALYZE TABLE t PARTITION (ds='2017-08-01', hr=10) COMPUTE STATISTICS; + +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10); + +-- Collect stats for 2 partitions +ANALYZE TABLE t PARTITION (ds='2017-08-01') COMPUTE STATISTICS; + +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10); +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=11); + +-- Collect stats for all partitions +ANALYZE TABLE t PARTITION (ds, hr) COMPUTE STATISTICS; + +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10); +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=11); +DESC EXTENDED t PARTITION (ds='2017-09-01', hr=5); + +-- DROP TEST TABLES/VIEWS +DROP TABLE t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/describe-query.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/describe-query.sql new file mode 100644 index 000000000000..f8fbb43ba2f8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/describe-query.sql @@ -0,0 +1,31 @@ +-- Test tables +CREATE table desc_temp1 (key int COMMENT 'column_comment', val string) USING PARQUET; +CREATE table desc_temp2 (key int, val string) USING PARQUET; + +-- Simple Describe query +DESC SELECT key, key + 1 as plusone FROM desc_temp1; +DESC QUERY SELECT * FROM desc_temp2; +DESC SELECT key, COUNT(*) as count FROM desc_temp1 group by key; +DESC SELECT 10.00D as col1; +DESC QUERY SELECT key FROM desc_temp1 UNION ALL select CAST(1 AS DOUBLE); +DESC QUERY VALUES(1.00D, 'hello') as tab1(col1, col2); +DESC QUERY FROM desc_temp1 a SELECT *; +DESC WITH s AS (SELECT 'hello' as col1) SELECT * FROM s; +DESCRIBE QUERY WITH s AS (SELECT * from desc_temp1) SELECT * FROM s; +DESCRIBE SELECT * FROM (FROM desc_temp2 select * select *); + +-- Error cases. +DESCRIBE INSERT INTO desc_temp1 values (1, 'val1'); +DESCRIBE INSERT INTO desc_temp1 SELECT * FROM desc_temp2; +DESCRIBE + FROM desc_temp1 a + insert into desc_temp1 select * + insert into desc_temp2 select *; + +-- Explain +EXPLAIN DESC QUERY SELECT * FROM desc_temp2 WHERE key > 0; +EXPLAIN EXTENDED DESC WITH s AS (SELECT 'hello' as col1) SELECT * FROM s; + +-- cleanup +DROP TABLE desc_temp1; +DROP TABLE desc_temp2; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/describe-table-after-alter-table.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/describe-table-after-alter-table.sql new file mode 100644 index 000000000000..69bff6656c43 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/describe-table-after-alter-table.sql @@ -0,0 +1,29 @@ +CREATE TABLE table_with_comment (a STRING, b INT, c STRING, d STRING) USING parquet COMMENT 'added'; + +DESC FORMATTED table_with_comment; + +-- ALTER TABLE BY MODIFYING COMMENT +ALTER TABLE table_with_comment SET TBLPROPERTIES("comment"= "modified comment", "type"= "parquet"); + +DESC FORMATTED table_with_comment; + +-- DROP TEST TABLE +DROP TABLE table_with_comment; + +-- CREATE TABLE WITHOUT COMMENT +CREATE TABLE table_comment (a STRING, b INT) USING parquet; + +DESC FORMATTED table_comment; + +-- ALTER TABLE BY ADDING COMMENT +ALTER TABLE table_comment SET TBLPROPERTIES(comment = "added comment"); + +DESC formatted table_comment; + +-- ALTER UNSET PROPERTIES COMMENT +ALTER TABLE table_comment UNSET TBLPROPERTIES IF EXISTS ('comment'); + +DESC FORMATTED table_comment; + +-- DROP TEST TABLE +DROP TABLE table_comment; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/describe.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/describe.sql new file mode 100644 index 000000000000..dbe5bc840bce --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/describe.sql @@ -0,0 +1,145 @@ +CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet + OPTIONS (a '1', b '2', password 'password') + PARTITIONED BY (c, d) CLUSTERED BY (a) SORTED BY (b ASC) INTO 2 BUCKETS + COMMENT 'table_comment' + TBLPROPERTIES (t 'test', password 'password'); + +CREATE TEMPORARY VIEW temp_v AS SELECT * FROM t; + +CREATE TEMPORARY VIEW temp_Data_Source_View + USING org.apache.spark.sql.sources.DDLScanSource + OPTIONS ( + From '1', + To '10', + Table 'test1'); + +CREATE VIEW v AS SELECT * FROM t; + +ALTER TABLE t SET TBLPROPERTIES (e = '3'); + +ALTER TABLE t ADD PARTITION (c='Us', d=1); + +DESCRIBE t; + +DESCRIBE EXTENDED t AS JSON; + +-- AnalysisException: describe table as json must be extended +DESCRIBE t AS JSON; + +-- AnalysisException: describe col as json unsupported +DESC FORMATTED t a AS JSON; + +DESC default.t; + +DESC TABLE t; + +DESC FORMATTED t; + +DESC EXTENDED t; + +ALTER TABLE t UNSET TBLPROPERTIES (e); + +DESC EXTENDED t; + +ALTER TABLE t UNSET TBLPROPERTIES (comment); + +DESC EXTENDED t; + +DESC t PARTITION (c='Us', d=1); + +DESC EXTENDED t PARTITION (c='Us', d=1) AS JSON; + +DESC EXTENDED t PARTITION (c='Us', d=1); + +DESC FORMATTED t PARTITION (c='Us', d=1); + +DESC EXTENDED t PARTITION (C='Us', D=1); + +-- NoSuchPartitionException: Partition not found in table +DESC t PARTITION (c='Us', d=2); + +-- AnalysisException: Partition spec is invalid +DESC t PARTITION (c='Us'); + +-- ParseException: PARTITION specification is incomplete +DESC t PARTITION (c='Us', d); + +-- DESC Temp View + +DESC temp_v; + +DESC TABLE temp_v; + +DESC FORMATTED temp_v; + +DESC EXTENDED temp_v; + +DESC temp_Data_Source_View; + +-- AnalysisException DESC PARTITION is not allowed on a temporary view +DESC temp_v PARTITION (c='Us', d=1); + +-- DESC Persistent View + +DESC v; + +DESC TABLE v; + +DESC FORMATTED v; + +DESC EXTENDED v; + +-- AnalysisException DESC PARTITION is not allowed on a view +DESC v PARTITION (c='Us', d=1); + +-- Explain Describe Table +EXPLAIN DESC t; +EXPLAIN DESC EXTENDED t; +EXPLAIN EXTENDED DESC t; +EXPLAIN DESCRIBE t b; +EXPLAIN DESCRIBE t PARTITION (c='Us', d=2); +EXPLAIN DESCRIBE EXTENDED t PARTITION (c='Us', d=2) AS JSON; + +-- Show column default values +CREATE TABLE d (a STRING DEFAULT 'default-value', b INT DEFAULT 42) USING parquet COMMENT 'table_comment'; + +DESC d; + +DESC EXTENDED d; + +DESC TABLE EXTENDED d; + +DESC FORMATTED d; + +-- Show column default values with newlines in the string +CREATE TABLE e (a STRING DEFAULT CONCAT('a\n b\n ', 'c\n d'), b INT DEFAULT 42) USING parquet COMMENT 'table_comment'; + +DESC e; + +DESC EXTENDED e; + +DESC TABLE EXTENDED e; + +DESC FORMATTED e; + +CREATE TABLE f USING json PARTITIONED BY (B, C) AS SELECT 'APACHE' A, CAST('SPARK' AS BINARY) B, TIMESTAMP'2018-11-17 13:33:33' C; + +DESC FORMATTED f PARTITION (B='SPARK', C=TIMESTAMP'2018-11-17 13:33:33'); + +DESC TABLE EXTENDED f PARTITION (B='SPARK', C=TIMESTAMP'2018-11-17 13:33:33') AS JSON; + +-- DROP TEST TABLES/VIEWS + +DROP VIEW temp_v; + +DROP VIEW temp_Data_Source_View; + +DROP VIEW v; + +DROP TABLE t; + +DROP TABLE d; + +DROP TABLE e; + +DROP TABLE f; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/double-quoted-identifiers-enabled.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/double-quoted-identifiers-enabled.sql new file mode 100644 index 000000000000..9547d011c76e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/double-quoted-identifiers-enabled.sql @@ -0,0 +1,3 @@ +--SET spark.sql.ansi.doubleQuotedIdentifiers=true +--IMPORT double-quoted-identifiers.sql + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/double-quoted-identifiers.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/double-quoted-identifiers.sql new file mode 100644 index 000000000000..ffb52b403346 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/double-quoted-identifiers.sql @@ -0,0 +1,57 @@ +-- All these should error out in the parser +SELECT 1 FROM "not_exist"; + +USE SCHEMA "not_exist"; + +ALTER TABLE "not_exist" ADD COLUMN not_exist int; + +ALTER TABLE not_exist ADD COLUMN "not_exist" int; + +SELECT 1 AS "not_exist" FROM not_exist; + +SELECT 1 FROM not_exist AS X("hello"); + +SELECT "not_exist"(); + +SELECT "not_exist".not_exist(); + +-- All these should error out in analysis +SELECT 1 FROM `hello`; + +USE SCHEMA `not_exist`; + +ALTER TABLE `not_exist` ADD COLUMN not_exist int; + +ALTER TABLE not_exist ADD COLUMN `not_exist` int; + +SELECT 1 AS `not_exist` FROM `not_exist`; + +SELECT 1 FROM not_exist AS X(`hello`); + +SELECT `not_exist`(); + +SELECT `not_exist`.not_exist(); + +-- Strings in various situations all work +SELECT "hello"; + +CREATE TEMPORARY VIEW v(c1 COMMENT "hello") AS SELECT 1; +DROP VIEW v; + +SELECT INTERVAL "1" YEAR; + +-- Single ticks still work +SELECT 'hello'; + +CREATE TEMPORARY VIEW v(c1 COMMENT 'hello') AS SELECT 1; +DROP VIEW v; + +SELECT INTERVAL '1' YEAR; + +-- A whole scenario +CREATE SCHEMA "myschema"; +CREATE TEMPORARY VIEW "myview"("c1") AS + WITH "v"("a") AS (SELECT 1) SELECT "a" FROM "v"; +SELECT "a1" AS "a2" FROM "myview" AS "atab"("a1"); +DROP TABLE "myview"; +DROP SCHEMA "myschema"; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/except-all.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/except-all.sql new file mode 100644 index 000000000000..e28f0721a644 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/except-all.sql @@ -0,0 +1,160 @@ +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (0), (1), (2), (2), (2), (2), (3), (null), (null) AS tab1(c1); +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1), (2), (2), (3), (5), (5), (null) AS tab2(c1); +CREATE TEMPORARY VIEW tab3 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (2, 3), + (2, 2) + AS tab3(k, v); +CREATE TEMPORARY VIEW tab4 AS SELECT * FROM VALUES + (1, 2), + (2, 3), + (2, 2), + (2, 2), + (2, 20) + AS tab4(k, v); + +-- Basic EXCEPT ALL +SELECT * FROM tab1 +EXCEPT ALL +SELECT * FROM tab2; + +-- MINUS ALL (synonym for EXCEPT) +SELECT * FROM tab1 +MINUS ALL +SELECT * FROM tab2; + +-- EXCEPT ALL same table in both branches +SELECT * FROM tab1 +EXCEPT ALL +SELECT * FROM tab2 WHERE c1 IS NOT NULL; + +-- Empty left relation +SELECT * FROM tab1 WHERE c1 > 5 +EXCEPT ALL +SELECT * FROM tab2; + +-- Empty right relation +SELECT * FROM tab1 +EXCEPT ALL +SELECT * FROM tab2 WHERE c1 > 6; + +-- Type Coerced ExceptAll +SELECT * FROM tab1 +EXCEPT ALL +SELECT CAST(1 AS BIGINT); + +-- Error as types of two side are not compatible +SELECT * FROM tab1 +EXCEPT ALL +SELECT array(1); + +-- Basic +SELECT * FROM tab3 +EXCEPT ALL +SELECT * FROM tab4; + +-- Basic +SELECT * FROM tab4 +EXCEPT ALL +SELECT * FROM tab3; + +-- EXCEPT ALL + INTERSECT +SELECT * FROM tab4 +EXCEPT ALL +SELECT * FROM tab3 +INTERSECT DISTINCT +SELECT * FROM tab4; + +-- EXCEPT ALL + EXCEPT +SELECT * FROM tab4 +EXCEPT ALL +SELECT * FROM tab3 +EXCEPT DISTINCT +SELECT * FROM tab4; + +-- Chain of set operations +SELECT * FROM tab3 +EXCEPT ALL +SELECT * FROM tab4 +UNION ALL +SELECT * FROM tab3 +EXCEPT DISTINCT +SELECT * FROM tab4; + +-- Mismatch on number of columns across both branches +SELECT k FROM tab3 +EXCEPT ALL +SELECT k, v FROM tab4; + +-- Chain of set operations +SELECT * FROM tab3 +EXCEPT ALL +SELECT * FROM tab4 +UNION +SELECT * FROM tab3 +EXCEPT DISTINCT +SELECT * FROM tab4; + +-- Using MINUS ALL +SELECT * FROM tab3 +MINUS ALL +SELECT * FROM tab4 +UNION +SELECT * FROM tab3 +MINUS DISTINCT +SELECT * FROM tab4; + +-- Chain of set operations +SELECT * FROM tab3 +EXCEPT ALL +SELECT * FROM tab4 +EXCEPT DISTINCT +SELECT * FROM tab3 +EXCEPT DISTINCT +SELECT * FROM tab4; + +-- Join under except all. Should produce empty resultset since both left and right sets +-- are same. +SELECT * +FROM (SELECT tab3.k, + tab4.v + FROM tab3 + JOIN tab4 + ON tab3.k = tab4.k) +EXCEPT ALL +SELECT * +FROM (SELECT tab3.k, + tab4.v + FROM tab3 + JOIN tab4 + ON tab3.k = tab4.k); + +-- Join under except all (2) +SELECT * +FROM (SELECT tab3.k, + tab4.v + FROM tab3 + JOIN tab4 + ON tab3.k = tab4.k) +EXCEPT ALL +SELECT * +FROM (SELECT tab4.v AS k, + tab3.k AS v + FROM tab3 + JOIN tab4 + ON tab3.k = tab4.k); + +-- Group by under ExceptAll +SELECT v FROM tab3 GROUP BY v +EXCEPT ALL +SELECT k FROM tab4 GROUP BY k; + +-- Clean-up +DROP VIEW IF EXISTS tab1; +DROP VIEW IF EXISTS tab2; +DROP VIEW IF EXISTS tab3; +DROP VIEW IF EXISTS tab4; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/except.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/except.sql new file mode 100644 index 000000000000..ffdf1f4f3d24 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/except.sql @@ -0,0 +1,76 @@ +-- Tests different scenarios of except operation +create temporary view t1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", NULL) + as t1(k, v); + +create temporary view t2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5), + ("one", NULL), + (NULL, 5) + as t2(k, v); + + +-- Except operation that will be replaced by left anti join +SELECT * FROM t1 EXCEPT SELECT * FROM t2; + + +-- Except operation that will be replaced by Filter: SPARK-22181 +SELECT * FROM t1 EXCEPT SELECT * FROM t1 where v <> 1 and v <> 2; + + +-- Except operation that will be replaced by Filter: SPARK-22181 +SELECT * FROM t1 where v <> 1 and v <> 22 EXCEPT SELECT * FROM t1 where v <> 2 and v >= 3; + + +-- Except operation that will be replaced by Filter: SPARK-22181 +SELECT t1.* FROM t1, t2 where t1.k = t2.k +EXCEPT +SELECT t1.* FROM t1, t2 where t1.k = t2.k and t1.k != 'one'; + + +-- Except operation that will be replaced by left anti join +SELECT * FROM t2 where v >= 1 and v <> 22 EXCEPT SELECT * FROM t1; + + +-- Except operation that will be replaced by left anti join +SELECT (SELECT min(k) FROM t2 WHERE t2.k = t1.k) min_t2 FROM t1 +MINUS +SELECT (SELECT min(k) FROM t2) abs_min_t2 FROM t1 WHERE t1.k = 'one'; + + +-- Except operation that will be replaced by left anti join +SELECT t1.k +FROM t1 +WHERE t1.v <= (SELECT max(t2.v) + FROM t2 + WHERE t2.k = t1.k) +MINUS +SELECT t1.k +FROM t1 +WHERE t1.v >= (SELECT min(t2.v) + FROM t2 + WHERE t2.k = t1.k); + +-- SPARK-32638: corrects references when adding aliases in WidenSetOperationTypes +CREATE OR REPLACE TEMPORARY VIEW t3 AS VALUES (decimal(1)) tbl(v); +SELECT t.v FROM ( + SELECT v FROM t3 + EXCEPT + SELECT v + v AS v FROM t3 +) t; + +SELECT SUM(t.v) FROM ( + SELECT v FROM t3 + EXCEPT + SELECT v + v AS v FROM t3 +) t; + +-- Clean-up +DROP VIEW IF EXISTS t1; +DROP VIEW IF EXISTS t2; +DROP VIEW IF EXISTS t3; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/execute-immediate.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/execute-immediate.sql new file mode 100644 index 000000000000..f7d27c6c0b03 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/execute-immediate.sql @@ -0,0 +1,149 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES + (10, 'name1', named_struct('f1', 1, 's2', named_struct('f2', 101, 'f3', 'a'))), + (20, 'name2', named_struct('f1', 2, 's2', named_struct('f2', 202, 'f3', 'b'))), + (30, 'name3', named_struct('f1', 3, 's2', named_struct('f2', 303, 'f3', 'c'))), + (40, 'name4', named_struct('f1', 4, 's2', named_struct('f2', 404, 'f3', 'd'))), + (50, 'name5', named_struct('f1', 5, 's2', named_struct('f2', 505, 'f3', 'e'))), + (60, 'name6', named_struct('f1', 6, 's2', named_struct('f2', 606, 'f3', 'f'))), + (70, 'name7', named_struct('f1', 7, 's2', named_struct('f2', 707, 'f3', 'g'))) +AS tbl_view(id, name, data); +CREATE TABLE x (id INT) USING csv; + +DECLARE sql_string STRING; +SET VAR sql_string = 'SELECT * from tbl_view where name = \'name1\''; + +-- test commands +EXECUTE IMMEDIATE 'SET spark.sql.ansi.enabled=true'; +EXECUTE IMMEDIATE 'CREATE TEMPORARY VIEW IDENTIFIER(:tblName) AS SELECT id, name FROM tbl_view' USING 'tbl_view_tmp' as tblName; +EXECUTE IMMEDIATE 'SELECT * FROM tbl_view_tmp'; + +EXECUTE IMMEDIATE 'REFRESH TABLE IDENTIFIER(:tblName)' USING 'x' as tblName; + +-- test execute immediate without parameters +EXECUTE IMMEDIATE sql_string; +EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = \'name1\''; + +-- test positional parameters +SET VAR sql_string = 'SELECT * from tbl_view where name = ? or name = ?'; +DECLARE a STRING; +SET VAR a = 'name1'; +EXECUTE IMMEDIATE sql_string USING 'name1', 'name3'; +EXECUTE IMMEDIATE sql_string USING a, 'name2'; +EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING 'name1', 'name3'; +EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING a, 'name2'; +EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING (a, 'name2'); +-- test positional command +EXECUTE IMMEDIATE 'INSERT INTO x VALUES(?)' USING 1; +SELECT * from x; + +-- test named parameters +SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'; +DECLARE b INT; +SET VAR b = 40; +EXECUTE IMMEDIATE sql_string USING 40 as second, 'name7' as first; +EXECUTE IMMEDIATE sql_string USING b as second, 'name7' as first; +EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 40 as second, 'name7' as first; +EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 'name7' as first, b as second; +-- named parameter used multiple times +EXECUTE IMMEDIATE 'SELECT tbl_view.*, :first as p FROM tbl_view WHERE name = :first' USING 'name7' as first; + +-- test named command and setup for next test +EXECUTE IMMEDIATE 'SET VAR sql_string = ?' USING 'SELECT id from tbl_view where name = :first'; +SELECT sql_string; + +-- test into +DECLARE res_id INT; +EXECUTE IMMEDIATE sql_string INTO res_id USING 'name7' as first; +SELECT res_id; +EXECUTE IMMEDIATE sql_string INTO res_id USING a as first; +SELECT res_id; + +-- test into without using +SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'; +EXECUTE IMMEDIATE 'SELECT 42' INTO res_id; +SELECT res_id; + +-- multiple INTOs +EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO b, a USING 10; +SELECT b, a; + +-- use AS for using positional params +EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where id = ? AND name = ?' USING b as first, a; + +-- empty query +EXECUTE IMMEDIATE 'SELECT 42 WHERE 2 = 1' INTO res_id; +SELECT res_id; + +-- implicit casting +EXECUTE IMMEDIATE 'SELECT \'1707\'' INTO res_id; +SELECT res_id; + +-- test errors +-- string to int error +EXECUTE IMMEDIATE 'SELECT \'invalid_cast_error_expected\'' INTO res_id; + +-- require query when using INTO +EXECUTE IMMEDIATE 'INSERT INTO x VALUES (?)' INTO res_id USING 1; + +-- use column in using - should fail as we expect variable here +EXECUTE IMMEDIATE 'SELECT * FROM tbl_view WHERE ? = id' USING id; + +-- either positional or named parameters must be used +EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where ? = id and :first = name' USING 1, 'name2' as first; + +-- all paramerers must be named +EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where :first = name' USING 1, 'name2' as first; + +-- internal syntax error +EXECUTE IMMEDIATE 'SELCT Fa'; + +-- internal syntax error - test that both parseQuery and parsePlan fail +EXECUTE IMMEDIATE 'SELCT Fa' INTO res_id; + +-- Parameter passed must be STRING +EXECUTE IMMEDIATE b; + +-- test expressions should fail with parser error +SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'; +SET VAR a = 'na'; + +-- expressions not supported - feature not supported +EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first' USING CONCAT(a , "me1") as first; +EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first' USING (SELECT 42) as first, 'name2' as second; + +-- INTO variables not matching scalar types +EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO a, b USING 10; + +-- INTO does not support braces - parser error +EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO (a, b) USING 10; + +-- Error too many rows +EXECUTE IMMEDIATE 'SELECT id FROM tbl_view' INTO res_id; + +-- Error mismatch cardinality +EXECUTE IMMEDIATE 'SELECT id, data.f1 FROM tbl_view' INTO res_id; +EXECUTE IMMEDIATE 'SELECT id FROM tbl_view' INTO res_id, b; + +-- duplicate aliases +EXECUTE IMMEDIATE 'SELECT id FROM tbl_view WHERE id = :first' USING 10 as first, 20 as first; + +-- no alias +DECLARE p = 10; +EXECUTE IMMEDIATE 'SELECT id FROM tbl_view WHERE id = :p' USING p; + +-- mixing literals and named parameters +EXECUTE IMMEDIATE 'SELECT id FROM tbl_view WHERE id = :p' USING p, 'p'; + +-- duplicate into entry +EXECUTE IMMEDIATE 'SELECT id, data.f1 FROM tbl_view WHERE id = 10' INTO res_id, res_id; + +-- nested execute immediate +EXECUTE IMMEDIATE 'EXECUTE IMMEDIATE \'SELECT id FROM tbl_view WHERE id = ? USING 10\''; + +-- sqlString is null +SET VAR sql_string = null; +EXECUTE IMMEDIATE sql_string; + +DROP TABLE x; \ No newline at end of file diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/explain-aqe.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/explain-aqe.sql new file mode 100644 index 000000000000..7aef901da4fb --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/explain-aqe.sql @@ -0,0 +1,4 @@ +--IMPORT explain.sql + +--SET spark.sql.adaptive.enabled=true +--SET spark.sql.maxMetadataStringLength = 500 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/explain-cbo.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/explain-cbo.sql new file mode 100644 index 000000000000..eeb2180f7a54 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/explain-cbo.sql @@ -0,0 +1,27 @@ +--SET spark.sql.cbo.enabled=true +--SET spark.sql.maxMetadataStringLength = 500 + +CREATE TABLE explain_temp1(a INT, b INT) USING PARQUET; +CREATE TABLE explain_temp2(c INT, d INT) USING PARQUET; + +ANALYZE TABLE explain_temp1 COMPUTE STATISTICS FOR ALL COLUMNS; +ANALYZE TABLE explain_temp2 COMPUTE STATISTICS FOR ALL COLUMNS; + +EXPLAIN COST WITH max_store_sales AS +( + SELECT max(csales) tpcds_cmax + FROM ( + SELECT sum(b) csales + FROM explain_temp1 WHERE a < 100 + ) x +), +best_ss_customer AS +( + SELECT c + FROM explain_temp2 + WHERE d > (SELECT * FROM max_store_sales) +) +SELECT c FROM best_ss_customer; + +DROP TABLE explain_temp1; +DROP TABLE explain_temp2; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/explain.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/explain.sql new file mode 100644 index 000000000000..698ca009b4ff --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/explain.sql @@ -0,0 +1,136 @@ +--SET spark.sql.codegen.wholeStage = true +--SET spark.sql.adaptive.enabled = false +--SET spark.sql.maxMetadataStringLength = 500 + +-- Test tables +CREATE table explain_temp1 (key int, val int) USING PARQUET; +CREATE table explain_temp2 (key int, val int) USING PARQUET; +CREATE table explain_temp3 (key int, val int) USING PARQUET; +CREATE table explain_temp4 (key int, val string) USING PARQUET; +CREATE table explain_temp5 (key int) USING PARQUET PARTITIONED BY(val string); + +SET spark.sql.codegen.wholeStage = true; + +-- distinct func +EXPLAIN EXTENDED + SELECT sum(distinct val) + FROM explain_temp1; + +-- single table +EXPLAIN FORMATTED + SELECT key, max(val) + FROM explain_temp1 + WHERE key > 0 + GROUP BY key + ORDER BY key; + +EXPLAIN FORMATTED + SELECT key, max(val) + FROM explain_temp1 + WHERE key > 0 + GROUP BY key + HAVING max(val) > 0; + +-- simple union +EXPLAIN FORMATTED + SELECT key, val FROM explain_temp1 WHERE key > 0 + UNION + SELECT key, val FROM explain_temp1 WHERE key > 1; + +-- Join +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 a, + explain_temp2 b + WHERE a.key = b.key; + +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 a + LEFT OUTER JOIN explain_temp2 b + ON a.key = b.key; + +-- Subqueries nested. +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 + WHERE key = (SELECT max(key) + FROM explain_temp2 + WHERE key = (SELECT max(key) + FROM explain_temp3 + WHERE val > 0) + AND val = 2) + AND val > 3; + +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 + WHERE key = (SELECT max(key) + FROM explain_temp2 + WHERE val > 0) + OR + key = (SELECT avg(key) + FROM explain_temp3 + WHERE val > 0); + +-- Reuse subquery +EXPLAIN FORMATTED + SELECT (SELECT Avg(key) FROM explain_temp1) + (SELECT Avg(key) FROM explain_temp1) + FROM explain_temp1; + +-- CTE + ReuseExchange +EXPLAIN FORMATTED + WITH cte1 AS ( + SELECT * + FROM explain_temp1 + WHERE key > 10 + ) + SELECT * FROM cte1 a, cte1 b WHERE a.key = b.key; + +EXPLAIN FORMATTED + WITH cte1 AS ( + SELECT key, max(val) + FROM explain_temp1 + WHERE key > 10 + GROUP BY key + ) + SELECT * FROM cte1 a, cte1 b WHERE a.key = b.key; + +-- A spark plan which has innerChildren other than subquery +EXPLAIN FORMATTED + CREATE VIEW explain_view AS + SELECT key, val FROM explain_temp1; + +-- HashAggregate +EXPLAIN FORMATTED + SELECT + COUNT(val) + SUM(key) as TOTAL, + COUNT(key) FILTER (WHERE val > 1) + FROM explain_temp1; + +-- ObjectHashAggregate +EXPLAIN FORMATTED + SELECT key, sort_array(collect_set(val))[0] + FROM explain_temp4 + GROUP BY key; + +-- SortAggregate +EXPLAIN FORMATTED + SELECT key, MIN(val) + FROM explain_temp4 + GROUP BY key; + +-- V1 Write +EXPLAIN EXTENDED INSERT INTO TABLE explain_temp5 SELECT * FROM explain_temp4; + +-- cleanup +DROP TABLE explain_temp1; +DROP TABLE explain_temp2; +DROP TABLE explain_temp3; +DROP TABLE explain_temp4; +DROP TABLE explain_temp5; + +-- SPARK-35479: Format PartitionFilters IN strings in scan nodes +CREATE table t(v array) USING PARQUET; +EXPLAIN SELECT * FROM t WHERE v IN (array('a'), null); +DROP TABLE t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/extract.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/extract.sql new file mode 100644 index 000000000000..5fd3362c6964 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/extract.sql @@ -0,0 +1,167 @@ +CREATE TEMPORARY VIEW t AS select '2011-05-06 07:08:09.1234567' as c, to_timestamp_ntz('2011-05-06 07:08:09.1234567') as ntz, interval 10 year 20 month as i, interval 30 day 40 hour 50 minute 6.7890 second as j; + +select extract(year from c), extract(year from ntz), extract(year from i) from t; +select extract(y from c), extract(y from ntz), extract(y from i) from t; +select extract(years from c), extract(years from ntz), extract(years from i) from t; +select extract(yr from c), extract(yr from ntz), extract(yr from i) from t; +select extract(yrs from c), extract(yrs from ntz), extract(yrs from i) from t; + +select extract(yearofweek from c), extract(yearofweek from ntz) from t; + +select extract(quarter from c), extract(quarter from ntz) from t; +select extract(qtr from c), extract(qtr from ntz) from t; + +select extract(month from c), extract(month from ntz), extract(month from i) from t; +select extract(mon from c), extract(mon from ntz), extract(mon from i) from t; +select extract(mons from c), extract(mons from ntz), extract(mons from i) from t; +select extract(months from c), extract(months from ntz), extract(months from i) from t; + +select extract(week from c), extract(week from ntz) from t; +select extract(w from c), extract(w from ntz) from t; +select extract(weeks from c), extract(weeks from ntz) from t; + +select extract(day from c), extract(day from ntz), extract(day from j) from t; +select extract(d from c), extract(d from ntz), extract(d from j) from t; +select extract(days from c), extract(days from ntz), extract(days from j) from t; + +select extract(dayofweek from c), extract(dayofweek from ntz) from t; +select extract(dow from c), extract(dow from ntz) from t; + +select extract(dayofweek_iso from c), extract(dayofweek_iso from ntz) from t; +select extract(dow_iso from c), extract(dow_iso from ntz) from t; + +select extract(doy from c), extract(doy from ntz) from t; + +select extract(hour from c), extract(hour from ntz), extract(hour from j) from t; +select extract(h from c), extract(h from ntz), extract(h from j) from t; +select extract(hours from c), extract(hours from ntz), extract(hours from j) from t; +select extract(hr from c), extract(hr from ntz), extract(hr from j) from t; +select extract(hrs from c), extract(hrs from ntz), extract(hrs from j) from t; + +select extract(minute from c), extract(minute from ntz), extract(minute from j) from t; +select extract(m from c), extract(m from ntz), extract(m from j) from t; +select extract(min from c), extract(min from ntz), extract(min from j) from t; +select extract(mins from c), extract(mins from ntz), extract(mins from j) from t; +select extract(minutes from c), extract(minutes from ntz), extract(minutes from j) from t; + +select extract(second from c), extract(second from ntz), extract(second from j) from t; +select extract(s from c), extract(s from ntz), extract(s from j) from t; +select extract(sec from c), extract(sec from ntz), extract(sec from j) from t; +select extract(seconds from c), extract(seconds from ntz), extract(seconds from j) from t; +select extract(secs from c), extract(secs from ntz), extract(secs from j) from t; + +select extract(not_supported from c) from t; +select extract(not_supported from i) from t; +select extract(not_supported from j) from t; + +select date_part('year', c), date_part('year', ntz), date_part('year', i) from t; +select date_part('y', c), date_part('y', ntz), date_part('y', i) from t; +select date_part('years', c), date_part('years', ntz), date_part('years', i) from t; +select date_part('yr', c), date_part('yr', ntz), date_part('yr', i) from t; +select date_part('yrs', c), date_part('yrs', ntz), date_part('yrs', i) from t; + +select date_part('yearofweek', c), date_part('yearofweek', ntz) from t; + +select date_part('quarter', c), date_part('quarter', ntz) from t; +select date_part('qtr', c), date_part('qtr', ntz) from t; + +select date_part('month', c), date_part('month', ntz), date_part('month', i) from t; +select date_part('mon', c), date_part('mon', ntz), date_part('mon', i) from t; +select date_part('mons', c), date_part('mons', ntz), date_part('mons', i) from t; +select date_part('months', c), date_part('months', ntz), date_part('months', i) from t; + +select date_part('week', c), date_part('week', ntz) from t; +select date_part('w', c), date_part('w', ntz) from t; +select date_part('weeks', c), date_part('weeks', ntz) from t; + +select date_part('day', c), date_part('day', ntz), date_part('day', j) from t; +select date_part('d', c), date_part('d', ntz), date_part('d', j) from t; +select date_part('days', c), date_part('days', ntz), date_part('days', j) from t; + +select date_part('dayofweek', c), date_part('dayofweek', ntz) from t; +select date_part('dow', c), date_part('dow', ntz) from t; + +select date_part('dayofweek_iso', c), date_part('dayofweek_iso', ntz) from t; +select date_part('dow_iso', c), date_part('dow_iso', ntz) from t; + +select date_part('doy', c), date_part('doy', ntz) from t; + +select date_part('hour', c), date_part('hour', ntz), date_part('hour', j) from t; +select date_part('h', c), date_part('h', ntz), date_part('h', j) from t; +select date_part('hours', c), date_part('hours', ntz), date_part('hours', j) from t; +select date_part('hr', c), date_part('hr', ntz), date_part('hr', j) from t; +select date_part('hrs', c), date_part('hrs', ntz), date_part('hrs', j) from t; + +select date_part('minute', c), date_part('minute', ntz), date_part('minute', j) from t; +select date_part('m', c), date_part('m', ntz), date_part('m', j) from t; +select date_part('min', c), date_part('min', ntz), date_part('min', j) from t; +select date_part('mins', c), date_part('mins', ntz), date_part('mins', j) from t; +select date_part('minutes', c), date_part('minutes', ntz), date_part('minutes', j) from t; + +select date_part('second', c), date_part('second', ntz), date_part('second', j) from t; +select date_part('s', c), date_part('s', ntz), date_part('s', j) from t; +select date_part('sec', c), date_part('sec', ntz), date_part('sec', j) from t; +select date_part('seconds', c), date_part('seconds', ntz), date_part('seconds', j) from t; +select date_part('secs', c), date_part('secs', ntz), date_part('secs', j) from t; + +select date_part('not_supported', c) from t; +select date_part(c, c) from t; +select date_part(null, c) from t; + +select date_part(i, i) from t; +select date_part(null, i) from t; + +-- In SPARK-31476, we've supported extract('field', source), too +select extract('year', c) from t; +select extract('quarter', c) from t; +select extract('month', c) from t; +select extract('week', c) from t; +select extract('day', c) from t; +select extract('days', c) from t; +select extract('dayofweek', c) from t; +select extract('dow', c) from t; +select extract('doy', c) from t; +select extract('hour', c) from t; +select extract('minute', c) from t; +select extract('second', c) from t; + +select c - j from t; +select day(c - j) from t; +select extract(day from c - j) from t; +select extract(month from to_timestamp(c) - i) from t; +select extract(second from to_timestamp(c) - j) from t; + +-- extract fields from year-month/day-time intervals +select extract(YEAR from interval '2-1' YEAR TO MONTH); +select date_part('YEAR', interval '2-1' YEAR TO MONTH); +select extract(YEAR from -interval '2-1' YEAR TO MONTH); +select extract(MONTH from interval '2-1' YEAR TO MONTH); +select date_part('MONTH', interval '2-1' YEAR TO MONTH); +select extract(MONTH from -interval '2-1' YEAR TO MONTH); +select date_part(NULL, interval '2-1' YEAR TO MONTH); + +-- invalid +select extract(DAY from interval '2-1' YEAR TO MONTH); +select date_part('DAY', interval '2-1' YEAR TO MONTH); +select date_part('not_supported', interval '2-1' YEAR TO MONTH); + +select extract(DAY from interval '123 12:34:56.789123123' DAY TO SECOND); +select date_part('DAY', interval '123 12:34:56.789123123' DAY TO SECOND); +select extract(DAY from -interval '123 12:34:56.789123123' DAY TO SECOND); +select extract(HOUR from interval '123 12:34:56.789123123' DAY TO SECOND); +select date_part('HOUR', interval '123 12:34:56.789123123' DAY TO SECOND); +select extract(HOUR from -interval '123 12:34:56.789123123' DAY TO SECOND); +select extract(MINUTE from interval '123 12:34:56.789123123' DAY TO SECOND); +select date_part('MINUTE', interval '123 12:34:56.789123123' DAY TO SECOND); +select extract(MINUTE from -interval '123 12:34:56.789123123' DAY TO SECOND); +select extract(SECOND from interval '123 12:34:56.789123123' DAY TO SECOND); +select date_part('SECOND', interval '123 12:34:56.789123123' DAY TO SECOND); +select extract(SECOND from -interval '123 12:34:56.789123123' DAY TO SECOND); +select date_part(NULL, interval '123 12:34:56.789123123' DAY TO SECOND); + +select extract(MONTH from interval '123 12:34:56.789123123' DAY TO SECOND); +select date_part('not_supported', interval '123 12:34:56.789123123' DAY TO SECOND); + +-- alias for date_part +select datepart('year', c), datepart('year', ntz), datepart('year', i) from t; +select datepart('DAY', interval '123 12:34:56.789123123' DAY TO SECOND); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-analytics.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-analytics.sql new file mode 100644 index 000000000000..d6381e59e0d8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-analytics.sql @@ -0,0 +1,93 @@ +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData(a, b); + +-- CUBE on overlapping columns +SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH CUBE; + +SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH CUBE; + +-- ROLLUP on overlapping columns +SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP; + +SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH ROLLUP; + +CREATE OR REPLACE TEMPORARY VIEW courseSales AS SELECT * FROM VALUES +("dotNET", 2012, 10000), ("Java", 2012, 20000), ("dotNET", 2012, 5000), ("dotNET", 2013, 48000), ("Java", 2013, 30000) +AS courseSales(course, year, earnings); + +-- ROLLUP +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY course, year; +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year, (course, year)) ORDER BY course, year; +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year, (course, year), ()) ORDER BY course, year; + +-- CUBE +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, year; +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year, (course, year)) ORDER BY course, year; +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year, (course, year), ()) ORDER BY course, year; + +-- GROUPING SETS +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year); +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year, ()); +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course); +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year); + +-- Partial ROLLUP/CUBE/GROUPING SETS +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, CUBE(course, year) ORDER BY course, year; +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year), ROLLUP(course, year) ORDER BY course, year; +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year), ROLLUP(course, year), GROUPING SETS(course, year) ORDER BY course, year; + +-- GROUPING SETS with aggregate functions containing groupBy columns +SELECT course, SUM(earnings) AS sum FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum; +SELECT course, SUM(earnings) AS sum, GROUPING_ID(course, earnings) FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum; + +-- GROUPING/GROUPING_ID +SELECT course, year, GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales +GROUP BY CUBE(course, year); +SELECT course, year, GROUPING(course) FROM courseSales GROUP BY course, year; +SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY course, year; +SELECT course, year, grouping__id FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, year; + +-- GROUPING/GROUPING_ID in having clause +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) +HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0 ORDER BY course, year; +SELECT course, year FROM courseSales GROUP BY course, year HAVING GROUPING(course) > 0; +SELECT course, year FROM courseSales GROUP BY course, year HAVING GROUPING_ID(course) > 0; +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) HAVING grouping__id > 0; + +-- GROUPING/GROUPING_ID in orderBy clause +SELECT course, year, GROUPING(course), GROUPING(year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, year; +SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, year; +SELECT course, year FROM courseSales GROUP BY course, year ORDER BY GROUPING(course); +SELECT course, year FROM courseSales GROUP BY course, year ORDER BY GROUPING_ID(course); +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, year; + +-- Aliases in SELECT could be used in ROLLUP/CUBE/GROUPING SETS +SELECT a + b AS k1, b AS k2, SUM(a - b) FROM testData GROUP BY CUBE(k1, k2); +SELECT a + b AS k, b, SUM(a - b) FROM testData GROUP BY ROLLUP(k, b); +SELECT a + b, b AS k, SUM(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k); + +-- GROUP BY use mixed Separate columns and CUBE/ROLLUP/Gr +SELECT a, b, count(1) FROM testData GROUP BY a, b, CUBE(a, b); +SELECT a, b, count(1) FROM testData GROUP BY a, b, ROLLUP(a, b); +SELECT a, b, count(1) FROM testData GROUP BY CUBE(a, b), ROLLUP(a, b); +SELECT a, b, count(1) FROM testData GROUP BY a, CUBE(a, b), ROLLUP(b); +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS((a, b), (a), ()); +SELECT a, b, count(1) FROM testData GROUP BY a, CUBE(a, b), GROUPING SETS((a, b), (a), ()); +SELECT a, b, count(1) FROM testData GROUP BY a, CUBE(a, b), ROLLUP(a, b), GROUPING SETS((a, b), (a), ()); + +-- Support nested CUBE/ROLLUP/GROUPING SETS in GROUPING SETS +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(ROLLUP(a, b)); +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b), (a), ())); + +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS((a, b), GROUPING SETS(ROLLUP(a, b))); +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS((a, b, a, b), (a, b, a), (a, b)); +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b, a, b), (a, b, a), (a, b))); + +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(ROLLUP(a, b), CUBE(a, b)); +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b), (a), ()), GROUPING SETS((a, b), (a), (b), ())); +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS((a, b), (a), (), (a, b), (a), (b), ()); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by-alias.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by-alias.sql new file mode 100644 index 000000000000..75afc82f998d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by-alias.sql @@ -0,0 +1,58 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b); + +-- GROUP BY alias should work with case insensitive names +SELECT a from testData GROUP BY A; + +-- Aliases in SELECT could be used in GROUP BY +SELECT a AS k, COUNT(b) FROM testData GROUP BY k; +SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1; +SELECT col1 AS k, SUM(col2) FROM testData AS t(col1, col2) GROUP BY k; +SELECT a as alias FROM testData GROUP BY ALIAS; + +-- GROUP BY literal +SELECT a AS k FROM testData GROUP BY 'k'; +SELECT 1 AS k FROM testData GROUP BY 'k'; + +-- GROUP BY alias with the function name +SELECT concat_ws(' ', a, b) FROM testData GROUP BY `concat_ws( , a, b)`; + +-- GROUP BY column with name same as an alias used in the project list +SELECT 1 AS a FROM testData GROUP BY a; +SELECT 1 AS a FROM testData GROUP BY `a`; + +-- GROUP BY implicit alias +SELECT 1 GROUP BY `1`; + +-- GROUP BY alias with the subquery name +SELECT (SELECT a FROM testData) + (SELECT b FROM testData) group by `(scalarsubquery() + scalarsubquery())`; + +-- GROUP BY with expression subqueries +SELECT a, count(*) FROM testData GROUP BY (SELECT b FROM testData); +SELECT a, count(*) FROM testData GROUP BY a, (SELECT b FROM testData); +SELECT a, count(*) FROM testData GROUP BY a, (SELECT b FROM testData LIMIT 1); +SELECT a, count(*) FROM testData GROUP BY a, b IN (SELECT a FROM testData); +SELECT a, count(*) FROM testData GROUP BY a, a IN (SELECT b FROM testData); +SELECT a, count(*) FROM testData GROUP BY a, EXISTS(SELECT b FROM testData); + +-- GROUP BY alias with invalid col in SELECT list +SELECT a AS k, COUNT(non_existing) FROM testData GROUP BY k; + +-- Aggregate functions cannot be used in GROUP BY +SELECT COUNT(b) AS k FROM testData GROUP BY k; + +-- Ordinal is replaced correctly when grouping by alias of a literal +SELECT MAX(col1), 3 as abc FROM VALUES(1),(2),(3),(4) GROUP BY col1 % abc; + +-- turn off group by aliases +set spark.sql.groupByAliases=false; + +-- Check analysis exceptions +SELECT a AS k, COUNT(b) FROM testData GROUP BY k; +SELECT 1 GROUP BY `1`; +SELECT 1 AS col FROM testData GROUP BY `col`; + +-- GROUP BY attribute takes precedence over alias +SELECT 1 AS a FROM testData GROUP BY `a`; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by-all-duckdb.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by-all-duckdb.sql new file mode 100644 index 000000000000..ac6d425a956f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by-all-duckdb.sql @@ -0,0 +1,29 @@ +-- group by all +-- additional test cases from DuckDB, given to us by Mosha + +create temporary view integers as select * from values + (0, 1), + (0, 2), + (1, 3), + (1, NULL) + as integers(g, i); + + +SELECT g, SUM(i) FROM integers GROUP BY ALL ORDER BY 1; + +SELECT g, SUM(i), COUNT(*), COUNT(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1; + +SELECT i%2, SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1; + +SELECT (g+i)%2, SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1; + +SELECT (g+i)%2 + SUM(i), SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1; + +SELECT g, i, g%2, SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1, 2, 3, 4; + +SELECT c0 FROM (SELECT 1 c0) t0 GROUP BY ALL HAVING c0>0; + +SELECT c0 FROM (SELECT 1 c0, 1 c1 UNION ALL SELECT 1, 2) t0 GROUP BY ALL ORDER BY c0; + +SELECT c0 FROM (SELECT 1 c0, 1 c1 UNION ALL SELECT 1, 2) t0 GROUP BY ALL HAVING c1>0 ORDER BY c0; + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by-all-mosha.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by-all-mosha.sql new file mode 100644 index 000000000000..451f745a97ee --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by-all-mosha.sql @@ -0,0 +1,32 @@ +-- group by all +-- additional group by star test cases from Mosha +create temporary view stuff as select * from values + (42, 9.75, 'hello world', '1970-08-07', '13.37', array(1,20,300)), + (1337, 1.2345, 'oh no', '2000-01-01', '42.0', array(4000,50000,600000)), + (42, 13.37, 'test', '1970-08-07', '1234567890', array(7000000,80000000,900000000)) + as stuff(i, f, s, t, d, a); + +SELECT 100 * SUM(i) + SUM(f) / COUNT(s) AS f1, i AS f2 FROM stuff GROUP BY ALL ORDER BY f2; + +SELECT i + 1 AS i1, COUNT(i - 2) ci, f / i AS fi, SUM(i + f) sif FROM stuff GROUP BY ALL ORDER BY 1, 3; + +SELECT i AS i, COUNT(i) ci, f AS f, SUM(i + f) sif FROM stuff GROUP BY ALL ORDER BY 1, i, 2, ci, 3, f, 4, sif; + +SELECT i + 1, f / i, substring(s, 2, 3), extract(year from t), octet_length(d), size(a) FROM stuff +GROUP BY ALL ORDER BY 1, 3, 4, 5, 6, 2; + +-- unlike Mosha, I'm failing this case because IMO it is too implicit to automatically group by i. +SELECT i + SUM(f) FROM stuff GROUP BY ALL; + +SELECT s AS s, COUNT(*) c FROM stuff GROUP BY ALL HAVING SUM(f) > 0 ORDER BY s; + +SELECT SUM(i) si FROM stuff GROUP BY ALL HAVING si > 2; + +SELECT SUM(i) si FROM stuff GROUP BY ALL HAVING si < 2; + +-- negative test, i shouldn't propagate through the aggregate so the having should fail +SELECT SUM(i) si FROM stuff GROUP BY ALL HAVING i > 2; + +-- negative test, i shouldn't propagate through the aggregate so the order by should fail +SELECT SUM(i) si FROM stuff GROUP BY ALL ORDER BY i DESC; + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by-all.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by-all.sql new file mode 100644 index 000000000000..4400c0b57866 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by-all.sql @@ -0,0 +1,85 @@ +-- group by all +-- see https://www.linkedin.com/posts/mosha_duckdb-firebolt-snowflake-activity-7009615821006131200-VQ0o + +create temporary view data as select * from values + ("USA", "San Francisco", "Reynold", 1, 11.0), + ("USA", "San Francisco", "Matei", 2, 12.0), + ("USA", "Berkeley", "Xiao", 3, 13.0), + ("China", "Hangzhou", "Wenchen", 4, 14.0), + ("China", "Shanghai", "Shanghaiese", 5, 15.0), + ("Korea", "Seoul", "Hyukjin", 6, 16.0), + ("UK", "London", "Sean", 7, 17.0) + as data(country, city, name, id, power); + +-- basic +select country, count(*) from data group by ALL; + +-- different case +select country, count(*) from data group by aLl; + +-- a column named "all" would still work +select all, city, count(*) from (select country as all, city, id from data) group by all, city; + +-- a column named "all" should take precedence over the normal group by all expansion +-- if all refers to the column, then the following should return 3 rows. +-- if all refers to the global aggregate, then 1 row. +SELECT count(1) FROM VALUES(1), (2), (3) AS T(all) GROUP BY all; + +-- two grouping columns and two aggregates +select country, city, count(*), sum(power) from data group by all; + +-- different ordering +select count(*), country, city, sum(power) from data group by all; + +-- alias in grouping column +select country as con, count(*) from data group by all; + + +-- alias in aggregate column +select country, count(*) as cnt from data group by all; + +-- scalar expression in grouping column +select upper(country), count(*) as powerup from data group by all; + +-- scalar expression in aggregate column +select country, sum(power) + 10 as powerup from data group by all; + +-- group by all without aggregate, which should just become a distinct +select country, city from data group by all; + +-- make sure aliases are propagated through correctly +select con, powerup from + (select country as con, sum(power) + 10 as powerup from data group by all); + +-- having +select country, count(id) as cnt from data group by all having cnt > 1; + +-- no grouping column +select count(id) from data group by all; + +-- a more complex no grouping column case +select count(id + power / 2) * 3 from data group by all; + +-- no grouping column on an empty relation +-- this should still return one row because we rewrite this to a global aggregate, as opposed to +-- returning zero row (grouping by a constant). +select count(*) from (select * from data where country = "DNS") group by all; + +-- complex cases that we choose not to infer; fail with a useful error message +select id + count(*) from data group by all; + +-- an even more complex case that we choose not to infer; fail with a useful error message +select (id + id) / 2 + count(*) * 2 from data group by all; + +-- uncorrelated subquery should work +select country, (select count(*) from data) as cnt, count(id) as cnt_id from data group by all; + +-- correlated subquery should also work +select country, (select count(*) from data d1 where d1.country = d2.country), count(id) from data d2 group by all; + +-- correlated subquery together with aggregate function doesn't work. +-- make sure we report the right error UNRESOLVED_ALL_IN_GROUP_BY, rather than some random subquery error. +select (select count(*) from data d1 where d1.country = d2.country) + count(id) from data d2 group by all; + +-- SELECT list contains unresolved column, should not report UNRESOLVED_ALL_IN_GROUP_BY +select non_exist from data group by all; \ No newline at end of file diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by-filter.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by-filter.sql new file mode 100644 index 000000000000..6d8d0790ec8e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by-filter.sql @@ -0,0 +1,179 @@ +-- Test filter clause for aggregate expression with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b); + +CREATE OR REPLACE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id); + +CREATE OR REPLACE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +CREATE OR REPLACE TEMPORARY VIEW FilterExpressionTestData AS SELECT * FROM VALUES + (1, 2, "asd"), + (3, 4, "fgh"), + (5, 6, "jkl") +AS FilterExpressionTestData(num1, num2, str); + +-- Aggregate with filter and empty GroupBy expressions. +SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData; +SELECT COUNT(a) FILTER (WHERE a = 1), COUNT(b) FILTER (WHERE a > 1) FROM testData; +SELECT COUNT(id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp; +SELECT COUNT(id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp; +SELECT COUNT(id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")) FROM emp; +SELECT COUNT(id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") = "2001-01-01") FROM emp; +SELECT COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp; +SELECT COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp; +SELECT COUNT(DISTINCT id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")), COUNT(DISTINCT id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp; +SELECT SUM(salary), COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp; +SELECT COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData; +SELECT COUNT(DISTINCT id) FILTER (WHERE true) FROM emp; +SELECT COUNT(DISTINCT id) FILTER (WHERE false) FROM emp; +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id = 40) FROM emp; +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id = 40) FROM emp; +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id > 0) FROM emp; +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id > 0) FROM emp; +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id = 40) FROM emp; +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id = 40) FROM emp; +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id > 0) FROM emp; +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id > 0) FROM emp; + +-- Aggregate with filter and non-empty GroupBy expressions. +SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData GROUP BY a; +SELECT a, COUNT(b) FILTER (WHERE a != 2) FROM testData GROUP BY b; +SELECT COUNT(a) FILTER (WHERE a >= 0), COUNT(b) FILTER (WHERE a >= 3) FROM testData GROUP BY a; +SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > date "2003-01-01") FROM emp GROUP BY dept_id; +SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > to_date("2003-01-01")) FROM emp GROUP BY dept_id; +SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > to_timestamp("2003-01-01 00:00:00")) FROM emp GROUP BY dept_id; +SELECT dept_id, SUM(salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") > "2003-01-01") FROM emp GROUP BY dept_id; +SELECT dept_id, SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id; +SELECT dept_id, SUM(DISTINCT salary), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id; +SELECT dept_id, SUM(DISTINCT salary) FILTER (WHERE hiredate > date "2001-01-01"), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id; +SELECT dept_id, COUNT(id), SUM(DISTINCT salary), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") > "2001-01-01") FROM emp GROUP BY dept_id; +SELECT b, COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData GROUP BY b; + +-- Aggregate with filter and grouped by literals. +SELECT 'foo', COUNT(a) FILTER (WHERE b <= 2) FROM testData GROUP BY 1; +SELECT 'foo', SUM(salary) FILTER (WHERE hiredate >= date "2003-01-01") FROM emp GROUP BY 1; +SELECT 'foo', SUM(salary) FILTER (WHERE hiredate >= to_date("2003-01-01")) FROM emp GROUP BY 1; +SELECT 'foo', SUM(salary) FILTER (WHERE hiredate >= to_timestamp("2003-01-01")) FROM emp GROUP BY 1; + +-- Aggregate with filter, more than one aggregate function goes with distinct. +select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary), sum(salary) filter (where id + dept_id > 500) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id + dept_id > 500) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id; +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id; +select dept_id, sum(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id; +select dept_id, sum(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id; +select dept_id, avg(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id; +select dept_id, count(distinct emp_name, hiredate) filter (where id > 200), sum(salary) from emp group by dept_id; +select dept_id, count(distinct emp_name, hiredate) filter (where id > 0), sum(salary) from emp group by dept_id; +select dept_id, count(distinct 1), count(distinct 1) filter (where id > 200), sum(salary) from emp group by dept_id; + +-- Aggregate with filter and grouped by literals (hash aggregate), here the input table is filtered using WHERE. +SELECT 'foo', APPROX_COUNT_DISTINCT(a) FILTER (WHERE b >= 0) FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate with filter and grouped by literals (sort aggregate), here the input table is filtered using WHERE. +SELECT 'foo', MAX(STRUCT(a)) FILTER (WHERE b >= 1) FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate with filter and complex GroupBy expressions. +SELECT a + b, COUNT(b) FILTER (WHERE b >= 2) FROM testData GROUP BY a + b; +SELECT a + 2, COUNT(b) FILTER (WHERE b IN (1, 2)) FROM testData GROUP BY a + 1; +SELECT a + 1 + 1, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY a + 1; + +-- Aggregate with filter, foldable input and multiple distinct groups. +SELECT COUNT(DISTINCT b) FILTER (WHERE b > 0), COUNT(DISTINCT b, c) FILTER (WHERE b > 0 AND c > 2) +FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; + +-- Check analysis exceptions +SELECT a AS k, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY k; + +-- Aggregate with filter contains exists subquery +SELECT emp.dept_id, + avg(salary), + avg(salary) FILTER (WHERE id > (SELECT 200)) +FROM emp +GROUP BY dept_id; + +SELECT emp.dept_id, + avg(salary), + avg(salary) FILTER (WHERE emp.dept_id = (SELECT dept_id FROM dept LIMIT 1)) +FROM emp +GROUP BY dept_id; + +-- [SPARK-30220] Support Filter expression uses IN/EXISTS predicate sub-queries +SELECT emp.dept_id, + avg(salary), + avg(salary) FILTER (WHERE EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id)) +FROM emp +GROUP BY dept_id; + +SELECT emp.dept_id, + Sum(salary), + Sum(salary) FILTER (WHERE NOT EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id)) +FROM emp +GROUP BY dept_id; + +SELECT emp.dept_id, + avg(salary), + avg(salary) FILTER (WHERE emp.dept_id IN (SELECT DISTINCT dept_id + FROM dept)) +FROM emp +GROUP BY dept_id; +SELECT emp.dept_id, + Sum(salary), + Sum(salary) FILTER (WHERE emp.dept_id NOT IN (SELECT DISTINCT dept_id + FROM dept)) +FROM emp +GROUP BY dept_id; + +-- Aggregate with filter is subquery +SELECT t1.b FROM (SELECT COUNT(b) FILTER (WHERE a >= 2) AS b FROM testData) t1; + +-- SPARK-47256: Wrong use of FILTER expression in aggregate functions +SELECT count(num1) FILTER (WHERE rand(int(num2)) > 1) FROM FilterExpressionTestData; + +SELECT sum(num1) FILTER (WHERE str) FROM FilterExpressionTestData; + +SELECT sum(num1) FILTER (WHERE max(num2) > 1) FROM FilterExpressionTestData; + +SELECT sum(num1) FILTER (WHERE nth_value(num2, 2) OVER(ORDER BY num2) > 1) FROM FilterExpressionTestData; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by-ordinal.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by-ordinal.sql new file mode 100644 index 000000000000..b773396c050d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by-ordinal.sql @@ -0,0 +1,96 @@ +-- group by ordinal positions + +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b); + +-- basic case +select a, sum(b) from data group by 1; + +-- constant case +select 1, 2, sum(b) from data group by 1, 2; + +-- duplicate group by column +select a, 1, sum(b) from data group by a, 1; +select a, 1, sum(b) from data group by 1, 2; + +-- group by a non-aggregate expression's ordinal +select a, b + 2, count(2) from data group by a, 2; + +-- with alias +select a as aa, b + 2 as bb, count(2) from data group by 1, 2; + +-- foldable non-literal: this should be the same as no grouping. +select sum(b) from data group by 1 + 0; + +-- negative cases: ordinal out of range +select a, b from data group by -1; +select a, b from data group by 0; +select a, b from data group by 3; + +-- negative case: position is an aggregate expression +select a, b, sum(b) from data group by 3; +select a, b, sum(b) + 2 from data group by 3; + +-- negative case: nondeterministic expression +select a, rand(0), sum(b) +from +(select /*+ REPARTITION(1) */ a, b from data) group by a, 2; + +-- negative case: star +select * from data group by a, b, 1; + +-- group by ordinal followed by order by +select a, count(a) from (select 1 as a) tmp group by 1 order by 1; + +-- group by ordinal followed by having +select count(a), a from (select 1 as a) tmp group by 2 having a > 0; + +-- mixed cases: group-by ordinals and aliases +select a, a AS k, count(b) from data group by k, 1; + +-- can use ordinal in CUBE +select a, b, count(1) from data group by cube(1, 2); + +-- mixed cases: can use ordinal in CUBE +select a, b, count(1) from data group by cube(1, b); + +-- can use ordinal with cube +select a, b, count(1) from data group by 1, 2 with cube; + +-- can use ordinal in ROLLUP +select a, b, count(1) from data group by rollup(1, 2); + +-- mixed cases: can use ordinal in ROLLUP +select a, b, count(1) from data group by rollup(1, b); + +-- can use ordinal with rollup +select a, b, count(1) from data group by 1, 2 with rollup; + +-- can use ordinal in GROUPING SETS +select a, b, count(1) from data group by grouping sets((1), (2), (1, 2)); + +-- mixed cases: can use ordinal in GROUPING SETS +select a, b, count(1) from data group by grouping sets((1), (b), (a, 2)); + +select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2)); + +-- range error +select a, b, count(1) from data group by a, -1; + +select a, b, count(1) from data group by a, 3; + +select a, b, count(1) from data group by cube(-1, 2); + +select a, b, count(1) from data group by cube(1, 3); + +-- turn off group by ordinal +set spark.sql.groupByOrdinal=false; + +-- can now group by negative literal +select sum(b) from data group by -1; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by.sql new file mode 100644 index 000000000000..c1b5e88c43f2 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/group-by.sql @@ -0,0 +1,329 @@ +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b); + +-- Aggregate with empty GroupBy expressions. +SELECT a, COUNT(b) FROM testData; +SELECT COUNT(a), COUNT(b) FROM testData; + +-- Aggregate with non-empty GroupBy expressions. +SELECT a, COUNT(b) FROM testData GROUP BY a; +SELECT a, COUNT(b) FROM testData GROUP BY b; +SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a; + +-- Aggregate grouped by literals. +SELECT 'foo', COUNT(a) FROM testData GROUP BY 1; + +-- Aggregate grouped by literals (whole stage code generation). +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate grouped by literals (hash aggregate). +SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate grouped by literals (sort aggregate). +SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate with complex GroupBy expressions. +SELECT a + b, COUNT(b) FROM testData GROUP BY a + b; +SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1; +SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1; + +-- struct() in group by +SELECT count(1) FROM testData GROUP BY struct(a + 0.1 AS aa); + +-- Aggregate with nulls. +SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) +FROM testData; + +-- Aggregate with foldable input and multiple distinct groups. +SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v); +SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a; + +-- turn off group by aliases +set spark.sql.groupByAliases=false; + +-- Aggregate with empty input and non-empty GroupBy expressions. +SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a; + +-- Aggregate with empty input and empty GroupBy expressions. +SELECT COUNT(1) FROM testData WHERE false; +SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t; + +-- Aggregate with empty GroupBy expressions and filter on top +SELECT 1 from ( + SELECT 1 AS z, + MIN(a.x) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z; + +-- SPARK-24369 multiple distinct aggregations having the same argument set +SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y); + +-- SPARK-25708 HAVING without GROUP BY means global aggregate +SELECT 1 FROM range(10) HAVING true; + +SELECT 1 FROM range(10) HAVING MAX(id) > 0; + +SELECT id FROM range(10) HAVING id > 0; + +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=true; + +SELECT 1 FROM range(10) HAVING true; + +SELECT 1 FROM range(10) HAVING MAX(id) > 0; + +SELECT id FROM range(10) HAVING id > 0; + +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=false; + +-- Test data +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v); + +-- empty table +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0; + +-- all null values +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4; + +-- aggregates are null Filtering +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5; + +-- group by +SELECT k, every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k; + +-- having +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false; +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL; + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY k; + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY k; + +-- input type checking Int +SELECT every(1); + +-- input type checking Short +SELECT some(1S); + +-- input type checking Long +SELECT any(1L); + +-- input type checking String +SELECT every("true"); + +-- input type checking Decimal +SELECT bool_and(1.0); + +-- input type checking double +SELECT bool_or(1.0D); + +-- every/some/any aggregates/bool_and/bool_or are supported as windows expression. +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, bool_and(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, bool_or(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; + +-- Having referencing aggregate expressions is ok. +SELECT count(*) FROM test_agg HAVING count(*) > 1L; +SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true; + +-- Aggrgate expressions can be referenced through an alias +SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L; + +-- Error when aggregate expressions are in where clause directly +SELECT count(*) FROM test_agg WHERE count(*) > 1L; +SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L; +SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1; + +-- Aggregate with multiple distinct decimal columns +SELECT AVG(DISTINCT decimal_col), SUM(DISTINCT decimal_col) FROM VALUES (CAST(1 AS DECIMAL(9, 0))) t(decimal_col); + +-- SPARK-34581: Don't optimize out grouping expressions from aggregate expressions without aggregate function +SELECT not(a IS NULL), count(*) AS c +FROM testData +GROUP BY a IS NULL; + +SELECT if(not(a IS NULL), rand(0), 1), count(*) AS c +FROM testData +GROUP BY a IS NULL; + + +-- Histogram aggregates with different numeric input types +SELECT + histogram_numeric(col, 2) as histogram_2, + histogram_numeric(col, 3) as histogram_3, + histogram_numeric(col, 5) as histogram_5, + histogram_numeric(col, 10) as histogram_10 +FROM VALUES + (1), (2), (3), (4), (5), (6), (7), (8), (9), (10), + (11), (12), (13), (14), (15), (16), (17), (18), (19), (20), + (21), (22), (23), (24), (25), (26), (27), (28), (29), (30), + (31), (32), (33), (34), (35), (3), (37), (38), (39), (40), + (41), (42), (43), (44), (45), (46), (47), (48), (49), (50) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1), (2), (3) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1L), (2L), (3L) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1F), (2F), (3F) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1D), (2D), (3D) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (1S), (2S), (3S) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BYTE)), (CAST(2 AS BYTE)), (CAST(3 AS BYTE)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS TINYINT)), (CAST(2 AS TINYINT)), (CAST(3 AS TINYINT)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS SMALLINT)), (CAST(2 AS SMALLINT)), (CAST(3 AS SMALLINT)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BIGINT)), (CAST(2 AS BIGINT)), (CAST(3 AS BIGINT)) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS DECIMAL(4, 2))), (CAST(2 AS DECIMAL(4, 2))), (CAST(3 AS DECIMAL(4, 2))) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (TIMESTAMP '2017-03-01 00:00:00'), + (TIMESTAMP '2017-04-01 00:00:00'), (TIMESTAMP '2017-05-01 00:00:00') AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '100-00' YEAR TO MONTH), + (INTERVAL '110-00' YEAR TO MONTH), (INTERVAL '120-00' YEAR TO MONTH) AS tab(col); +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '12 20:4:0' DAY TO SECOND), + (INTERVAL '12 21:4:0' DAY TO SECOND), (INTERVAL '12 22:4:0' DAY TO SECOND) AS tab(col); +SELECT histogram_numeric(col, 3) +FROM VALUES (NULL), (NULL), (NULL) AS tab(col); +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)) AS tab(col); +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS INT)), (CAST(NULL AS INT)), (CAST(NULL AS INT)) AS tab(col); + +-- SPARK-27974: Support ANSI Aggregate Function: array_agg +SELECT + collect_list(col), + array_agg(col) +FROM VALUES + (1), (2), (1) AS tab(col); +SELECT + a, + collect_list(b), + array_agg(b) +FROM VALUES + (1,4),(2,3),(1,4),(2,4) AS v(a,b) +GROUP BY a; + +-- SPARK-44846: PushFoldableIntoBranches in complex grouping expressions cause bindReference error +SELECT c * 2 AS d +FROM ( + SELECT if(b > 1, 1, b) AS c + FROM ( + SELECT if(a < 0, 0, a) AS b + FROM VALUES (-1), (1), (2) AS t1(a) + ) t2 + GROUP BY b + ) t3 +GROUP BY c; + +-- SPARK-45599: Check that "weird" doubles group and sort as desired. +SELECT col1, count(*) AS cnt +FROM VALUES + (0.0), + (-0.0), + (double('NaN')), + (double('NaN')), + (double('Infinity')), + (double('Infinity')), + (-double('Infinity')), + (-double('Infinity')) +GROUP BY col1 +ORDER BY col1 +; + +-- SC-170296: Verify that group by works when MapType is inside complex type for column type +-- ARRAY> +SELECT count(*) +FROM VALUES (ARRAY(MAP(1, 2, 2, 3), MAP(1, 3))), (ARRAY(MAP(2, 3), MAP(1, 3))), (ARRAY(MAP(2, 3, 1, 2), MAP(1, 3))) as t(a) +GROUP BY a; + +-- STRUCT> +SELECT count(*) +FROM VALUES (named_struct('b', map(1, 2, 2, 3))), (named_struct('b', map(1, 3))), (named_struct('b', map(2, 3, 1, 2))) as t(a) +GROUP BY a; + +SELECT count(*) +FROM VALUES (named_struct('b', map(1, 2, 2, 3))), (named_struct('b', map(1, 3))), (named_struct('b', map(2, 3, 1, 2))) as t(a) +GROUP BY a.b; + +-- STRUCT>> +SELECT count(*) +FROM VALUES (named_struct('b', array(map(1, 2, 2, 3), map(1, 3)))), (named_struct('b', array(map(2, 3), map(1, 3)))), (named_struct('b', array(map(2, 3, 1, 2), map(1, 3)))) as t(a) +GROUP BY a; + +-- ARRAY>> +SELECT count(*) +FROM VALUES (ARRAY(named_struct('b', map(1, 2, 2, 3)), named_struct('b', map(1, 3)))), (ARRAY(named_struct('b', map(2, 3)), named_struct('b', map(1, 3)))), (ARRAY(named_struct('b', map(2, 3, 1, 2)), named_struct('b', map(1, 3)))) as t(a) +group BY a; + +-- MAP,INT> +SELECT count(*) +FROM VALUES (map(array(1, 2), 2, array(2, 3), 3)), (map(array(1, 3), 3)), (map(array(2, 3), 3, array(1, 2), 2)) as t(a) +group BY a; + +SELECT count(*) +FROM VALUES (map(array(1, 2, 3), 3)), (map(array(3, 2, 1), 3)) as t(a) +group BY a; + +-- ARRAY,INT>> +SELECT count(*) +FROM VALUES (ARRAY(map(array(1, 2), 2, array(2, 3), 3))), (ARRAY(MAP(ARRAY(1, 3), 3))), (ARRAY(map(array(2, 3), 3, array(1, 2), 2))) as t(a) +group BY a; + +-- MAP,INT> +SELECT count(*) +FROM VALUES (map(named_struct('b', 1), 2, named_struct('b', 2), 3)), (map(named_struct('b', 1), 3)), (map(named_struct('b', 2), 3, named_struct('b', 1), 2)) as t(a) +group BY a; + +-- STRUCT,INT>> +SELECT count(*) +FROM VALUES (named_struct('b', map(named_struct('c', 1), 2, named_struct('c', 2), 3))), (named_struct('b', map(named_struct('c', 1), 3))), (named_struct('b', map(named_struct('c', 2), 3, named_struct('c', 1), 2))) as t(a) +group BY a; + +SELECT count(*) +FROM VALUES (named_struct('b', map(named_struct('c', 1), 2, named_struct('c', 2), 3))), (named_struct('b', map(named_struct('c', 1), 3))), (named_struct('b', map(named_struct('c', 2), 3, named_struct('c', 1), 2))) as t(a) +group BY a.b; + +-- Map valueType contains MapType (possibly nested) +-- MAP> +SELECT count(*) +FROM VALUES (Map(1, Map(1,2), 2, Map(2, 3, 1, 2))), (Map(2, Map(1, 2, 2,3), 1, Map(1, 2))), (Map(1, Map(1,2), 2, Map(2, 4))) as t(a) +GROUP BY a; + +-- MAP>> +SELECT count(*) +FROM VALUES (Map(1, Array(Map(1,2)), 2, Array(Map(2, 3, 1, 2)))), (Map(2, Array(Map(1, 2, 2,3)), 1, Array(Map(1, 2)))), (Map(1, Array(Map(1,2)), 2, Array(Map(2, 4)))) as t(a) +GROUP BY a; \ No newline at end of file diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/grouping_set.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/grouping_set.sql new file mode 100644 index 000000000000..909c36c926c1 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/grouping_set.sql @@ -0,0 +1,66 @@ +CREATE TEMPORARY VIEW grouping AS SELECT * FROM VALUES + ("1", "2", "3", 1), + ("4", "5", "6", 1), + ("7", "8", "9", 1) + as grouping(a, b, c, d); + +-- SPARK-17849: grouping set throws NPE #1 +SELECT a, b, c, count(d) FROM grouping GROUP BY a, b, c GROUPING SETS (()); + +-- SPARK-17849: grouping set throws NPE #2 +SELECT a, b, c, count(d) FROM grouping GROUP BY a, b, c GROUPING SETS ((a)); + +-- SPARK-17849: grouping set throws NPE #3 +SELECT a, b, c, count(d) FROM grouping GROUP BY a, b, c GROUPING SETS ((c)); + +-- Group sets without explicit group by +SELECT c1, sum(c2) FROM (VALUES ('x', 10, 0), ('y', 20, 0)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1); + +-- Group sets without group by and with grouping +SELECT c1, sum(c2), grouping(c1) FROM (VALUES ('x', 10, 0), ('y', 20, 0)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1); + +-- Mutiple grouping within a grouping set +SELECT c1, c2, Sum(c3), grouping__id +FROM (VALUES ('x', 'a', 10), ('y', 'b', 20) ) AS t (c1, c2, c3) +GROUP BY GROUPING SETS ( ( c1 ), ( c2 ) ) +HAVING GROUPING__ID > 1; + +-- Group sets without explicit group by +SELECT grouping(c1) FROM (VALUES ('x', 'a', 10), ('y', 'b', 20)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1,c2); + +-- Mutiple grouping within a grouping set +SELECT -c1 AS c1 FROM (values (1,2), (3,2)) t(c1, c2) GROUP BY GROUPING SETS ((c1), (c1, c2)); + +-- complex expression in grouping sets +SELECT a + b, b, sum(c) FROM (VALUES (1,1,1),(2,2,2)) AS t(a,b,c) GROUP BY GROUPING SETS ( (a + b), (b)); + +-- complex expression in grouping sets +SELECT a + b, b, sum(c) FROM (VALUES (1,1,1),(2,2,2)) AS t(a,b,c) GROUP BY GROUPING SETS ( (a + b), (b + a), (b)); + +-- more query constructs with grouping sets +SELECT c1 AS col1, c2 AS col2 +FROM (VALUES (1, 2), (3, 2)) t(c1, c2) +GROUP BY GROUPING SETS ( ( c1 ), ( c1, c2 ) ) +HAVING col2 IS NOT NULL +ORDER BY -col1; + +-- negative tests - must have at least one grouping expression +SELECT a, b, c, count(d) FROM grouping GROUP BY WITH ROLLUP; + +SELECT a, b, c, count(d) FROM grouping GROUP BY WITH CUBE; + +SELECT c1 FROM (values (1,2), (3,2)) t(c1, c2) GROUP BY GROUPING SETS (()); + +-- duplicate entries in grouping sets +SELECT k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)); + +SELECT grouping__id, k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)); + +SELECT grouping(k1), k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)); + +-- grouping_id function +SELECT grouping_id(k1, k2), avg(v) from (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY k1, k2 GROUPING SETS ((k2, k1), k1); + +SELECT CASE WHEN a IS NULL THEN count(b) WHEN b IS NULL THEN count(c) END +FROM grouping +GROUP BY GROUPING SETS (a, b, c); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/having.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/having.sql new file mode 100644 index 000000000000..4c25a60c8abb --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/having.sql @@ -0,0 +1,41 @@ +create temporary view hav as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", 5) + as hav(k, v); + +-- having clause +SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2; + +-- having condition contains grouping column +SELECT count(k) FROM hav GROUP BY v + 1 HAVING v + 1 = 2; + +-- invalid having condition contains grouping column +SELECT count(k) FROM hav GROUP BY v HAVING v = array(1); + +-- SPARK-11032: resolve having correctly +SELECT MIN(t.v) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(COUNT(1) > 0); + +-- SPARK-20329: make sure we handle timezones correctly +SELECT a + b FROM VALUES (1L, 2), (3L, 4) AS T(a, b) GROUP BY a + b HAVING a + b > 1; + +-- SPARK-31519: Cast in having aggregate expressions returns the wrong result +SELECT SUM(a) AS b, CAST('2020-01-01' AS DATE) AS fake FROM VALUES (1, 10), (2, 20) AS T(a, b) GROUP BY b HAVING b > 10; + +-- SPARK-31663: Grouping sets with having clause returns the wrong result +SELECT SUM(a) AS b FROM VALUES (1, 10), (2, 20) AS T(a, b) GROUP BY GROUPING SETS ((b), (a, b)) HAVING b > 10; +SELECT SUM(a) AS b FROM VALUES (1, 10), (2, 20) AS T(a, b) GROUP BY CUBE(a, b) HAVING b > 10; +SELECT SUM(a) AS b FROM VALUES (1, 10), (2, 20) AS T(a, b) GROUP BY ROLLUP(a, b) HAVING b > 10; + +-- SPARK-33131: Grouping sets with having clause can not resolve qualified col name. +SELECT c1 FROM VALUES (1, 2) as t(c1, c2) GROUP BY GROUPING SETS(t.c1) HAVING t.c1 = 1; +SELECT c1 FROM VALUES (1, 2) as t(c1, c2) GROUP BY CUBE(t.c1) HAVING t.c1 = 1; +SELECT c1 FROM VALUES (1, 2) as t(c1, c2) GROUP BY ROLLUP(t.c1) HAVING t.c1 = 1; +SELECT c1 FROM VALUES (1, 2) as t(c1, c2) GROUP BY t.c1 HAVING t.c1 = 1; + +-- SPARK-28386: Resolve ORDER BY agg function with HAVING clause, while the agg function presents on SELECT list +SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2 ORDER BY sum(v); + +-- SPARK-28386: Resolve ORDER BY agg function with HAVING clause, while the agg function does not present on SELECT list +SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2 ORDER BY avg(v); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/higher-order-functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/higher-order-functions.sql new file mode 100644 index 000000000000..37081de012e9 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/higher-order-functions.sql @@ -0,0 +1,105 @@ +-- Test higher order functions with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +create or replace temporary view nested as values + (1, array(32, 97), array(array(12, 99), array(123, 42), array(1))), + (2, array(77, -76), array(array(6, 96, 65), array(-1, -2))), + (3, array(12), array(array(17))) + as t(x, ys, zs); + +-- Only allow lambda's in higher order functions. +select upper(x -> x) as v; +-- Also test functions registered with `ExpressionBuilder`. +select ceil(x -> x) as v; + +-- Identity transform an array +select transform(zs, z -> z) as v from nested; + +-- Transform an array +select transform(ys, y -> y * y) as v from nested; + +-- Transform an array with index +select transform(ys, (y, i) -> y + i) as v from nested; + +-- Transform an array with reference +select transform(zs, z -> concat(ys, z)) as v from nested; + +-- Transform an array to an array of 0's +select transform(ys, 0) as v from nested; + +-- Transform a null array +select transform(cast(null as array), x -> x + 1) as v; + +-- Filter. +select filter(ys, y -> y > 30) as v from nested; + +-- Filter a null array +select filter(cast(null as array), y -> true) as v; + +-- Filter nested arrays +select transform(zs, z -> filter(z, zz -> zz > 50)) as v from nested; + +-- Aggregate. +select aggregate(ys, 0, (y, a) -> y + a + x) as v from nested; + +-- Aggregate average. +select aggregate(ys, (0 as sum, 0 as n), (acc, x) -> (acc.sum + x, acc.n + 1), acc -> acc.sum / acc.n) as v from nested; + +-- Aggregate nested arrays +select transform(zs, z -> aggregate(z, 1, (acc, val) -> acc * val * size(z))) as v from nested; + +-- Aggregate a null array +select aggregate(cast(null as array), 0, (a, y) -> a + y + 1, a -> a + 2) as v; + +-- alias for Aggregate. +select reduce(ys, 0, (y, a) -> y + a + x) as v from nested; +select reduce(ys, (0 as sum, 0 as n), (acc, x) -> (acc.sum + x, acc.n + 1), acc -> acc.sum / acc.n) as v from nested; +select transform(zs, z -> reduce(z, 1, (acc, val) -> acc * val * size(z))) as v from nested; +select reduce(cast(null as array), 0, (a, y) -> a + y + 1, a -> a + 2) as v; + +-- Check for element existence +select exists(ys, y -> y > 30) as v from nested; + +-- Check for element existence in a null array +select exists(cast(null as array), y -> y > 30) as v; + +-- Zip with array +select zip_with(ys, zs, (a, b) -> a + size(b)) as v from nested; + +-- Zip with array with concat +select zip_with(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) -> concat(x, y)) as v; + +-- Zip with array coalesce +select zip_with(array('a'), array('d', null, 'f'), (x, y) -> coalesce(x, y)) as v; + +create or replace temporary view nested as values + (1, map(1, 1, 2, 2, 3, 3)), + (2, map(4, 4, 5, 5, 6, 6)) + as t(x, ys); + +-- Identity Transform Keys in a map +select transform_keys(ys, (k, v) -> k) as v from nested; + +-- Transform Keys in a map by adding constant +select transform_keys(ys, (k, v) -> k + 1) as v from nested; + +-- Transform Keys in a map using values +select transform_keys(ys, (k, v) -> k + v) as v from nested; + +-- Identity Transform values in a map +select transform_values(ys, (k, v) -> v) as v from nested; + +-- Transform values in a map by adding constant +select transform_values(ys, (k, v) -> v + 1) as v from nested; + +-- Transform values in a map using values +select transform_values(ys, (k, v) -> k + v) as v from nested; + +-- use non reversed keywords: all is non reversed only if !ansi +select transform(ys, all -> all * all) as v from values (array(32, 97)) as t(ys); +select transform(ys, (all, i) -> all + i) as v from values (array(32, 97)) as t(ys); + +-- SPARK-32819: Aggregate on nested string arrays +select aggregate(split('abcdefgh',''), array(array('')), (acc, x) -> array(array(x))); \ No newline at end of file diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/hll.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/hll.sql new file mode 100644 index 000000000000..a0c29cb25a5b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/hll.sql @@ -0,0 +1,76 @@ +-- Positive test cases +-- Create a table with some testing data. +DROP TABLE IF EXISTS t1; +CREATE TABLE t1 USING JSON AS VALUES (0), (1), (2), (2), (2), (3), (4) as tab(col); + +SELECT hll_sketch_estimate(hll_sketch_agg(col)) AS result FROM t1; + +SELECT hll_sketch_estimate(hll_sketch_agg(col, 12)) +FROM VALUES (50), (60), (60), (60), (75), (100) tab(col); + +SELECT hll_sketch_estimate(hll_sketch_agg(col)) +FROM VALUES ('abc'), ('def'), ('abc'), ('ghi'), ('abc') tab(col); + +SELECT hll_sketch_estimate( + hll_union( + hll_sketch_agg(col1), + hll_sketch_agg(col2))) + FROM VALUES + (1, 4), + (1, 4), + (2, 5), + (2, 5), + (3, 6) AS tab(col1, col2); + +SELECT hll_sketch_estimate(hll_union_agg(sketch, true)) + FROM (SELECT hll_sketch_agg(col) as sketch + FROM VALUES (1) AS tab(col) + UNION ALL + SELECT hll_sketch_agg(col, 20) as sketch + FROM VALUES (1) AS tab(col)); + +-- Negative test cases +SELECT hll_sketch_agg(col) +FROM VALUES (ARRAY(1, 2)), (ARRAY(3, 4)) tab(col); + +SELECT hll_sketch_agg(col, 2) +FROM VALUES (50), (60), (60) tab(col); + +SELECT hll_sketch_agg(col, 40) +FROM VALUES (50), (60), (60) tab(col); + +SELECT hll_union( + hll_sketch_agg(col1, 12), + hll_sketch_agg(col2, 13)) + FROM VALUES + (1, 4), + (1, 4), + (2, 5), + (2, 5), + (3, 6) AS tab(col1, col2); + +SELECT hll_union_agg(sketch, false) +FROM (SELECT hll_sketch_agg(col, 12) as sketch + FROM VALUES (1) AS tab(col) + UNION ALL + SELECT hll_sketch_agg(col, 20) as sketch + FROM VALUES (1) AS tab(col)); + +SELECT hll_union(1, 2) + FROM VALUES + (1, 4), + (1, 4), + (2, 5), + (2, 5), + (3, 6) AS tab(col1, col2); + +-- The HLL functions receive invalid buffers as inputs. +SELECT hll_sketch_estimate(CAST ('abc' AS BINARY)); + +SELECT hll_union(CAST ('abc' AS BINARY), CAST ('def' AS BINARY)); + +SELECT hll_union_agg(buffer, false) +FROM (SELECT CAST('abc' AS BINARY) AS buffer); + +-- Clean up +DROP TABLE IF EXISTS t1; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/identifier-clause.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/identifier-clause.sql new file mode 100644 index 000000000000..4aa8019097fd --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/identifier-clause.sql @@ -0,0 +1,177 @@ +------------------------------------------------------------------------------------------------- +-- Ths testfile tests the ability to templatize identifiers such as table and column names in SQL +-- Common patterns are to to use variable substitution or parameter markers (test in another file) +-------------------------------------------------------------------------------------------------- + +-- Sanity test variable substitution +SET hivevar:colname = 'c'; +SELECT IDENTIFIER(${colname} || '_1') FROM VALUES(1) AS T(c_1); + +-- Column references +SELECT IDENTIFIER('c1') FROM VALUES(1) AS T(c1); +SELECT IDENTIFIER('t.c1') FROM VALUES(1) AS T(c1); +SELECT IDENTIFIER('`t`.c1') FROM VALUES(1) AS T(c1); +SELECT IDENTIFIER('`c 1`') FROM VALUES(1) AS T(`c 1`); +SELECT IDENTIFIER('``') FROM VALUES(1) AS T(``); +SELECT IDENTIFIER('c' || '1') FROM VALUES(1) AS T(c1); + +-- Table references +CREATE SCHEMA IF NOT EXISTS s; +CREATE TABLE s.tab(c1 INT) USING CSV; +USE SCHEMA s; + +INSERT INTO IDENTIFIER('ta' || 'b') VALUES(1); +DELETE FROM IDENTIFIER('ta' || 'b') WHERE 1=0; +UPDATE IDENTIFIER('ta' || 'b') SET c1 = 2; +MERGE INTO IDENTIFIER('ta' || 'b') AS t USING IDENTIFIER('ta' || 'b') AS s ON s.c1 = t.c1 + WHEN MATCHED THEN UPDATE SET c1 = 3; +SELECT * FROM IDENTIFIER('tab'); +SELECT * FROM IDENTIFIER('s.tab'); +SELECT * FROM IDENTIFIER('`s`.`tab`'); +SELECT * FROM IDENTIFIER('t' || 'a' || 'b'); + +USE SCHEMA default; +DROP TABLE s.tab; +DROP SCHEMA s; + +-- Function reference +SELECT IDENTIFIER('COAL' || 'ESCE')(NULL, 1); +SELECT IDENTIFIER('abs')(c1) FROM VALUES(-1) AS T(c1); +SELECT * FROM IDENTIFIER('ra' || 'nge')(0, 1); + +-- Table DDL +CREATE TABLE IDENTIFIER('tab')(c1 INT) USING CSV; +DROP TABLE IF EXISTS IDENTIFIER('ta' || 'b'); + +CREATE SCHEMA identifier_clauses; +USE identifier_clauses; +CREATE TABLE IDENTIFIER('ta' || 'b')(c1 INT) USING CSV; +DROP TABLE IF EXISTS IDENTIFIER('identifier_clauses.' || 'tab'); +CREATE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV; +REPLACE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV; +CACHE TABLE IDENTIFIER('ta' || 'b'); +UNCACHE TABLE IDENTIFIER('ta' || 'b'); +DROP TABLE IF EXISTS IDENTIFIER('ta' || 'b'); +USE default; +DROP SCHEMA identifier_clauses; + +CREATE TABLE tab(c1 INT) USING CSV; +INSERT INTO tab VALUES (1); +SELECT c1 FROM tab; +DESCRIBE IDENTIFIER('ta' || 'b'); +ANALYZE TABLE IDENTIFIER('ta' || 'b') COMPUTE STATISTICS; +ALTER TABLE IDENTIFIER('ta' || 'b') ADD COLUMN c2 INT; +SHOW TBLPROPERTIES IDENTIFIER('ta' || 'b'); +SHOW COLUMNS FROM IDENTIFIER('ta' || 'b'); +COMMENT ON TABLE IDENTIFIER('ta' || 'b') IS 'hello'; +REFRESH TABLE IDENTIFIER('ta' || 'b'); +REPAIR TABLE IDENTIFIER('ta' || 'b'); +TRUNCATE TABLE IDENTIFIER('ta' || 'b'); +DROP TABLE IF EXISTS tab; + +-- View +CREATE OR REPLACE VIEW IDENTIFIER('v')(c1) AS VALUES(1); +SELECT * FROM v; +ALTER VIEW IDENTIFIER('v') AS VALUES(2); +DROP VIEW IDENTIFIER('v'); +CREATE TEMPORARY VIEW IDENTIFIER('v')(c1) AS VALUES(1); +DROP VIEW IDENTIFIER('v'); + +-- Schema +CREATE SCHEMA IDENTIFIER('id' || 'ent'); +ALTER SCHEMA IDENTIFIER('id' || 'ent') SET PROPERTIES (somekey = 'somevalue'); +ALTER SCHEMA IDENTIFIER('id' || 'ent') SET LOCATION 'someloc'; +COMMENT ON SCHEMA IDENTIFIER('id' || 'ent') IS 'some comment'; +DESCRIBE SCHEMA IDENTIFIER('id' || 'ent'); +SHOW TABLES IN IDENTIFIER('id' || 'ent'); +SHOW TABLE EXTENDED IN IDENTIFIER('id' || 'ent') LIKE 'hello'; +USE IDENTIFIER('id' || 'ent'); +SHOW CURRENT SCHEMA; +USE SCHEMA IDENTIFIER('id' || 'ent'); +USE SCHEMA default; +DROP SCHEMA IDENTIFIER('id' || 'ent'); + +-- Function +CREATE SCHEMA ident; +CREATE FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg'; +DESCRIBE FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg'); +REFRESH FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg'); +DROP FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg'); +DROP SCHEMA ident; +CREATE TEMPORARY FUNCTION IDENTIFIER('my' || 'DoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg'; +DROP TEMPORARY FUNCTION IDENTIFIER('my' || 'DoubleAvg'); + +-- IDENTIFIER + variable +DECLARE var = 'sometable'; +CREATE TABLE IDENTIFIER(var)(c1 INT) USING CSV; + +SET VAR var = 'c1'; +SELECT IDENTIFIER(var) FROM VALUES(1) AS T(c1); + +SET VAR var = 'some'; +DROP TABLE IDENTIFIER(var || 'table'); + +-- Error conditions +SELECT IDENTIFIER('c 1') FROM VALUES(1) AS T(`c 1`); +SELECT IDENTIFIER('') FROM VALUES(1) AS T(``); +VALUES(IDENTIFIER(CAST(NULL AS STRING))); +VALUES(IDENTIFIER(1)); +VALUES(IDENTIFIER(SUBSTR('HELLO', 1, RAND() + 1))); +SELECT `IDENTIFIER`('abs')(c1) FROM VALUES(-1) AS T(c1); + +CREATE TABLE IDENTIFIER(1)(c1 INT) USING csv; +CREATE TABLE IDENTIFIER('a.b.c')(c1 INT) USING csv; +CREATE VIEW IDENTIFIER('a.b.c')(c1) AS VALUES(1); +DROP TABLE IDENTIFIER('a.b.c'); +DROP VIEW IDENTIFIER('a.b.c'); +COMMENT ON TABLE IDENTIFIER('a.b.c.d') IS 'hello'; +VALUES(IDENTIFIER(1)()); +VALUES(IDENTIFIER('a.b.c.d')()); + +CREATE TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg'; +DROP TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg'); +CREATE TEMPORARY VIEW IDENTIFIER('default.v')(c1) AS VALUES(1); + +-- SPARK-48273: Aggregation operation in statements using identifier clause for table name +create temporary view identifier('v1') as (select my_col from (values (1), (2), (1) as (my_col)) group by 1); +cache table identifier('t1') as (select my_col from (values (1), (2), (1) as (my_col)) group by 1); +create table identifier('t2') using csv as (select my_col from (values (1), (2), (1) as (my_col)) group by 1); +insert into identifier('t2') select my_col from (values (3) as (my_col)) group by 1; +drop view v1; +drop table t1; +drop table t2; + +-- SPARK-46625: CTE reference with identifier clause and session variables +DECLARE agg = 'max'; +DECLARE col = 'c1'; +DECLARE tab = 'T'; + +WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), + T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) +SELECT IDENTIFIER(agg)(IDENTIFIER(col)) FROM IDENTIFIER(tab); + +WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), + T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) +SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('T'); + +WITH ABC(c1, c2) AS (VALUES(1, 2), (2, 3)) +SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('A' || 'BC'); + +-- Not supported +SELECT row_number() OVER IDENTIFIER('x.win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1); +SELECT T1.c1 FROM VALUES(1) AS T1(c1) JOIN VALUES(1) AS T2(c1) USING (IDENTIFIER('c1')); +SELECT IDENTIFIER('t').c1 FROM VALUES(1) AS T(c1); +SELECT map('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1); +SELECT named_struct('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1); +SELECT * FROM s.IDENTIFIER('tab'); +SELECT * FROM IDENTIFIER('s').IDENTIFIER('tab'); +SELECT * FROM IDENTIFIER('s').tab; +SELECT row_number() OVER IDENTIFIER('win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1); +SELECT row_number() OVER win FROM VALUES(1) AS T(c1) WINDOW IDENTIFIER('win') AS (ORDER BY c1); +WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v); +INSERT INTO tab(IDENTIFIER('c1')) VALUES(1); +CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1); +CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV; + + + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/ignored.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/ignored.sql new file mode 100644 index 000000000000..0b5331a1d022 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/ignored.sql @@ -0,0 +1,4 @@ +-- This is a query file that has been ignored. +-- It includes a query that should crash Spark. +-- If the test case is run, the whole suite would fail. +some random not working query that should crash Spark. diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/ilike-all.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/ilike-all.sql new file mode 100644 index 000000000000..747cf1c3acab --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/ilike-all.sql @@ -0,0 +1,41 @@ +-- test cases for ilike all + +CREATE OR REPLACE TEMPORARY VIEW ilike_all_table AS SELECT * FROM (VALUES + ('gOOgle', '%oo%'), + ('facebook', '%OO%'), + ('liNkedin', '%In')) + as t1(company, pat); + +SELECT company FROM ilike_all_table WHERE company ILIKE ALL ('%oO%', '%Go%'); + +SELECT company FROM ilike_all_table WHERE company ILIKE ALL ('microsoft', '%yoo%'); + +SELECT + company, + CASE + WHEN company ILIKE ALL ('%oo%', '%GO%') THEN 'Y' + ELSE 'N' + END AS is_available, + CASE + WHEN company ILIKE ALL ('%OO%', 'go%') OR company ILIKE ALL ('%IN', 'ms%') THEN 'Y' + ELSE 'N' + END AS mix +FROM ilike_all_table ; + +-- Mix test with constant pattern and column value +SELECT company FROM ilike_all_table WHERE company ILIKE ALL ('%oo%', pat); + +-- not ilike all test +SELECT company FROM ilike_all_table WHERE company NOT ILIKE ALL ('%oo%', '%In', 'Fa%'); +SELECT company FROM ilike_all_table WHERE company NOT ILIKE ALL ('microsoft', '%yoo%'); +SELECT company FROM ilike_all_table WHERE company NOT ILIKE ALL ('%oo%', 'fA%'); +SELECT company FROM ilike_all_table WHERE NOT company ILIKE ALL ('%oO%', 'fa%'); + +-- null test +SELECT company FROM ilike_all_table WHERE company ILIKE ALL ('%OO%', NULL); +SELECT company FROM ilike_all_table WHERE company NOT ILIKE ALL ('%Oo%', NULL); +SELECT company FROM ilike_all_table WHERE company ILIKE ALL (NULL, NULL); +SELECT company FROM ilike_all_table WHERE company NOT ILIKE ALL (NULL, NULL); + +-- negative case +SELECT company FROM ilike_any_table WHERE company ILIKE ALL (); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/ilike-any.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/ilike-any.sql new file mode 100644 index 000000000000..615b9fcc70bf --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/ilike-any.sql @@ -0,0 +1,41 @@ +-- test cases for ilike any + +CREATE OR REPLACE TEMPORARY VIEW ilike_any_table AS SELECT * FROM (VALUES + ('Google', '%Oo%'), + ('FaceBook', '%oO%'), + ('linkedIn', '%IN')) + as t1(company, pat); + +SELECT company FROM ilike_any_table WHERE company ILIKE ANY ('%oo%', '%IN', 'fA%'); + +SELECT company FROM ilike_any_table WHERE company ILIKE ANY ('microsoft', '%yoo%'); + +select + company, + CASE + WHEN company ILIKE ANY ('%oO%', '%IN', 'Fa%') THEN 'Y' + ELSE 'N' + END AS is_available, + CASE + WHEN company ILIKE ANY ('%OO%', 'fa%') OR company ILIKE ANY ('%in', 'MS%') THEN 'Y' + ELSE 'N' + END AS mix +FROM ilike_any_table; + +-- Mix test with constant pattern and column value +SELECT company FROM ilike_any_table WHERE company ILIKE ANY ('%zZ%', pat); + +-- not ilike any test +SELECT company FROM ilike_any_table WHERE company NOT ILIKE ANY ('%oO%', '%iN', 'fa%'); +SELECT company FROM ilike_any_table WHERE company NOT ILIKE ANY ('microsoft', '%yOo%'); +SELECT company FROM ilike_any_table WHERE company NOT ILIKE ANY ('%oo%', 'Fa%'); +SELECT company FROM ilike_any_table WHERE NOT company ILIKE ANY ('%OO%', 'fa%'); + +-- null test +SELECT company FROM ilike_any_table WHERE company ILIKE ANY ('%oO%', NULL); +SELECT company FROM ilike_any_table WHERE company NOT ILIKE ANY ('%oo%', NULL); +SELECT company FROM ilike_any_table WHERE company ILIKE ANY (NULL, NULL); +SELECT company FROM ilike_any_table WHERE company NOT ILIKE ANY (NULL, NULL); + +-- negative case +SELECT company FROM ilike_any_table WHERE company ILIKE ANY (); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/ilike.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/ilike.sql new file mode 100644 index 000000000000..701685982df8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/ilike.sql @@ -0,0 +1,39 @@ +-- test cases for ilike + +-- null handling +select null ilike 'a'; +select 'a' ilike null; +select null ilike null; + +-- simple patterns +select 'a' ilike 'a'; +select 'a' ilike 'b'; +select 'A' ilike 'a'; +select 'a' ilike 'A'; +select 'abdef' ilike 'aBdef'; +select 'a_%b' ilike 'a\\__b'; +select 'addb' ilike 'A_%b'; +select 'abC' ilike 'a%'; +select 'a\nb' ilike 'a_B'; + +-- empty input +select '' ilike ''; +select 'A' ilike ''; +select '' ilike 'a'; + +-- double-escaping backslash +select ilike('\__', '\\\__'); +select ilike('\\\__', '%\\%\%'); + +-- unicode +select 'a\u20ACA' ilike '_\u20AC_'; +select 'A€a' ilike '_€_'; +select 'a€AA' ilike '_\u20AC_a'; +select 'a\u20ACaz' ilike '_€_Z'; +select 'ЀЁЂѺΏỀ' ilike 'ѐёђѻώề'; + +-- escape char +select 'Addb' ilike 'a%#%b' escape '#'; +select 'a_%b' ilike 'a%#%B' escape '#'; +select 'Addb' ilike 'A%$%b' escape '$'; +select 'a_%b' ilike 'a%+%B' escape '+'; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/inline-table.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/inline-table.sql new file mode 100644 index 000000000000..8f65dc77c960 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/inline-table.sql @@ -0,0 +1,68 @@ + +-- single row, without table and column alias +select * from values ("one", 1); + +-- single row, without column alias +select * from values ("one", 1) as data; + +-- single row +select * from values ("one", 1) as data(a, b); + +-- single column multiple rows +select * from values 1, 2, 3 as data(a); + +-- three rows +select * from values ("one", 1), ("two", 2), ("three", null) as data(a, b); + +-- null type +select * from values ("one", null), ("two", null) as data(a, b); + +-- int and long coercion +select * from values ("one", 1), ("two", 2L) as data(a, b); + +-- foldable expressions +select * from values ("one", 1 + 0), ("two", 1 + 3L) as data(a, b); + +-- expressions with alias +select * from values ("one", 1 as one) as data(a, b); + +-- literal functions +select a from values ("one", current_timestamp) as data(a, b); + +-- complex types +select * from values ("one", array(0, 1)), ("two", array(2, 3)) as data(a, b); + +-- decimal and double coercion +select * from values ("one", 2.0), ("two", 3.0D) as data(a, b); + +-- error reporting: nondeterministic function rand +select * from values ("one", rand(5)), ("two", 3.0D) as data(a, b); + +-- error reporting: different number of columns +select * from values ("one", 2.0), ("two") as data(a, b); + +-- error reporting: types that are incompatible +select * from values ("one", array(0, 1)), ("two", struct(1, 2)) as data(a, b); + +-- error reporting: number aliases different from number data values +select * from values ("one"), ("two") as data(a, b); + +-- error reporting: unresolved expression +select * from values ("one", random_not_exist_func(1)), ("two", 2) as data(a, b); + +-- error reporting: aggregate expression +select * from values ("one", count(1)), ("two", 2) as data(a, b); + +-- string to timestamp +select * from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991-12-06 01:00:00.0'), timestamp('1991-12-06 12:00:00.0'))) as data(a, b); + +-- ReplaceExpressions as row +select * from values (try_add(5, 0)); +select * from values (try_divide(5, 0)); +select * from values (10 + try_divide(5, 0)); + +-- now() should be kept as tempResolved inline expression. +select count(distinct ct) from values now(), now(), now() as data(ct); + +-- current_timestamp() should be kept as tempResolved inline expression. +select count(distinct ct) from values current_timestamp(), current_timestamp() as data(ct); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/inner-join.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/inner-join.sql new file mode 100644 index 000000000000..6a761ad5a74e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/inner-join.sql @@ -0,0 +1,29 @@ +-- There are 2 dimensions we want to test +-- 1. run with broadcast hash join, sort merge join or shuffle hash join. +-- 2. run with whole-stage-codegen, operator codegen or no codegen. + +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=10485760 +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.forceApplyShuffledHashJoin=true + +--CONFIG_DIM2 spark.sql.codegen.wholeStage=true +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a); +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a); +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a); +CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a); + +CREATE TEMPORARY VIEW ta AS +SELECT a, 'a' AS tag FROM t1 +UNION ALL +SELECT a, 'b' AS tag FROM t2; + +CREATE TEMPORARY VIEW tb AS +SELECT a, 'a' AS tag FROM t3 +UNION ALL +SELECT a, 'b' AS tag FROM t4; + +-- SPARK-19766 Constant alias columns in INNER JOIN should not be folded by FoldablePropagation rule +SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/intersect-all.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/intersect-all.sql new file mode 100644 index 000000000000..077caa5dd44a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/intersect-all.sql @@ -0,0 +1,175 @@ +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (1, 3), + (2, 3), + (null, null), + (null, null) + AS tab1(k, v); +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (2, 3), + (3, 4), + (null, null), + (null, null) + AS tab2(k, v); + +-- Basic INTERSECT ALL +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab2; + +-- INTERSECT ALL same table in both branches +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab1 WHERE k = 1; + +-- Empty left relation +SELECT * FROM tab1 WHERE k > 2 +INTERSECT ALL +SELECT * FROM tab2; + +-- Empty right relation +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab2 WHERE k > 3; + +-- Type Coerced INTERSECT ALL +SELECT * FROM tab1 +INTERSECT ALL +SELECT CAST(1 AS BIGINT), CAST(2 AS BIGINT); + +-- Error as types of two side are not compatible +SELECT * FROM tab1 +INTERSECT ALL +SELECT array(1), 2; + +-- Mismatch on number of columns across both branches +SELECT k FROM tab1 +INTERSECT ALL +SELECT k, v FROM tab2; + +-- Basic +SELECT * FROM tab2 +INTERSECT ALL +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab2; + +-- Chain of different `set operations +SELECT * FROM tab1 +EXCEPT +SELECT * FROM tab2 +UNION ALL +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab2 +; + +-- Chain of different `set operations +SELECT * FROM tab1 +EXCEPT +SELECT * FROM tab2 +EXCEPT +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab2 +; + +-- test use parenthesis to control order of evaluation +( + ( + ( + SELECT * FROM tab1 + EXCEPT + SELECT * FROM tab2 + ) + EXCEPT + SELECT * FROM tab1 + ) + INTERSECT ALL + SELECT * FROM tab2 +) +; + +-- Join under intersect all +SELECT * +FROM (SELECT tab1.k, + tab2.v + FROM tab1 + JOIN tab2 + ON tab1.k = tab2.k) +INTERSECT ALL +SELECT * +FROM (SELECT tab1.k, + tab2.v + FROM tab1 + JOIN tab2 + ON tab1.k = tab2.k); + +-- Join under intersect all (2) +SELECT * +FROM (SELECT tab1.k, + tab2.v + FROM tab1 + JOIN tab2 + ON tab1.k = tab2.k) +INTERSECT ALL +SELECT * +FROM (SELECT tab2.v AS k, + tab1.k AS v + FROM tab1 + JOIN tab2 + ON tab1.k = tab2.k); + +-- Group by under intersect all +SELECT v FROM tab1 GROUP BY v +INTERSECT ALL +SELECT k FROM tab2 GROUP BY k; + +-- Test pre spark2.4 behaviour of set operation precedence +-- All the set operators are given equal precedence and are evaluated +-- from left to right as they appear in the query. + +-- Set the property +SET spark.sql.legacy.setopsPrecedence.enabled= true; + +SELECT * FROM tab1 +EXCEPT +SELECT * FROM tab2 +UNION ALL +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab2; + +SELECT * FROM tab1 +EXCEPT +SELECT * FROM tab2 +UNION ALL +SELECT * FROM tab1 +INTERSECT +SELECT * FROM tab2; + +-- Restore the property +SET spark.sql.legacy.setopsPrecedence.enabled = false; + +-- SPARK-32638: corrects references when adding aliases in WidenSetOperationTypes +CREATE OR REPLACE TEMPORARY VIEW tab3 AS VALUES (decimal(1)), (decimal(2)) tbl3(v); +SELECT t.v FROM ( + SELECT v FROM tab3 + INTERSECT + SELECT v + v AS v FROM tab3 +) t; + +SELECT SUM(t.v) FROM ( + SELECT v FROM tab3 + INTERSECT + SELECT v + v AS v FROM tab3 +) t; + +-- Clean-up +DROP VIEW IF EXISTS tab1; +DROP VIEW IF EXISTS tab2; +DROP VIEW IF EXISTS tab3; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/interval.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/interval.sql new file mode 100644 index 000000000000..e4da28c2e758 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/interval.sql @@ -0,0 +1,387 @@ +-- test for intervals + +-- multiply and divide an interval by a number +select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15'); +select interval 4 month 2 weeks 3 microseconds * 1.5; +select interval 2 years 4 months; +select interval 2 weeks 3 microseconds * 1.5; +select (timestamp'2019-10-15' - timestamp'2019-10-14') / 1.5; +select interval 2147483647 month * 2; +select interval 2147483647 month / 0.5; +select interval 2147483647 day * 2; +select interval 2147483647 day / 0.5; + +-- multiply and divide an interval by a string literal +select interval 2 second * '2'; +select interval 2 second / '2'; +select interval 2 year * '2'; +select interval 2 year / '2'; +select interval 2 second * 'a'; +select interval 2 second / 'a'; +select interval 2 year * 'a'; +select interval 2 year / 'a'; + +select '2' * interval 2 second; +select '2' * interval 2 year; +select 'a' * interval 2 second; +select 'a' * interval 2 year; + +-- invalid: string literal / interval +select '2' / interval 2 second; +select '2' / interval 2 year; + +-- interval operation with null and zero case +select interval '2 seconds' / 0; +select interval '2 seconds' / null; +select interval '2 seconds' * null; +select null * interval '2 seconds'; +select interval '2' year / 0; +select interval '2' year / null; +select interval '2' year * null; +select null * interval '2' year; + +-- invalid: divide by interval +select 2 / interval '2' year; +select 2 / interval '2' hour; +select null / interval '2' year; +select null / interval '2' hour; + +-- interval with a positive/negative sign +select -interval '-1 month 1 day -1 second'; +select -interval '-1 year 1 month'; +select -interval '-1 day 1 hour -1 minute 1 second'; +select -interval -1 month 1 day -1 second; +select -interval -1 year 1 month; +select -interval -1 day 1 hour -1 minute 1 second; +select +interval '-1 month 1 day -1 second'; +select +interval '-1 year 1 month'; +select +interval '-1 day 1 hour -1 minute 1 second'; +select +interval -1 month 1 day -1 second; +select +interval -1 year 1 month; +select +interval -1 day 1 hour -1 minute 1 second; +select interval -'1-1' year to month; +select interval -'-1-1' year to month; +select interval +'-1-1' year to month; +select interval - '1 2:3:4.001' day to second; +select interval +'1 2:3:4.001' day to second; +select interval -'-1 2:3:4.001' day to second; +select interval -'1' year; +select interval -'-1' year; +select interval -'11' month; +select interval -'-11' month; +select interval -'1' day; +select interval -'-1' day; +select interval -'23' hour; +select interval -'-23' hour; +select interval -'59' minute; +select interval -'-59' minute; +select interval -'59' second; +select interval -'-59' second; + +-- make intervals +select make_interval(1); +select make_interval(1, 2); +select make_interval(1, 2, 3); +select make_interval(1, 2, 3, 4); +select make_interval(1, 2, 3, 4, 5); +select make_interval(1, 2, 3, 4, 5, 6); +select make_interval(1, 2, 3, 4, 5, 6, 7.008009); +select make_interval(1, 2, 3, 4, 0, 0, 123456789012.123456); +select make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789); + +-- make_dt_interval +select make_dt_interval(1); +select make_dt_interval(1, 2); +select make_dt_interval(1, 2, 3); +select make_dt_interval(1, 2, 3, 4.005006); +select make_dt_interval(1, 0, 0, 123456789012.123456); +select make_dt_interval(2147483647); + +-- make_ym_interval +select make_ym_interval(1); +select make_ym_interval(1, 2); +select make_ym_interval(0, 1); +select make_ym_interval(178956970, 7); +select make_ym_interval(178956970, 8); +select make_ym_interval(-178956970, -8); +select make_ym_interval(-178956970, -9); + +-- cast string to intervals +select cast('1 second' as interval); +select cast('+1 second' as interval); +select cast('-1 second' as interval); +select cast('+ 1 second' as interval); +select cast('- 1 second' as interval); +select cast('- -1 second' as interval); +select cast('- +1 second' as interval); + +-- interval literal +select interval 13.123456789 seconds, interval -13.123456789 second; +select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond; +select interval 1 year 2 month; +select interval 4 day 5 hour 6 minute 7 seconds; +select interval 3 week 8 millisecond 9 microsecond; +select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second; +select interval '30' year '25' month; +select interval '-100' day '40' hour '80' minute '299.889987299' second; +select interval '0-0' year to month; +select interval '0 0:0:0' day to second; +select interval '0 0:0:0.1' day to second; +select interval '10-9' year to month; +select interval '20 15' day to hour; +select interval '20 15:40' day to minute; +select interval '20 15:40:32.99899999' day to second; +select interval '15:40' hour to minute; +select interval '15:40:32.99899999' hour to second; +select interval '40:32.99899999' minute to second; +select interval '40:32' minute to second; +select interval 30 day day; +select interval 30 days days; + +-- invalid day-time string intervals +select interval '20 15:40:32.99899999' day to hour; +select interval '20 15:40:32.99899999' day to minute; +select interval '15:40:32.99899999' hour to minute; +select interval '15:40.99899999' hour to second; +select interval '15:40' hour to second; +select interval '20 40:32.99899999' minute to second; + +-- ns is not supported +select interval 10 nanoseconds; + +-- map + interval test +select map(1, interval 1 day, 2, interval 3 week); +select map(1, interval 1 day, 2, interval 2 day); +select map(1, interval 1 year, 2, interval 2 month); +select map(1, interval 1 month, 2, interval 2 month); +select map(1, interval 1 week, 2, interval 2 day); +select map(1, interval 2 millisecond, 3, interval 3 microsecond); + +-- typed interval expression +select interval 'interval 3 year 1 month'; +select interval '3 year 1 month'; +SELECT interval 'interval 2 weeks 2 days 1 hour 3 minutes 2 seconds 100 millisecond 200 microseconds'; +SELECT interval '2 weeks 2 days 1 hour 3 minutes 2 seconds 100 millisecond 200 microseconds'; + +-- malformed interval literal +select interval 1 fake_unit; +select interval 1 year to month; +select interval '1' year to second; +select interval '10-9' year to month '2-1' year to month; +select interval '10-9' year to month '12:11:10' hour to second; +select interval '1 15:11' day to minute '12:11:10' hour to second; +select interval 1 year '2-1' year to month; +select interval 1 year '12:11:10' hour to second; +select interval '10-9' year to month '1' year; +select interval '12:11:10' hour to second '1' year; +select interval (-30) day; +select interval (a + 1) day; +select interval 30 day day day; +select interval (-30) days; +select interval (a + 1) days; +select interval 30 days days days; +SELECT INTERVAL '178956970-7' YEAR TO MONTH; +SELECT INTERVAL '178956970-8' YEAR TO MONTH; +SELECT INTERVAL '-178956970-8' YEAR TO MONTH; +SELECT INTERVAL -'178956970-8' YEAR TO MONTH; + +-- interval +/- interval +select + interval '2-2' year to month + interval '3' month, + interval '2' year - interval '3-3' year to month, + interval '99 11:22:33.123456789' day to second + interval '10 9:8' day to minute, + interval '22:33.123456789' minute to second - interval '10' day; +-- if one side is string/null literal, promote it to interval type. +select + interval '2' year + '3-3 year to month', + interval '2' year - '3 month', + '3-2 year to month' + interval '2-2' year to month, + '3 year' - interval '2-2' year to month, + interval '99 11:22:33.123456789' day to second + '12:12 hour to second', + interval '99 11:22:33.123456789' day to second - '12 hour', + '4 day' + interval '10' day, + '4 22 day to hour' - interval '10' day; +select + interval '2' year + null, + interval '2' year - null, + interval '2' hour + null, + interval '2' hour - null, + null + interval '2' year, + null - interval '2' year, + null + interval '2' hour, + null - interval '2' hour; + +-- invalid: malformed interval string +select interval '2' year + '3-3'; +select interval '2' year - '4'; +select '4 11:11' - interval '4 22:12' day to minute; +select '4 12:12:12' + interval '4 22:12' day to minute; +-- invalid: non-literal string column +create temporary view interval_view as select '1' str; +select interval '2' year + str from interval_view; +select interval '2' year - str from interval_view; +select str - interval '4 22:12' day to minute from interval_view; +select str + interval '4 22:12' day to minute from interval_view; + +-- invalid: mixed year-month and day-time interval +select interval '2-2' year to month + interval '3' day; +select interval '3' day + interval '2-2' year to month; +select interval '2-2' year to month - interval '3' day; +select interval '3' day - interval '2-2' year to month; + +-- invalid: number +/- interval +select 1 - interval '2' second; +select 1 + interval '2' month; +select interval '2' second + 1; +select interval '2' month - 1; + +-- control characters as white spaces +select interval '\t interval 1 day'; +select interval 'interval \t 1\tday'; +select interval 'interval\t1\tday'; +select interval '1\t' day; +select interval '1 ' day; +select interval '2-2\t' year to month; +select interval '-\t2-2\t' year to month; +select interval '\n0 12:34:46.789\t' day to second; +select interval '\n-\t10\t 12:34:46.789\t' day to second; +select interval '中文 interval 1 day'; +select interval 'interval中文 1 day'; +select interval 'interval 1中文day'; + +-- interval overflow: if (ansi) exception else NULL +select -(a) from values (interval '-2147483648 months', interval '2147483647 months') t(a, b); +select a - b from values (interval '-2147483648 months', interval '2147483647 months') t(a, b); +select b + interval '1 month' from values (interval '-2147483648 months', interval '2147483647 months') t(a, b); +select a * 1.1 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b); +select a / 0.5 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b); + +-- interval support for csv and json functions +SELECT + from_csv('1, 1 day', 'a INT, b interval'), + from_csv('1, 1', 'a INT, b interval day'), + to_csv(from_csv('1, 1 day', 'a INT, b interval')), + to_csv(from_csv('1, 1', 'a INT, b interval day')), + to_csv(named_struct('a', interval 32 hour, 'b', interval 70 minute)), + from_csv(to_csv(named_struct('a', interval 32 hour, 'b', interval 70 minute)), 'a interval hour, b interval minute'); +SELECT + from_json('{"a":"1 days"}', 'a interval'), + from_csv('1, 1', 'a INT, b interval year'), + to_json(from_json('{"a":"1 days"}', 'a interval')), + to_csv(from_csv('1, 1', 'a INT, b interval year')), + to_csv(named_struct('a', interval 32 year, 'b', interval 10 month)), + from_csv(to_csv(named_struct('a', interval 32 year, 'b', interval 10 month)), 'a interval year, b interval month'); +SELECT + from_json('{"a":"1"}', 'a interval day'), + to_json(from_json('{"a":"1"}', 'a interval day')), + to_json(map('a', interval 100 day 130 minute)), + from_json(to_json(map('a', interval 100 day 130 minute)), 'a interval day to minute'); +SELECT + from_json('{"a":"1"}', 'a interval year'), + to_json(from_json('{"a":"1"}', 'a interval year')), + to_json(map('a', interval 32 year 10 month)), + from_json(to_json(map('a', interval 32 year 10 month)), 'a interval year to month'); + +select interval '+'; +select interval '+.'; +select interval '1'; +select interval '1.2'; +select interval '- 2'; +select interval '1 day -'; +select interval '1 day 1'; + +select interval '1 day 2' day; +select interval 'interval 1' day; +select interval '-\t 1' day; + +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / 2; +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / 5; +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1; +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1L; +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1.0; +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1.0D; + +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / 2; +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / 5; +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1; +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1L; +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1.0; +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1.0D; + +SELECT INTERVAL '106751991 04' DAY TO HOUR; +SELECT INTERVAL '106751991 04:00' DAY TO MINUTE; +SELECT INTERVAL '106751991 04:00:54.775807' DAY TO SECOND; +SELECT INTERVAL '2562047788:00' HOUR TO MINUTE; +SELECT INTERVAL '2562047788:00:54.775807' HOUR TO SECOND; +SELECT INTERVAL '153722867280:54.775807' MINUTE TO SECOND; +SELECT INTERVAL '-106751991 04' DAY TO HOUR; +SELECT INTERVAL '-106751991 04:00' DAY TO MINUTE; +SELECT INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND; +SELECT INTERVAL '-2562047788:00' HOUR TO MINUTE; +SELECT INTERVAL '-2562047788:00:54.775808' HOUR TO SECOND; +SELECT INTERVAL '-153722867280:54.775808' MINUTE TO SECOND; + +SELECT INTERVAL '106751992 04' DAY TO HOUR; +SELECT INTERVAL '-106751992 04' DAY TO HOUR; +SELECT INTERVAL '2562047789:00' HOUR TO MINUTE; +SELECT INTERVAL '-2562047789:00' HOUR TO MINUTE; +SELECT INTERVAL '153722867281:54.775808' MINUTE TO SECOND; +SELECT INTERVAL '-153722867281:54.775808' MINUTE TO SECOND; + +SELECT INTERVAL '178956970' YEAR; +SELECT INTERVAL '-178956970' YEAR; +SELECT INTERVAL '2147483647' MONTH; +SELECT INTERVAL '-2147483647' MONTH; + +SELECT INTERVAL '106751991' DAY; +SELECT INTERVAL '-106751991' DAY; +SELECT INTERVAL '2562047788' HOUR; +SELECT INTERVAL '-2562047788' HOUR; +SELECT INTERVAL '153722867280' MINUTE; +SELECT INTERVAL '-153722867280' MINUTE; +SELECT INTERVAL '54.775807' SECOND; +SELECT INTERVAL '-54.775807' SECOND; + +SELECT INTERVAL '1' DAY > INTERVAL '1' HOUR; +SELECT INTERVAL '1 02' DAY TO HOUR = INTERVAL '02:10:55' HOUR TO SECOND; +SELECT INTERVAL '1' YEAR < INTERVAL '1' MONTH; +SELECT INTERVAL '-1-1' YEAR TO MONTH = INTERVAL '-13' MONTH; +SELECT INTERVAL 1 MONTH > INTERVAL 20 DAYS; + +-- compare interval with string +SELECT INTERVAL '1' DAY < '1'; +SELECT INTERVAL '1' DAY = '1'; +SELECT INTERVAL '1' DAY > '1'; +SELECT '1' < INTERVAL '1' DAY; +SELECT '1' = INTERVAL '1' DAY; +SELECT '1' > INTERVAL '1' DAY; +SELECT INTERVAL '1' YEAR < '1'; +SELECT INTERVAL '1' YEAR = '1'; +SELECT INTERVAL '1' YEAR > '1'; +SELECT '1' < INTERVAL '1' YEAR; +SELECT '1' = INTERVAL '1' YEAR; +SELECT '1' > INTERVAL '1' YEAR; + +SELECT array(INTERVAL '1' YEAR, INTERVAL '1' MONTH); +SELECT array(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE); +SELECT array(INTERVAL 1 MONTH, INTERVAL 20 DAYS); +SELECT coalesce(INTERVAL '1' YEAR, INTERVAL '1' MONTH); +SELECT coalesce(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE); +SELECT coalesce(INTERVAL 1 MONTH, INTERVAL 20 DAYS); +SELECT abs(INTERVAL '-10' YEAR); +SELECT abs(INTERVAL -'1 02:03:04.123' DAY TO SECOND); +SELECT div(INTERVAL '1-1' YEAR TO MONTH, INTERVAL '1' YEAR); +SELECT div(INTERVAL '1-1' YEAR TO MONTH, INTERVAL '-1' MONTH); +SELECT div(INTERVAL '1 06' DAY TO HOUR, INTERVAL '1' DAY); +SELECT div(INTERVAL '1 06' DAY TO HOUR, INTERVAL '-1' HOUR); +SELECT div(INTERVAL '1' MONTH, INTERVAL '-1' DAY); +SELECT signum(INTERVAL '-10' YEAR); +SELECT signum(INTERVAL '10' MONTH); +SELECT signum(INTERVAL '0-0' YEAR TO MONTH); +SELECT signum(INTERVAL '-10' DAY); +SELECT signum(INTERVAL '10' HOUR); +SELECT signum(INTERVAL '0 0:0:0' DAY TO SECOND); +SELECT width_bucket(INTERVAL '0' YEAR, INTERVAL '0' YEAR, INTERVAL '10' YEAR, 10); +SELECT width_bucket(INTERVAL '-1' YEAR, INTERVAL -'1-2' YEAR TO MONTH, INTERVAL '1-2' YEAR TO MONTH, 10); +SELECT width_bucket(INTERVAL '0' DAY, INTERVAL '0' DAY, INTERVAL '10' DAY, 10); +SELECT width_bucket(INTERVAL '-59' MINUTE, INTERVAL -'1 01' DAY TO HOUR, INTERVAL '1 2:3:4.001' DAY TO SECOND, 10); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/join-empty-relation.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/join-empty-relation.sql new file mode 100644 index 000000000000..8afa3270f4de --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/join-empty-relation.sql @@ -0,0 +1,28 @@ +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a); +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a); + +CREATE TEMPORARY VIEW empty_table as SELECT a FROM t2 WHERE false; + +SELECT * FROM t1 INNER JOIN empty_table; +SELECT * FROM t1 CROSS JOIN empty_table; +SELECT * FROM t1 LEFT OUTER JOIN empty_table; +SELECT * FROM t1 RIGHT OUTER JOIN empty_table; +SELECT * FROM t1 FULL OUTER JOIN empty_table; +SELECT * FROM t1 LEFT SEMI JOIN empty_table; +SELECT * FROM t1 LEFT ANTI JOIN empty_table; + +SELECT * FROM empty_table INNER JOIN t1; +SELECT * FROM empty_table CROSS JOIN t1; +SELECT * FROM empty_table LEFT OUTER JOIN t1; +SELECT * FROM empty_table RIGHT OUTER JOIN t1; +SELECT * FROM empty_table FULL OUTER JOIN t1; +SELECT * FROM empty_table LEFT SEMI JOIN t1; +SELECT * FROM empty_table LEFT ANTI JOIN t1; + +SELECT * FROM empty_table INNER JOIN empty_table; +SELECT * FROM empty_table CROSS JOIN empty_table; +SELECT * FROM empty_table LEFT OUTER JOIN empty_table; +SELECT * FROM empty_table RIGHT OUTER JOIN empty_table; +SELECT * FROM empty_table FULL OUTER JOIN empty_table; +SELECT * FROM empty_table LEFT SEMI JOIN empty_table; +SELECT * FROM empty_table LEFT ANTI JOIN empty_table; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/join-lateral.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/join-lateral.sql new file mode 100644 index 000000000000..e3cef9207d20 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/join-lateral.sql @@ -0,0 +1,559 @@ +-- Test cases for lateral join + +CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2); +CREATE VIEW t2(c1, c2) AS VALUES (0, 2), (0, 3); +CREATE VIEW t3(c1, c2) AS VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4)); +CREATE VIEW t4(c1, c2) AS VALUES (0, 1), (0, 2), (1, 1), (1, 3); + +-- lateral join with single column select +SELECT * FROM t1, LATERAL (SELECT c1); +SELECT * FROM t1, LATERAL (SELECT c1 FROM t2); +SELECT * FROM t1, LATERAL (SELECT t1.c1 FROM t2); +SELECT * FROM t1, LATERAL (SELECT t1.c1 + t2.c1 FROM t2); + +-- lateral join with star expansion +SELECT * FROM t1, LATERAL (SELECT *); +SELECT * FROM t1, LATERAL (SELECT * FROM t2); +SELECT * FROM t1, LATERAL (SELECT t1.*); +SELECT * FROM t1, LATERAL (SELECT t1.*, t2.* FROM t2); +SELECT * FROM t1, LATERAL (SELECT t1.* FROM t2 AS t1); +-- expect error: cannot resolve 't1.*' +-- TODO: Currently we don't allow deep correlation so t1.* cannot be resolved using the outermost query. +SELECT * FROM t1, LATERAL (SELECT t1.*, t2.* FROM t2, LATERAL (SELECT t1.*, t2.*, t3.* FROM t2 AS t3)); + +-- lateral join with different join types +SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3; +SELECT * FROM t1 LEFT JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3; +SELECT * FROM t1 CROSS JOIN LATERAL (SELECT c1 + c2 AS c3); +SELECT * FROM t1 NATURAL JOIN LATERAL (SELECT c1 + c2 AS c2); +SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c2) USING (c2); + +-- lateral join without outer column references +SELECT * FROM LATERAL (SELECT * FROM t1); +SELECT * FROM t1, LATERAL (SELECT * FROM t2); +SELECT * FROM LATERAL (SELECT * FROM t1), LATERAL (SELECT * FROM t2); +SELECT * FROM LATERAL (SELECT * FROM t1) JOIN LATERAL (SELECT * FROM t2); + +-- lateral join with subquery alias +SELECT a, b FROM t1, LATERAL (SELECT c1, c2) s(a, b); + +-- lateral join with foldable outer query references +SELECT * FROM (SELECT 1 AS c1, 2 AS c2), LATERAL (SELECT c1, c2); + +-- lateral join with correlated equality predicates +SELECT * FROM t1, LATERAL (SELECT c2 FROM t2 WHERE t1.c1 = t2.c1); + +-- lateral join with correlated non-equality predicates +SELECT * FROM t1, LATERAL (SELECT c2 FROM t2 WHERE t1.c2 < t2.c2); + +-- SPARK-36114: lateral join with aggregation and correlated non-equality predicates +SELECT * FROM t1, LATERAL (SELECT max(c2) AS m FROM t2 WHERE t1.c2 < t2.c2); + +-- lateral join can reference preceding FROM clause items +SELECT * FROM t1 JOIN t2 JOIN LATERAL (SELECT t1.c2 + t2.c2); +-- expect error: cannot resolve `t2.c1` +SELECT * FROM t1 JOIN LATERAL (SELECT t1.c1 AS a, t2.c1 AS b) s JOIN t2 ON s.b = t2.c1; + +-- SPARK-37716: lateral join with non-deterministic expressions. +-- non-deterministic lateral subquery with single row relation. +SELECT x FROM VALUES (0) t(x) JOIN LATERAL (SELECT x + rand(0) AS y); +SELECT x FROM (SELECT SUM(c1) AS x FROM t1), LATERAL (SELECT x + rand(0) AS y); +-- expect error: lateral subquery must be deterministic when joining with a multi-row relation. +SELECT * FROM t1, LATERAL (SELECT c1 + c2 + rand(0) AS c3); +SELECT * FROM t1, LATERAL (SELECT rand(0) FROM t2); +-- expect error: lateral join cannot have non-deterministic join condition. +SELECT * FROM t1 JOIN LATERAL (SELECT * FROM t2) s ON t1.c1 + rand(0) = s.c1; + +-- multiple lateral joins +SELECT * FROM t1, +LATERAL (SELECT c1 + c2 AS a), +LATERAL (SELECT c1 - c2 AS b), +LATERAL (SELECT a * b AS c); + +-- lateral join in between regular joins +SELECT * FROM t1 +LEFT OUTER JOIN LATERAL (SELECT c2 FROM t2 WHERE t1.c1 = t2.c1) s +LEFT OUTER JOIN t1 t3 ON s.c2 = t3.c2; + +-- nested lateral joins +SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT c1)); +SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1 + 1 AS c1), LATERAL (SELECT c1)); +SELECT * FROM t1, LATERAL ( + SELECT * FROM (SELECT c1, MIN(c2) m FROM t2 WHERE t1.c1 = t2.c1 GROUP BY c1) s, + LATERAL (SELECT m WHERE m > c1) +); +-- expect error: cannot resolve `t1.c1` +SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.c1 + t2.c1)); +-- expect error: cannot resolve `c2` +SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1), LATERAL (SELECT c2)); + +-- uncorrelated scalar subquery inside lateral join +SELECT * FROM t1, LATERAL (SELECT c2, (SELECT MIN(c2) FROM t2)); + +-- correlated scalar subquery inside lateral join +SELECT * FROM t1, LATERAL (SELECT (SELECT SUM(c2) FROM t2 WHERE c1 = a) FROM (SELECT c1 AS a)); +-- expect error: cannot resolve `t1.c1` +SELECT * FROM t1, LATERAL (SELECT c1, (SELECT SUM(c2) FROM t2 WHERE c1 = t1.c1)); + +-- lateral join inside uncorrelated subquery +SELECT * FROM t1 WHERE c1 = (SELECT MIN(a) FROM t2, LATERAL (SELECT c1 AS a)); + +-- lateral join inside correlated subquery +SELECT * FROM t1 WHERE c1 = (SELECT MIN(a) FROM t2, LATERAL (SELECT c1 AS a) WHERE c1 = t1.c1); + +-- join condition has a correlated reference to the left side of the lateral join +SELECT * FROM t1 JOIN lateral (SELECT * FROM t2 JOIN t4 ON t2.c1 = t4.c1 AND t2.c1 = t1.c1); +SELECT * FROM t1 JOIN lateral (SELECT * FROM t2 JOIN t4 ON t2.c1 != t4.c1 AND t2.c1 != t1.c1); +SELECT * FROM t1 LEFT JOIN lateral (SELECT * FROM t4 LEFT JOIN t2 ON t2.c1 = t4.c1 AND t2.c1 = t1.c1); + +-- COUNT bug with a single aggregate expression +SELECT * FROM t1, LATERAL (SELECT COUNT(*) cnt FROM t2 WHERE c1 = t1.c1); + +-- COUNT bug with multiple aggregate expressions +SELECT * FROM t1, LATERAL (SELECT COUNT(*) cnt, SUM(c2) sum FROM t2 WHERE c1 = t1.c1); + +-- COUNT bug without count aggregate +SELECT * FROM t1, LATERAL (SELECT SUM(c2) IS NULL FROM t2 WHERE t1.c1 = t2.c1); + +-- COUNT bug with complex aggregate expressions +SELECT * FROM t1, LATERAL (SELECT COUNT(*) + CASE WHEN sum(c2) IS NULL THEN 0 ELSE sum(c2) END FROM t2 WHERE t1.c1 = t2.c1); + +-- COUNT bug with non-empty group by columns (should not handle the count bug) +SELECT * FROM t1, LATERAL (SELECT c1, COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1 GROUP BY c1); +SELECT * FROM t1, LATERAL (SELECT c2, COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1 GROUP BY c2); + +-- COUNT bug with different join types +SELECT * FROM t1 JOIN LATERAL (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1); +SELECT * FROM t1 LEFT JOIN LATERAL (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1); +SELECT * FROM t1 CROSS JOIN LATERAL (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1); + +-- COUNT bug with group by columns and different join types +SELECT * FROM t1 LEFT JOIN LATERAL (SELECT c1, COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1 GROUP BY c1); +SELECT * FROM t1 CROSS JOIN LATERAL (SELECT c1, COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1 GROUP BY c1); + +-- COUNT bug with non-empty join conditions +SELECT * FROM t1 JOIN LATERAL (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) ON cnt + 1 = c1; + +-- COUNT bug with self join +SELECT * FROM t1, LATERAL (SELECT COUNT(*) cnt FROM t1 t2 WHERE t1.c1 = t2.c1); +SELECT * FROM t1, LATERAL (SELECT COUNT(*) cnt FROM t1 t2 WHERE t1.c1 = t2.c1 HAVING cnt > 0); + +-- COUNT bug with multiple aggregates +SELECT * FROM t1, LATERAL (SELECT SUM(cnt) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1)); +SELECT * FROM t1, LATERAL (SELECT COUNT(cnt) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1)); +SELECT * FROM t1, LATERAL (SELECT SUM(cnt) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1 GROUP BY c1)); +SELECT * FROM t1, LATERAL ( + SELECT COUNT(*) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) + JOIN t2 ON cnt = t2.c1 +); + +-- COUNT bug with correlated predicates above the left outer join +SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) WHERE cnt = c1 - 1); +SELECT * FROM t1, LATERAL (SELECT COUNT(*) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) WHERE cnt = c1 - 1); +SELECT * FROM t1, LATERAL ( + SELECT COUNT(*) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) + WHERE cnt = c1 - 1 GROUP BY cnt +); + +-- COUNT bug with joins in the subquery +SELECT * FROM t1, LATERAL ( + SELECT * FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) + JOIN t2 ON cnt = t2.c1 +); +SELECT * FROM t1, LATERAL ( + SELECT l.cnt + r.cnt + FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) l + JOIN (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) r +); + +-- lateral subquery with group by +SELECT * FROM t1 LEFT JOIN LATERAL (SELECT MIN(c2) FROM t2 WHERE c1 = t1.c1 GROUP BY c1); + +-- lateral join inside CTE +WITH cte1 AS ( + SELECT c1 FROM t1 +), cte2 AS ( + SELECT s.* FROM cte1, LATERAL (SELECT * FROM t2 WHERE c1 = cte1.c1) s +) +SELECT * FROM cte2; + +-- SPARK-41441: lateral join with outer references in Generate +SELECT * FROM t3 JOIN LATERAL (SELECT EXPLODE(c2)); +SELECT * FROM t3 JOIN LATERAL (SELECT EXPLODE_OUTER(c2)); +SELECT * FROM t3 JOIN LATERAL (SELECT EXPLODE(c2)) t(c3) ON c1 = c3; +SELECT * FROM t3 LEFT JOIN LATERAL (SELECT EXPLODE(c2)) t(c3) ON c1 = c3; + +-- Window func +SELECT * FROM t1 JOIN LATERAL + (SELECT sum(t2.c2) over (order by t2.c1) + FROM t2 + WHERE t2.c1 >= t1.c1); + +-- lateral join with union +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 = t1.c1); + +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + UNION DISTINCT + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2); + +-- INTERSECT +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + INTERSECT ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 = t1.c1); + +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + INTERSECT DISTINCT + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2); + +-- EXCEPT +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + EXCEPT ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 = t1.c1); + +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + EXCEPT DISTINCT + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2); + +-- COUNT bug with UNION in subquery +SELECT * FROM t1 JOIN LATERAL + (SELECT COUNT(t2.c2) + FROM t2 + WHERE t2.c1 = t1.c1 + UNION DISTINCT + SELECT COUNT(t4.c2) + FROM t4 + WHERE t4.c1 > t1.c2); + +-- Both correlated and uncorrelated children +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c1, t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + UNION ALL + SELECT t4.c2, t4.c1 + FROM t4 + WHERE t4.c1 = t1.c1); + +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2 + UNION ALL + SELECT t4.c2 + FROM t4); + +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4); + +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2 + UNION DISTINCT + SELECT t4.c2 + FROM t4); + +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2 + INTERSECT ALL + SELECT t4.c2 + FROM t4); + +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2 + INTERSECT DISTINCT + SELECT t4.c2 + FROM t4); + +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2 + EXCEPT ALL + SELECT t4.c2 + FROM t4); + +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2 + EXCEPT DISTINCT + SELECT t4.c2 + FROM t4); + +-- Correlation under group by +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + GROUP BY t2.c2 + UNION ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2 + GROUP BY t4.c2); + +-- Correlation in group by +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c1 - t1.c1 + FROM t2 + GROUP BY t2.c1 - t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2 + GROUP BY t4.c2); + +-- Window func - unsupported +SELECT * FROM t1 JOIN LATERAL + (SELECT sum(t2.c2) over (order by t2.c1) + FROM t2 + WHERE t2.c1 >= t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4); + +-- lateral join under union +SELECT * FROM t1 JOIN LATERAL (SELECT * FROM t2 WHERE t2.c1 = t1.c1) +UNION ALL +SELECT * FROM t1 JOIN t4; + +-- union above and below lateral join +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 = t1.c1) +UNION ALL +SELECT * FROM t2 JOIN LATERAL + (SELECT t1.c2 + FROM t1 + WHERE t2.c1 <= t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 < t2.c1); + +-- Combinations of set ops +SELECT * FROM t1 JOIN LATERAL + ((SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + EXCEPT DISTINCT + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2) + UNION DISTINCT + (SELECT t4.c1 + FROM t4 + WHERE t4.c1 <= t1.c2 + INTERSECT ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 <> t1.c1) +); + +SELECT * FROM t1 JOIN LATERAL + ((SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2) + INTERSECT DISTINCT + (SELECT t4.c1 + FROM t4 + WHERE t4.c1 <= t1.c2 + EXCEPT ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 <> t1.c1) +); + +-- Semi join with correlation on left side - supported +SELECT * FROM t1 JOIN LATERAL (SELECT sum(c1) FROM + (SELECT * + FROM t2 + WHERE t2.c1 <= t1.c1) lhs + LEFT SEMI JOIN + (SELECT * + FROM t4) rhs + ON lhs.c1 <=> rhs.c1 and lhs.c2 <=> rhs.c2 +); + +-- Semi join with correlation on right side - unsupported +SELECT * FROM t1 JOIN LATERAL (SELECT sum(c1) FROM + (SELECT * + FROM t2 + WHERE t2.c1 <= t1.c1) lhs + LEFT SEMI JOIN + (SELECT * + FROM t4 + WHERE t4.c1 > t1.c2) rhs + ON lhs.c1 <=> rhs.c1 and lhs.c2 <=> rhs.c2 +); + +-- SPARK-41961: lateral join with table-valued functions +SELECT * FROM LATERAL EXPLODE(ARRAY(1, 2)); +SELECT * FROM t1, LATERAL RANGE(3); +SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3); +SELECT * FROM t3, LATERAL EXPLODE(c2) t2(v); +SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v); +SELECT * FROM EXPLODE(ARRAY(1, 2)) t(v), LATERAL (SELECT v + 1); + +-- lateral join with table-valued functions and join conditions +SELECT * FROM t1 JOIN LATERAL EXPLODE(ARRAY(c1, c2)) t(c3) ON t1.c1 = c3; +SELECT * FROM t3 JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3; +SELECT * FROM t3 LEFT JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3; + +-- lateral join with table-valued functions in lateral subqueries +SELECT * FROM t1, LATERAL (SELECT * FROM EXPLODE(ARRAY(c1, c2))); +SELECT * FROM t1, LATERAL (SELECT t1.c1 + c3 FROM EXPLODE(ARRAY(c1, c2)) t(c3)); +SELECT * FROM t1, LATERAL (SELECT t1.c1 + c3 FROM EXPLODE(ARRAY(c1, c2)) t(c3) WHERE t1.c2 > 1); +SELECT * FROM t1, LATERAL (SELECT * FROM EXPLODE(ARRAY(c1, c2)) l(x) JOIN EXPLODE(ARRAY(c2, c1)) r(y) ON x = y); + +-- SPARK-42119: lateral join with table-valued functions inline and inline_outer; +CREATE OR REPLACE TEMPORARY VIEW array_struct(id, arr) AS VALUES + (1, ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))), + (2, ARRAY()), + (3, ARRAY(STRUCT(3, 'c'))); +SELECT * FROM t1, LATERAL INLINE(ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))); +SELECT c1, t.* FROM t1, LATERAL INLINE(ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))) t(x, y); +SELECT * FROM array_struct JOIN LATERAL INLINE(arr); +SELECT * FROM array_struct LEFT JOIN LATERAL INLINE(arr) t(k, v) ON id = k; +SELECT * FROM array_struct JOIN LATERAL INLINE_OUTER(arr); +DROP VIEW array_struct; + +-- lateral join with table-valued functions posexplode and posexplode_outer +SELECT * FROM LATERAL posexplode(ARRAY(1, 2)); +SELECT * FROM t1, LATERAL posexplode(ARRAY(c1, c2)) t2(pos, c3); +SELECT * FROM t1 JOIN LATERAL posexplode(ARRAY(c1, c2)) t(pos, c3) ON t1.c1 = c3; +SELECT * FROM t3, LATERAL posexplode(c2) t2(pos, v); +SELECT * FROM t3 JOIN LATERAL posexplode(c2) t(pos, c3) ON t3.c1 = c3; +SELECT * FROM t3, LATERAL posexplode_outer(c2) t2(pos, v); +SELECT * FROM t3 LEFT JOIN LATERAL posexplode(c2) t(pos, c3) ON t3.c1 = c3; +SELECT * FROM t3 LEFT JOIN LATERAL posexplode_outer(c2) t(pos, c3) ON t3.c1 = c3; + +-- lateral join with table-valued function json_tuple +CREATE OR REPLACE TEMP VIEW json_table(key, jstring) AS VALUES + ('1', '{"f1": "1", "f2": "2", "f3": 3, "f5": 5.23}'), + ('2', '{"f1": "1", "f3": "3", "f2": 2, "f4": 4.01}'), + ('3', '{"f1": 3, "f4": "4", "f3": "3", "f2": 2, "f5": 5.01}'), + ('4', cast(null as string)), + ('5', '{"f1": null, "f5": ""}'), + ('6', '[invalid JSON string]'); +SELECT t1.key, t2.* FROM json_table t1, LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2; +SELECT t1.key, t2.* FROM json_table t1, LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2 WHERE t2.c0 IS NOT NULL; +SELECT t1.key, t2.* FROM json_table t1 + JOIN LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2(f1, f2, f3, f4, f5) + ON t1.key = t2.f1; +SELECT t1.key, t2.* FROM json_table t1 + LEFT JOIN LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2(f1, f2, f3, f4, f5) + ON t1.key = t2.f1; +DROP VIEW json_table; + +-- lateral join with table-valued function stack +SELECT t.* FROM t1, LATERAL stack(2, 'Key', c1, 'Value', c2) t; +SELECT t.* FROM t1 JOIN LATERAL stack(1, c1, c2) t(x, y); +SELECT t.* FROM t1 JOIN t3 ON t1.c1 = t3.c1 JOIN LATERAL stack(1, t1.c2, t3.c2) t; +-- expect error +SELECT t.* FROM t1, LATERAL stack(c1, c2); + +-- SPARK-36191: ORDER BY/LIMIT in the correlated subquery +select * from t1 join lateral (select * from t2 where t1.c1 = t2.c1 and t1.c2 < t2.c2 limit 1); + +select * from t1 join lateral (select * from t4 where t1.c1 <= t4.c1 order by t4.c2 limit 10); + +select * from t1 join lateral (select c1, min(c2) as m + from t2 where t1.c1 = t2.c1 and t1.c2 < t2.c2 + group by t2.c1 + order by m); + +select * from t1 join lateral (select c1, min(c2) as m + from t4 where t1.c1 = t4.c1 + group by t4.c1 + limit 1); + +select * from t1 join lateral + ((select t4.c2 from t4 where t1.c1 <= t4.c1 order by t4.c2 limit 1) + union all + (select t4.c1 from t4 where t1.c1 = t4.c1 order by t4.c1 limit 3)); + +select * from t1 join lateral + (select * from + ((select t4.c2 as t from t4 where t1.c1 <= t4.c1) + union all + (select t4.c1 as t from t4 where t1.c1 = t4.c1)) as foo + order by foo.t limit 5); + + +select 1 +from t1 as t_outer +left join + lateral( + select b1,b2 + from + ( + select + t2.c1 as b1, + 1 as b2 + from t2 + union + select t_outer.c1 as b1, + null as b2 + ) as t_inner + where (t_inner.b1 < t_outer.c2 or t_inner.b1 is null) + and t_inner.b1 = t_outer.c1 + order by t_inner.b1,t_inner.b2 desc limit 1 + ) as lateral_table; + +-- clean up +DROP VIEW t1; +DROP VIEW t2; +DROP VIEW t3; +DROP VIEW t4; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/json-functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/json-functions.sql new file mode 100644 index 000000000000..245a6a6dfa7c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/json-functions.sql @@ -0,0 +1,111 @@ +-- to_json +select to_json(named_struct('a', 1, 'b', 2)); +select to_json(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy')); +select to_json(array(named_struct('a', 1, 'b', 2))); +select to_json(map(named_struct('a', 1, 'b', 2), named_struct('a', 1, 'b', 2))); +select to_json(map('a', named_struct('a', 1, 'b', 2))); +select to_json(map('a', 1)); +select to_json(array(map('a',1))); +select to_json(array(map('a',1), map('b',2))); +-- Check if errors handled +select to_json(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE')); +select to_json(named_struct('a', 1, 'b', 2), map('mode', 1)); +select to_json(); + +-- from_json +select from_json('{"a":1}', 'a INT'); +select from_json('{"time":"26/08/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')); +-- Check if errors handled +select from_json('{"a":1}', 1); +select from_json('{"a":1}', 'a InvalidType'); +select from_json('{"a":1}', 'a INT', named_struct('mode', 'PERMISSIVE')); +select from_json('{"a":1}', 'a INT', map('mode', 1)); +select from_json(); +-- json_tuple +SELECT json_tuple('{"a" : 1, "b" : 2}', CAST(NULL AS STRING), 'b', CAST(NULL AS STRING), 'a'); +CREATE TEMPORARY VIEW jsonTable(jsonField, a) AS SELECT * FROM VALUES ('{"a": 1, "b": 2}', 'a'); +SELECT json_tuple(jsonField, 'b', CAST(NULL AS STRING), a) FROM jsonTable; +-- json_tuple exists no foldable null field +SELECT json_tuple('{"a":"1"}', if(c1 < 1, null, 'a')) FROM ( SELECT rand() AS c1 ); +SELECT json_tuple('{"a":"1"}', if(c1 < 1, null, 'a'), if(c2 < 1, null, 'a')) FROM ( SELECT 0 AS c1, rand() AS c2 ); + +-- Clean up +DROP VIEW IF EXISTS jsonTable; + +-- from_json - complex types +select from_json('{"a":1, "b":2}', 'map'); +select from_json('{"a":1, "b":"2"}', 'struct'); + +-- infer schema of json literal +select schema_of_json('{"c1":0, "c2":[1]}'); +select from_json('{"c1":[1, 2, 3]}', schema_of_json('{"c1":[0]}')); + +-- from_json - array type +select from_json('[1, 2, 3]', 'array'); +select from_json('[1, "2", 3]', 'array'); +select from_json('[1, 2, null]', 'array'); + +select from_json('[{"a": 1}, {"a":2}]', 'array>'); +select from_json('{"a": 1}', 'array>'); +select from_json('[null, {"a":2}]', 'array>'); + +select from_json('[{"a": 1}, {"b":2}]', 'array>'); +select from_json('[{"a": 1}, 2]', 'array>'); + +-- from_json - datetime type +select from_json('{"d": "2012-12-15", "t": "2012-12-15 15:15:15"}', 'd date, t timestamp'); +select from_json( + '{"d": "12/15 2012", "t": "12/15 2012 15:15:15"}', + 'd date, t timestamp', + map('dateFormat', 'MM/dd yyyy', 'timestampFormat', 'MM/dd yyyy HH:mm:ss')); +select from_json( + '{"d": "02-29"}', + 'd date', + map('dateFormat', 'MM-dd')); +select from_json( + '{"t": "02-29"}', + 't timestamp', + map('timestampFormat', 'MM-dd')); + +-- to_json - array type +select to_json(array('1', '2', '3')); +select to_json(array(array(1, 2, 3), array(4))); + +-- infer schema of json literal using options +select schema_of_json('{"c1":1}', map('primitivesAsString', 'true')); +select schema_of_json('{"c1":01, "c2":0.1}', map('allowNumericLeadingZeros', 'true', 'prefersDecimal', 'true')); +select schema_of_json(null); +CREATE TEMPORARY VIEW jsonTable(jsonField, a) AS SELECT * FROM VALUES ('{"a": 1, "b": 2}', 'a'); +SELECT schema_of_json(jsonField) FROM jsonTable; + +-- json_array_length +select json_array_length(null); +select json_array_length(2); +select json_array_length(); +select json_array_length(''); +select json_array_length('[]'); +select json_array_length('[1,2,3]'); +select json_array_length('[[1,2],[5,6,7]]'); +select json_array_length('[{"a":123},{"b":"hello"}]'); +select json_array_length('[1,2,3,[33,44],{"key":[2,3,4]}]'); +select json_array_length('{"key":"not a json array"}'); +select json_array_length('[1,2,3,4,5'); + +-- json_object_keys +select json_object_keys(); +select json_object_keys(null); +select json_object_keys(200); +select json_object_keys(''); +select json_object_keys('{}'); +select json_object_keys('{"key": 1}'); +select json_object_keys('{"key": "value", "key2": 2}'); +select json_object_keys('{"arrayKey": [1, 2, 3]}'); +select json_object_keys('{"key":[1,2,3,{"key":"value"},[1,2,3]]}'); +select json_object_keys('{"f1":"abc","f2":{"f3":"a", "f4":"b"}}'); +select json_object_keys('{"k1": [1, 2, {"key": 5}], "k2": {"key2": [1, 2]}}'); +select json_object_keys('{[1,2]}'); +select json_object_keys('{"key": 45, "random_string"}'); +select json_object_keys('[1, 2, 3]'); + +-- Clean up +DROP VIEW IF EXISTS jsonTable; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/keywords-enforced.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/keywords-enforced.sql new file mode 100644 index 000000000000..b0c6e5929b18 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/keywords-enforced.sql @@ -0,0 +1,2 @@ +--SET spark.sql.ansi.enforceReservedKeywords = true +--IMPORT keywords.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/keywords.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/keywords.sql new file mode 100644 index 000000000000..d6b96b784c2a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/keywords.sql @@ -0,0 +1,3 @@ +SELECT * from SQL_KEYWORDS(); + +SELECT keyword from SQL_KEYWORDS() WHERE reserved; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/like-all.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/like-all.sql new file mode 100644 index 000000000000..0304e772e28e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/like-all.sql @@ -0,0 +1,41 @@ +-- test cases for like all + +CREATE OR REPLACE TEMPORARY VIEW like_all_table AS SELECT * FROM (VALUES + ('google', '%oo%'), + ('facebook', '%oo%'), + ('linkedin', '%in')) + as t1(company, pat); + +SELECT company FROM like_all_table WHERE company LIKE ALL ('%oo%', '%go%'); + +SELECT company FROM like_all_table WHERE company LIKE ALL ('microsoft', '%yoo%'); + +SELECT + company, + CASE + WHEN company LIKE ALL ('%oo%', '%go%') THEN 'Y' + ELSE 'N' + END AS is_available, + CASE + WHEN company LIKE ALL ('%oo%', 'go%') OR company LIKE ALL ('%in', 'ms%') THEN 'Y' + ELSE 'N' + END AS mix +FROM like_all_table ; + +-- Mix test with constant pattern and column value +SELECT company FROM like_all_table WHERE company LIKE ALL ('%oo%', pat); + +-- not like all test +SELECT company FROM like_all_table WHERE company NOT LIKE ALL ('%oo%', '%in', 'fa%'); +SELECT company FROM like_all_table WHERE company NOT LIKE ALL ('microsoft', '%yoo%'); +SELECT company FROM like_all_table WHERE company NOT LIKE ALL ('%oo%', 'fa%'); +SELECT company FROM like_all_table WHERE NOT company LIKE ALL ('%oo%', 'fa%'); + +-- null test +SELECT company FROM like_all_table WHERE company LIKE ALL ('%oo%', NULL); +SELECT company FROM like_all_table WHERE company NOT LIKE ALL ('%oo%', NULL); +SELECT company FROM like_all_table WHERE company LIKE ALL (NULL, NULL); +SELECT company FROM like_all_table WHERE company NOT LIKE ALL (NULL, NULL); + +-- negative case +SELECT company FROM like_all_table WHERE company LIKE ALL (); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/like-any.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/like-any.sql new file mode 100644 index 000000000000..a6e9827d58d9 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/like-any.sql @@ -0,0 +1,41 @@ +-- test cases for like any + +CREATE OR REPLACE TEMPORARY VIEW like_any_table AS SELECT * FROM (VALUES + ('google', '%oo%'), + ('facebook', '%oo%'), + ('linkedin', '%in')) + as t1(company, pat); + +SELECT company FROM like_any_table WHERE company LIKE ANY ('%oo%', '%in', 'fa%'); + +SELECT company FROM like_any_table WHERE company LIKE ANY ('microsoft', '%yoo%'); + +select + company, + CASE + WHEN company LIKE ANY ('%oo%', '%in', 'fa%') THEN 'Y' + ELSE 'N' + END AS is_available, + CASE + WHEN company LIKE ANY ('%oo%', 'fa%') OR company LIKE ANY ('%in', 'ms%') THEN 'Y' + ELSE 'N' + END AS mix +FROM like_any_table; + +-- Mix test with constant pattern and column value +SELECT company FROM like_any_table WHERE company LIKE ANY ('%zz%', pat); + +-- not like any test +SELECT company FROM like_any_table WHERE company NOT LIKE ANY ('%oo%', '%in', 'fa%'); +SELECT company FROM like_any_table WHERE company NOT LIKE ANY ('microsoft', '%yoo%'); +SELECT company FROM like_any_table WHERE company NOT LIKE ANY ('%oo%', 'fa%'); +SELECT company FROM like_any_table WHERE NOT company LIKE ANY ('%oo%', 'fa%'); + +-- null test +SELECT company FROM like_any_table WHERE company LIKE ANY ('%oo%', NULL); +SELECT company FROM like_any_table WHERE company NOT LIKE ANY ('%oo%', NULL); +SELECT company FROM like_any_table WHERE company LIKE ANY (NULL, NULL); +SELECT company FROM like_any_table WHERE company NOT LIKE ANY (NULL, NULL); + +-- negative case +SELECT company FROM like_any_table WHERE company LIKE ANY (); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/limit.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/limit.sql new file mode 100644 index 000000000000..b4c73cf33e53 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/limit.sql @@ -0,0 +1,32 @@ + +-- limit on various data types +SELECT * FROM testdata LIMIT 2; +SELECT * FROM arraydata LIMIT 2; +SELECT * FROM mapdata LIMIT 2; + +-- foldable non-literal in limit +SELECT * FROM testdata LIMIT 2 + 1; + +SELECT * FROM testdata LIMIT CAST(1 AS int); + +-- limit must be non-negative +SELECT * FROM testdata LIMIT -1; +SELECT * FROM testData TABLESAMPLE (-1 ROWS); + + +SELECT * FROM testdata LIMIT CAST(1 AS INT); +-- evaluated limit must not be null +SELECT * FROM testdata LIMIT CAST(NULL AS INT); + +-- limit must be foldable +SELECT * FROM testdata LIMIT key > 3; + +-- limit must be integer +SELECT * FROM testdata LIMIT true; +SELECT * FROM testdata LIMIT 'a'; + +-- limit within a subquery +SELECT * FROM (SELECT * FROM range(10) LIMIT 5) WHERE id > 3; + +-- limit ALL +SELECT * FROM testdata WHERE key < 3 LIMIT ALL; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/linear-regression.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/linear-regression.sql new file mode 100644 index 000000000000..df286d2a9b0a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/linear-regression.sql @@ -0,0 +1,52 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testRegression AS SELECT * FROM VALUES +(1, 10, null), (2, 10, 11), (2, 20, 22), (2, 25, null), (2, 30, 35), (2, null, 40) +AS testRegression(k, y, x); + +-- SPARK-37613: Support ANSI Aggregate Function: regr_count +SELECT regr_count(y, x) FROM testRegression; +SELECT regr_count(y, x) FROM testRegression WHERE x IS NOT NULL; +SELECT k, count(*), regr_count(y, x) FROM testRegression GROUP BY k; +SELECT k, count(*) FILTER (WHERE x IS NOT NULL), regr_count(y, x) FROM testRegression GROUP BY k; + +-- SPARK-37613: Support ANSI Aggregate Function: regr_r2 +SELECT regr_r2(y, x) FROM testRegression; +SELECT regr_r2(y, x) FROM testRegression WHERE x IS NOT NULL; +SELECT k, corr(y, x), regr_r2(y, x) FROM testRegression GROUP BY k; +SELECT k, corr(y, x) FILTER (WHERE x IS NOT NULL), regr_r2(y, x) FROM testRegression GROUP BY k; + +-- SPARK-37614: Support ANSI Aggregate Function: regr_avgx & regr_avgy +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression; +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, avg(x), avg(y), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k; +SELECT k, avg(x) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), avg(y) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k; + +-- SPARK-37672: Support ANSI Aggregate Function: regr_sxx +SELECT regr_sxx(y, x) FROM testRegression; +SELECT regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_sxx(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; + +-- SPARK-37681: Support ANSI Aggregate Function: regr_sxy +SELECT regr_sxy(y, x) FROM testRegression; +SELECT regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_sxy(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; + +-- SPARK-37702: Support ANSI Aggregate Function: regr_syy +SELECT regr_syy(y, x) FROM testRegression; +SELECT regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_syy(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; + +-- SPARK-39230: Support ANSI Aggregate Function: regr_slope +SELECT regr_slope(y, x) FROM testRegression; +SELECT regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_slope(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; + +-- SPARK-37623: Support ANSI Aggregate Function: regr_intercept +SELECT regr_intercept(y, x) FROM testRegression; +SELECT regr_intercept(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL; +SELECT k, regr_intercept(y, x) FROM testRegression GROUP BY k; +SELECT k, regr_intercept(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/listagg-collations.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/listagg-collations.sql new file mode 100644 index 000000000000..aa3d02dc2fb6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/listagg-collations.sql @@ -0,0 +1,14 @@ +-- Test cases with utf8_binary +SELECT listagg(c1) WITHIN GROUP (ORDER BY c1 COLLATE utf8_binary) FROM (VALUES ('a'), ('A'), ('b'), ('B')) AS t(c1); +WITH t(c1) AS (SELECT listagg(DISTINCT col1 COLLATE utf8_binary) FROM (VALUES ('a'), ('A'), ('b'), ('B'))) SELECT len(c1), regexp_count(c1, 'a'), regexp_count(c1, 'b'), regexp_count(c1, 'A'), regexp_count(c1, 'B') FROM t; +WITH t(c1) AS (SELECT listagg(col1) WITHIN GROUP (ORDER BY col1) FROM (VALUES ('abc '), ('abc '), ('abc\n'), ('abc'), ('x'))) SELECT replace(replace(c1, ' ', ''), '\n', '$') FROM t; +-- Test cases with utf8_lcase. Lower expression added for determinism +SELECT lower(listagg(c1) WITHIN GROUP (ORDER BY c1 COLLATE utf8_lcase)) FROM (VALUES ('a'), ('A'), ('b'), ('B')) AS t(c1); +WITH t(c1) AS (SELECT lower(listagg(DISTINCT col1 COLLATE utf8_lcase)) FROM (VALUES ('a'), ('A'), ('b'), ('B'))) SELECT len(c1), regexp_count(c1, 'a'), regexp_count(c1, 'b') FROM t; +SELECT lower(listagg(DISTINCT c1 COLLATE utf8_lcase) WITHIN GROUP (ORDER BY c1 COLLATE utf8_lcase)) FROM (VALUES ('a'), ('B'), ('b'), ('A')) AS t(c1); +-- Test cases with unicode_rtrim. +WITH t(c1) AS (SELECT replace(listagg(DISTINCT col1 COLLATE unicode_rtrim) COLLATE utf8_binary, ' ', '') FROM (VALUES ('xbc '), ('xbc '), ('a'), ('xbc'))) SELECT len(c1), regexp_count(c1, 'a'), regexp_count(c1, 'xbc') FROM t; +WITH t(c1) AS (SELECT listagg(col1) WITHIN GROUP (ORDER BY col1 COLLATE unicode_rtrim) FROM (VALUES ('abc '), ('abc\n'), ('abc'), ('x'))) SELECT replace(replace(c1, ' ', ''), '\n', '$') FROM t; + +-- Error case with collations +SELECT listagg(DISTINCT c1 COLLATE utf8_lcase) WITHIN GROUP (ORDER BY c1 COLLATE utf8_binary) FROM (VALUES ('a'), ('b'), ('A'), ('B')) AS t(c1); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/listagg.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/listagg.sql new file mode 100644 index 000000000000..021de9975d30 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/listagg.sql @@ -0,0 +1,41 @@ +--SET spark.sql.binaryOutputStyle=HEX + +-- Create temporary views +CREATE TEMP VIEW df AS +SELECT * FROM (VALUES ('a', 'b'), ('a', 'c'), ('b', 'c'), ('b', 'd'), (NULL, NULL)); + +CREATE TEMP VIEW df2 AS +SELECT * FROM (VALUES (1, true), (2, false), (3, false)); + +-- Test cases for listagg function +WITH t(col) AS (SELECT listagg(col2) FROM df GROUP BY col1) SELECT len(col), regexp_count(col, 'a'), regexp_count(col, 'b'), regexp_count(col, 'c'), regexp_count(col, 'd') FROM t; +WITH t(col) AS (SELECT string_agg(col2) FROM df GROUP BY col1) SELECT len(col), regexp_count(col, 'a'), regexp_count(col, 'b'), regexp_count(col, 'c'), regexp_count(col, 'd') FROM t; +WITH t(col) AS (SELECT listagg(col2, NULL) FROM df GROUP BY col1) SELECT len(col), regexp_count(col, 'a'), regexp_count(col, 'b'), regexp_count(col, 'c'), regexp_count(col, 'd') FROM t; +SELECT listagg(col2) FROM df WHERE 1 != 1; +WITH t(col) AS (SELECT listagg(col2, '|') FROM df GROUP BY col1) SELECT len(col), regexp_count(col, 'a'), regexp_count(col, 'b'), regexp_count(col, 'c'), regexp_count(col, 'd') FROM t; +WITH t(col) AS (SELECT listagg(col1) FROM df) SELECT len(col), regexp_count(col, 'a'), regexp_count(col, 'b') FROM t; +WITH t(col) AS (SELECT listagg(DISTINCT col1) FROM df) SELECT len(col), regexp_count(col, 'a'), regexp_count(col, 'b') FROM t; +SELECT listagg(col1) WITHIN GROUP (ORDER BY col1) FROM df; +SELECT listagg(col1) WITHIN GROUP (ORDER BY col1 DESC) FROM df; +SELECT listagg(col1) WITHIN GROUP (ORDER BY col1 DESC) OVER (PARTITION BY col2) FROM df; +SELECT listagg(col1) WITHIN GROUP (ORDER BY col2) FROM df; +WITH t(col) AS (SELECT listagg(col1) WITHIN GROUP (ORDER BY col2 DESC) FROM df) SELECT (col == 'baba') || (col == 'bbaa') FROM t; +WITH t(col) AS (SELECT listagg(col1, '|') WITHIN GROUP (ORDER BY col2 DESC) FROM df) SELECT (col == 'b|a|b|a') || (col == 'b|b|a|a') FROM t; +SELECT listagg(col1, '|') WITHIN GROUP (ORDER BY col2 DESC) FROM df; +SELECT listagg(col1) WITHIN GROUP (ORDER BY col2 DESC, col1 ASC) FROM df; +SELECT listagg(col1) WITHIN GROUP (ORDER BY col2 DESC, col1 DESC) FROM df; +WITH t(col) AS (SELECT listagg(col1) FROM (VALUES (X'DEAD'), (X'BEEF'))) SELECT len(col), regexp_count(col, X'DEAD'), regexp_count(col, X'BEEF') FROM t; +WITH t(col) AS (SELECT listagg(col1, NULL) FROM (VALUES (X'DEAD'), (X'BEEF'))) SELECT len(col), regexp_count(col, X'DEAD'), regexp_count(col, X'BEEF') FROM t; +WITH t(col) AS (SELECT listagg(col1, X'42') FROM (VALUES (X'DEAD'), (X'BEEF'))) SELECT len(col), regexp_count(col, X'42'), regexp_count(col, X'DEAD'), regexp_count(col, X'BEEF') FROM t; +WITH t(col1, col2) AS (SELECT listagg(col1), listagg(col2, ',') FROM df2) SELECT len(col1), regexp_count(col1, '1'), regexp_count(col1, '2'), regexp_count(col1, '3'), len(col2), regexp_count(col2, 'true'), regexp_count(col1, 'false') FROM t; + +-- Error cases +SELECT listagg(c1) FROM (VALUES (ARRAY('a', 'b'))) AS t(c1); +SELECT listagg(c1, ', ') FROM (VALUES (X'DEAD'), (X'BEEF')) AS t(c1); +SELECT listagg(col2, col1) FROM df GROUP BY col1; +SELECT listagg(col1) OVER (ORDER BY col1) FROM df; +SELECT listagg(col1) WITHIN GROUP (ORDER BY col1) OVER (ORDER BY col1) FROM df; +SELECT string_agg(col1) WITHIN GROUP (ORDER BY col1) OVER (ORDER BY col1) FROM df; +SELECT listagg(DISTINCT col1) OVER (ORDER BY col1) FROM df; +SELECT listagg(DISTINCT col1) WITHIN GROUP (ORDER BY col2) FROM df; +SELECT listagg(DISTINCT col1) WITHIN GROUP (ORDER BY col1, col2) FROM df; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/literals.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/literals.sql new file mode 100644 index 000000000000..e1e4a370bffd --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/literals.sql @@ -0,0 +1,123 @@ +-- Literal parsing + +-- null +select null, Null, nUll; + +-- boolean +select true, tRue, false, fALse; + +-- byte (tinyint) +select 1Y; +select 127Y, -128Y; + +-- out of range byte +select 128Y; + +-- short (smallint) +select 1S; +select 32767S, -32768S; + +-- out of range short +select 32768S; + +-- long (bigint) +select 1L, 2147483648L; +select 9223372036854775807L, -9223372036854775808L; + +-- out of range long +select 9223372036854775808L; + +-- integral parsing + +-- parse int +select 1, -1; + +-- parse int max and min value as int +select 2147483647, -2147483648; + +-- parse long max and min value as long +select 9223372036854775807, -9223372036854775808; + +-- parse as decimals (Long.MaxValue + 1, and Long.MinValue - 1) +select 9223372036854775808, -9223372036854775809; + +-- out of range decimal numbers +select 1234567890123456789012345678901234567890; +select 1234567890123456789012345678901234567890.0; + +-- float +select 1F, 1.2F, .10f, 0.10f; +select -1F, -1.2F, -.10F, -0.10F; +select -3.4028235E39f; + +-- double +select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1; +select -1D, -1.2D, -1e10, -1.5e5, -.10D, -0.10D, -.1e5; +-- negative double +select .e3; +-- very large decimals (overflowing double). +select 1E309, -1E309; + +-- decimal parsing +select 0.3, -0.8, .5, -.18, 0.1111, .1111; +select 0.3 F, 0.4 D, 0.5 BD; + +-- super large scientific notation double literals should still be valid doubles +select 123456789012345678901234567890123456789e10d, 123456789012345678901234567890123456789.1e10d; + +-- string +select "Hello Peter!", 'hello lee!'; +-- multi string +select 'hello' 'world', 'hello' " " 'lee'; +-- single quote within double quotes +select "hello 'peter'"; +select 'pattern%', 'no-pattern\%', 'pattern\\%', 'pattern\\\%'; +select '\'', '"', '\n', '\r', '\t', 'Z'; +-- "Hello!" in octals +select '\110\145\154\154\157\041'; +-- "World :)" in unicode +select '\u0057\u006F\u0072\u006C\u0064\u0020\u003A\u0029'; + +-- date +select dAte '2016-03-12'; +-- invalid date +select date 'mar 11 2016'; + +-- timestamp +select tImEstAmp '2016-03-11 20:54:00.000'; +-- invalid timestamp +select timestamp '2016-33-11 20:54:00.000'; + +-- unsupported data type +select GEO '(10,-6)'; + +-- big decimal parsing +select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD; + +-- out of range big decimal +select 1.20E-38BD; + +-- hexadecimal binary literal +select x'2379ACFe'; + +-- invalid hexadecimal binary literal +select X'XuZ'; + +-- Hive literal_double test. +SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8; + +-- awareness of the negative/positive sign before type +select +date '1999-01-01'; +select +timestamp '1999-01-01'; +select +interval '1 day'; +select +map(1, 2); +select +array(1,2); +select +named_struct('a', 1, 'b', 'spark'); +select +X'1'; +-- can't negate date/timestamp/binary +select -date '1999-01-01'; +select -timestamp '1999-01-01'; +select -x'2379ACFe'; + +-- normalize -0 and -0.0 +select -0, -0.0; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/map.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/map.sql new file mode 100644 index 000000000000..bff2f7a9ae28 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/map.sql @@ -0,0 +1,17 @@ +-- test cases for map functions + +-- key does not exist +select element_at(map(1, 'a', 2, 'b'), 5); +select map(1, 'a', 2, 'b')[5]; + + +-- map_contains_key +select map_contains_key(map(1, 'a', 2, 'b'), 5); +select map_contains_key(map(1, 'a', 2, 'b'), 1); +-- map_contains_key: input type is different from the key type +select map_contains_key(map(1, 'a', 2, 'b'), 5.0); +select map_contains_key(map(1, 'a', 2, 'b'), 1.0); +select map_contains_key(map(1.0, 'a', 2, 'b'), 5); +select map_contains_key(map(1.0, 'a', 2, 'b'), 1); +select map_contains_key(map('1', 'a', '2', 'b'), 1); +select map_contains_key(map(1, 'a', 2, 'b'), '1'); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/mask-functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/mask-functions.sql new file mode 100644 index 000000000000..b4dc8f183036 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/mask-functions.sql @@ -0,0 +1,58 @@ +-- mask function +SELECT mask('AbCD123-@$#'); +SELECT mask('AbCD123-@$#', 'Q'); +SELECT mask('AbCD123-@$#', 'Q', 'q'); +SELECT mask('AbCD123-@$#', 'Q', 'q', 'd'); +SELECT mask('AbCD123-@$#', 'Q', 'q', 'd', 'o'); +SELECT mask('AbCD123-@$#', 'Qa', 'qa', 'da', 'oa'); +SELECT mask('AbCD123-@$#', NULL, 'q', 'd', 'o'); +SELECT mask('AbCD123-@$#', NULL, NULL, 'd', 'o'); +SELECT mask('AbCD123-@$#', NULL, NULL, NULL, 'o'); +SELECT mask('AbCD123-@$#', NULL, NULL, NULL, NULL); +SELECT mask(NULL); +SELECT mask(NULL, NULL, 'q', 'd', 'o'); +SELECT mask(NULL, NULL, NULL, 'd', 'o'); +SELECT mask(NULL, NULL, NULL, NULL, 'o'); +SELECT mask('AbCD123-@$#', NULL, NULL, NULL, NULL); +SELECT mask(c1) from values ('AbCD123-@$#') as tab(c1); +SELECT mask(c1, 'Q') from values ('AbCD123-@$#') as tab(c1); +SELECT mask(c1, 'Q', 'q')from values ('AbCD123-@$#') as tab(c1); +SELECT mask(c1, 'Q', 'q', 'd') from values ('AbCD123-@$#') as tab(c1); +SELECT mask(c1, 'Q', 'q', 'd', 'o') from values ('AbCD123-@$#') as tab(c1); +SELECT mask(c1, NULL, 'q', 'd', 'o') from values ('AbCD123-@$#') as tab(c1); +SELECT mask(c1, NULL, NULL, 'd', 'o') from values ('AbCD123-@$#') as tab(c1); +SELECT mask(c1, NULL, NULL, NULL, 'o') from values ('AbCD123-@$#') as tab(c1); +SELECT mask(c1, NULL, NULL, NULL, NULL) from values ('AbCD123-@$#') as tab(c1); +SELECT mask(c1, NULL, 'q', 'd', 'o') from values ('AbCD123-@$#') as tab(c1); +SELECT mask(c1, 'Q', NULL, 'd', 'o') from values ('AbCD123-@$#') as tab(c1); +SELECT mask(c1, 'Q', 'q', NULL, 'o') from values ('AbCD123-@$#') as tab(c1); +SELECT mask(c1, 'Q', 'q', 'd', NULL) from values ('AbCD123-@$#') as tab(c1); +SELECT mask(NULL, 'Q', 'q', 'd', NULL) from values ('AbCD123-@$#') as tab(c1); +SELECT mask('abcd-EFGH-8765-4321'); +SELECT mask('abcd-EFGH-8765-4321', 'Q'); +SELECT mask('abcd-EFGH-8765-4321', 'Q', 'q'); +SELECT mask('abcd-EFGH-8765-4321', 'Q', 'q', 'd'); +SELECT mask('abcd-EFGH-8765-4321', 'Q', 'q', 'd', '*'); +SELECT mask('abcd-EFGH-8765-4321', NULL, 'q', 'd', '*'); +SELECT mask('abcd-EFGH-8765-4321', NULL, NULL, 'd', '*'); +SELECT mask('abcd-EFGH-8765-4321', NULL, NULL, NULL, '*'); +SELECT mask('abcd-EFGH-8765-4321', NULL, NULL, NULL, NULL); +SELECT mask(NULL); +SELECT mask(NULL, NULL, 'q', 'd', '*'); +SELECT mask(NULL, NULL, NULL, 'd', '*'); +SELECT mask(NULL, NULL, NULL, NULL, '*'); +SELECT mask(NULL, NULL, NULL, NULL, NULL); +SELECT mask(c1) from values ('abcd-EFGH-8765-4321') as tab(c1); +SELECT mask(c1, 'Q') from values ('abcd-EFGH-8765-4321') as tab(c1); +SELECT mask(c1, 'Q', 'q')from values ('abcd-EFGH-8765-4321') as tab(c1); +SELECT mask(c1, 'Q', 'q', 'd') from values ('abcd-EFGH-8765-4321') as tab(c1); +SELECT mask(c1, 'Q', 'q', 'd', '*') from values ('abcd-EFGH-8765-4321') as tab(c1); +SELECT mask(c1, NULL, 'q', 'd', '*') from values ('abcd-EFGH-8765-4321') as tab(c1); +SELECT mask(c1, NULL, NULL, 'd', '*') from values ('abcd-EFGH-8765-4321') as tab(c1); +SELECT mask(c1, NULL, NULL, NULL, '*') from values ('abcd-EFGH-8765-4321') as tab(c1); +SELECT mask(c1, NULL, NULL, NULL, NULL) from values ('abcd-EFGH-8765-4321') as tab(c1); +SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 'a') as t(c1, replaceArg); +SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 'ABC') as t(c1, replaceArg); +SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 123) as t(c1, replaceArg); +SELECT mask('abcd-EFGH-8765-4321', 'A', 'w', ''); +SELECT mask('abcd-EFGH-8765-4321', 'A', 'abc'); \ No newline at end of file diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/math.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/math.sql new file mode 100644 index 000000000000..14a647a610cc --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/math.sql @@ -0,0 +1,92 @@ +-- Round with Byte input +SELECT round(25y, 1); +SELECT round(25y, 0); +SELECT round(25y, -1); +SELECT round(25y, -2); +SELECT round(25y, -3); +SELECT round(127y, -1); +SELECT round(-128y, -1); + +-- Round with short integer input +SELECT round(525s, 1); +SELECT round(525s, 0); +SELECT round(525s, -1); +SELECT round(525s, -2); +SELECT round(525s, -3); +SELECT round(32767s, -1); +SELECT round(-32768s, -1); + +-- Round with integer input +SELECT round(525, 1); +SELECT round(525, 0); +SELECT round(525, -1); +SELECT round(525, -2); +SELECT round(525, -3); +SELECT round(2147483647, -1); +SELECT round(-2147483647, -1); + +-- Round with big integer input +SELECT round(525L, 1); +SELECT round(525L, 0); +SELECT round(525L, -1); +SELECT round(525L, -2); +SELECT round(525L, -3); +SELECT round(9223372036854775807L, -1); +SELECT round(-9223372036854775808L, -1); + +-- Bround with byte input +SELECT bround(25y, 1); +SELECT bround(25y, 0); +SELECT bround(25y, -1); +SELECT bround(25y, -2); +SELECT bround(25y, -3); +SELECT bround(127y, -1); +SELECT bround(-128y, -1); + +-- Bround with Short input +SELECT bround(525s, 1); +SELECT bround(525s, 0); +SELECT bround(525s, -1); +SELECT bround(525s, -2); +SELECT bround(525s, -3); +SELECT bround(32767s, -1); +SELECT bround(-32768s, -1); + +-- Bround with integer input +SELECT bround(525, 1); +SELECT bround(525, 0); +SELECT bround(525, -1); +SELECT bround(525, -2); +SELECT bround(525, -3); +SELECT bround(2147483647, -1); +SELECT bround(-2147483647, -1); + +-- Bround with big integer input +SELECT bround(525L, 1); +SELECT bround(525L, 0); +SELECT bround(525L, -1); +SELECT bround(525L, -2); +SELECT bround(525L, -3); +SELECT bround(9223372036854775807L, -1); +SELECT bround(-9223372036854775808L, -1); + +-- Conv +SELECT conv('100', 2, 10); +SELECT conv(-10, 16, -10); +SELECT conv('9223372036854775808', 10, 16); +SELECT conv('92233720368547758070', 10, 16); +SELECT conv('9223372036854775807', 36, 10); +SELECT conv('-9223372036854775807', 36, 10); + +SELECT BIN(0); +SELECT BIN(25); +SELECT BIN(25L); +SELECT BIN(25.5); + +SELECT POSITIVE(0Y); +SELECT POSITIVE(25); +SELECT POSITIVE(-25L); +SELECT POSITIVE(25.5); +SELECT POSITIVE("25.5"); +SELECT POSITIVE("invalid"); +SELECT POSITIVE(null); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/misc-functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/misc-functions.sql new file mode 100644 index 000000000000..e178f69c7d51 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/misc-functions.sql @@ -0,0 +1,45 @@ +-- test for misc functions +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +-- typeof +select typeof(null); +select typeof(true); +select typeof(1Y), typeof(1S), typeof(1), typeof(1L); +select typeof(cast(1.0 as float)), typeof(1.0D), typeof(1.2); +select typeof(date '1986-05-23'), typeof(timestamp '1986-05-23'), typeof(interval '23 days'); +select typeof(x'ABCD'), typeof('SPARK'); +select typeof(array(1, 2)), typeof(map(1, 2)), typeof(named_struct('a', 1, 'b', 'spark')); + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; +-- Spark-32793: Rewrite AssertTrue with RaiseError +SELECT assert_true(true), assert_true(boolean(1)); +SELECT assert_true(false); +SELECT assert_true(boolean(0)); +SELECT assert_true(null); +SELECT assert_true(boolean(null)); +SELECT assert_true(false, 'custom error message'); + +CREATE TEMPORARY VIEW tbl_misc AS SELECT * FROM (VALUES (1), (8), (2)) AS T(v); +SELECT raise_error('error message'); +SELECT if(v > 5, raise_error('too big: ' || v), v + 1) FROM tbl_misc; + +-- Too many parameters +SELECT raise_error('error message', Map()); + +-- Too many parameters +SELECT raise_error('error message', 'some args'); + +-- Too few parameters +SELECT raise_error(); + +-- Passing null as message +SELECT raise_error(NULL); + +-- Passing non-string type +SELECT raise_error(1); + +-- Check legacy config disables printing of [USER_RAISED_EXCEPTION] +SET spark.sql.legacy.raiseErrorWithoutErrorClass=true; +SELECT assert_true(false); +SELECT raise_error('hello'); +SET spark.sql.legacy.raiseErrorWithoutErrorClass=false; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/mode.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/mode.sql new file mode 100644 index 000000000000..85035da7ad3f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/mode.sql @@ -0,0 +1,139 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW basic_pays AS SELECT * FROM VALUES +('Diane Murphy','Accounting',8435), +('Mary Patterson','Accounting',9998), +('Jeff Firrelli','Accounting',8992), +('William Patterson','Accounting',8870), +('Gerard Bondur','Accounting',11472), +('Anthony Bow','Accounting',6627), +('Leslie Jennings','IT',8113), +('Leslie Thompson','IT',5186), +('Julie Firrelli','Sales',9181), +('Steve Patterson','Sales',9441), +('Foon Yue Tseng','Sales',6660), +('George Vanauf','Sales',10563), +('Loui Bondur','SCM',10449), +('Gerard Hernandez','SCM',6949), +('Pamela Castillo','SCM',11303), +('Larry Bott','SCM',11798), +('Barry Jones','SCM',10586) +AS basic_pays(employee_name, department, salary); + +SELECT mode(department), mode(salary) FROM basic_pays; +SELECT department, mode(salary) FROM basic_pays GROUP BY department ORDER BY department; +SELECT department, mode(DISTINCT salary) FROM basic_pays GROUP BY department ORDER BY department; + +-- SPARK-45034: Support deterministic mode function +SELECT mode(col) FROM VALUES (-10), (0), (10) AS tab(col); +SELECT mode(col, false) FROM VALUES (-10), (0), (10) AS tab(col); +SELECT mode(col, true) FROM VALUES (-10), (0), (10) AS tab(col); +SELECT mode(col, 'true') FROM VALUES (-10), (0), (10) AS tab(col); +SELECT mode(col, null) FROM VALUES (-10), (0), (10) AS tab(col); +SELECT mode(col, b) FROM VALUES (-10, false), (0, false), (10, false) AS tab(col, b); +SELECT mode(col) FROM VALUES (map(1, 'a')) AS tab(col); +SELECT mode(col, false) FROM VALUES (map(1, 'a')) AS tab(col); +SELECT mode(col, true) FROM VALUES (map(1, 'a')) AS tab(col); + +SELECT + mode() WITHIN GROUP (ORDER BY col), + mode() WITHIN GROUP (ORDER BY col DESC) +FROM VALUES (null), (null), (null) AS tab(col); + +SELECT + mode() WITHIN GROUP (ORDER BY salary), + mode() WITHIN GROUP (ORDER BY salary DESC) +FROM basic_pays +WHERE salary > 20000; + +SELECT + mode() WITHIN GROUP (ORDER BY salary), + mode() WITHIN GROUP (ORDER BY salary DESC) +FROM basic_pays; + +SELECT + mode() WITHIN GROUP (ORDER BY salary), + mode() WITHIN GROUP (ORDER BY salary) FILTER (WHERE salary > 10000) +FROM basic_pays; + +SELECT + department, + mode() WITHIN GROUP (ORDER BY salary), + mode() WITHIN GROUP (ORDER BY salary) FILTER (WHERE salary > 10000) +FROM basic_pays +GROUP BY department +ORDER BY department; + +SELECT + employee_name, + department, + salary, + mode() WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department) +FROM basic_pays +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + mode() WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary) +FROM basic_pays +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + mode() WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +FROM basic_pays +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + mode() WITHIN GROUP (ORDER BY salary) OVER w +FROM basic_pays +WHERE salary > 8900 +WINDOW w AS (PARTITION BY department) +ORDER BY salary; + +SELECT + mode(DISTINCT salary) WITHIN GROUP (ORDER BY salary) +FROM basic_pays; + +SELECT + mode() +FROM basic_pays; + +SELECT + mode(salary) WITHIN GROUP (ORDER BY salary) +FROM basic_pays; + +CREATE OR REPLACE TEMPORARY VIEW intervals AS SELECT * FROM VALUES +(0, INTERVAL '0' MONTH, INTERVAL '0' SECOND, INTERVAL '0' MINUTE), +(0, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(0, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(0, INTERVAL '30' MONTH, INTERVAL '30' SECOND, INTERVAL '30' MINUTE), +(0, INTERVAL '40' MONTH, INTERVAL '40' SECOND, INTERVAL '40' MINUTE), +(1, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(1, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(2, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(2, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(2, INTERVAL '25' MONTH, INTERVAL '25' SECOND, INTERVAL '25' MINUTE), +(2, INTERVAL '30' MONTH, INTERVAL '30' SECOND, INTERVAL '30' MINUTE), +(3, INTERVAL '60' MONTH, INTERVAL '60' SECOND, INTERVAL '60' MINUTE), +(4, null, null, null) +AS intervals(k, dt, ym, dt2); + +SELECT + mode() WITHIN GROUP (ORDER BY dt), + mode() WITHIN GROUP (ORDER BY dt DESC) +FROM intervals; + +SELECT + k, + mode() WITHIN GROUP (ORDER BY ym), + mode() WITHIN GROUP (ORDER BY dt DESC) +FROM intervals +GROUP BY k +ORDER BY k; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/named-function-arguments.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/named-function-arguments.sql new file mode 100644 index 000000000000..a795a19828c1 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/named-function-arguments.sql @@ -0,0 +1,74 @@ +-- Test for named arguments for Mask +SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd'); +SELECT mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbCD123-@$#'); +SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', digitChar => 'd'); +SELECT mask(lowerChar => 'q', upperChar => 'Q', digitChar => 'd', str => 'AbCD123-@$#'); + +-- Test for named arguments for CountMinSketchAgg +create temporary view t2 as select * from values + ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +SELECT hex(count_min_sketch(t2d, seed => 1, epsilon => 0.5d, confidence => 0.5d)) FROM t2; +SELECT hex(count_min_sketch(seed => 1, epsilon => 0.5d, confidence => 0.5d, column => t2d)) FROM t2; +SELECT hex(count_min_sketch(t2d, 0.5d, seed => 1, confidence => 0.5d)) FROM t2; + +-- Test for tabled value functions explode and explode_outer +SELECT * FROM explode(collection => array(1, 2)); +SELECT * FROM explode_outer(collection => map('a', 1, 'b', 2)); +SELECT * FROM explode(array(1, 2)), explode(array(3, 4)); +SELECT * FROM explode(array(1, 2)) AS t, LATERAL explode(array(3 * t.col, 4 * t.col)); +SELECT num, val, 'Spark' FROM explode(map(1, 'a', 2, 'b')) AS t(num, val); + +-- Test for tabled value functions posexplode and posexplode_outer +SELECT * FROM posexplode(collection => array(1, 2)); +SELECT * FROM posexplode_outer(collection => map('a', 1, 'b', 2)); +SELECT * FROM posexplode(array(1, 2)), posexplode(array(3, 4)); +SELECT * FROM posexplode(array(1, 2)) AS t, LATERAL posexplode(array(3 * t.col, 4 * t.col)); +SELECT pos, num, val, 'Spark' FROM posexplode(map(1, 'a', 2, 'b')) AS t(pos, num, val); + +-- Test for tabled value functions inline and inline_outer +SELECT * FROM inline(input => array(struct(1, 'a'), struct(2, 'b'))); +SELECT * FROM inline_outer(input => array(struct(1, 'a'), struct(2, 'b'))); +SELECT * FROM inline(array(struct(1, 'a'), struct(2, 'b'))), inline(array(struct(3, 'c'), struct(4, 'd'))); +SELECT * FROM inline(array(struct(1, 'a'), struct(2, 'b'))) AS t, LATERAL inline(array(struct(3 * t.col1, 4 * t.col1))); +SELECT num, val, 'Spark' FROM inline(array(struct(1, 'a'), struct(2, 'b'))) AS t(num, val); + +-- Test for wrapped EXPLODE call to check error preservation +SELECT * FROM explode(collection => explode(array(1))); +SELECT * FROM explode(collection => explode(collection => array(1))); + +-- Test with TABLE parser rule +CREATE OR REPLACE TEMPORARY VIEW v AS SELECT id FROM range(0, 8); +SELECT * FROM explode(collection => TABLE(v)); + +-- Unexpected positional argument +SELECT mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd'); + +-- Duplicate parameter assignment +SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', digitChar => 'e'); +SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbC'); + +-- Required parameter not found +SELECT mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd'); + +-- Unrecognized parameter name +SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', cellular => 'automata'); + +-- Named arguments not supported +SELECT encode(str => 'a', charset => 'utf-8'); + +-- Wrong number of arguments +SELECT mask('AbCD123-@$#', 'Q', 'q', 'd', 'o', 'k'); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/natural-join.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/natural-join.sql new file mode 100644 index 000000000000..5815eee2d9e4 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/natural-join.sql @@ -0,0 +1,77 @@ +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1); + +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2); + +create temporary view nt3 as select * from values + ("one", 4), + ("two", 5), + ("one", 6) + as nt3(k, v3); + +create temporary view nt4 as select * from values + ("one", 7), + ("two", 8), + ("one", 9) + as nt4(k, v4); + +SELECT * FROM nt1 natural join nt2; + +SELECT * FROM nt1 natural join nt2 where k = "one"; + +SELECT * FROM nt1 natural left join nt2 order by v1, v2; + +SELECT * FROM nt1 natural right join nt2 order by v1, v2; + +SELECT count(*) FROM nt1 natural full outer join nt2; + +SELECT k FROM nt1 natural join nt2; + +SELECT k FROM nt1 natural join nt2 where k = "one"; + +SELECT nt1.* FROM nt1 natural join nt2; + +SELECT nt2.* FROM nt1 natural join nt2; + +SELECT sbq.* from (SELECT * FROM nt1 natural join nt2) sbq; + +SELECT sbq.k from (SELECT * FROM nt1 natural join nt2) sbq; + +SELECT nt1.*, nt2.* FROM nt1 natural join nt2; + +SELECT *, nt2.k FROM nt1 natural join nt2; + +SELECT nt1.k, nt2.k FROM nt1 natural join nt2; + +SELECT k FROM (SELECT nt2.k FROM nt1 natural join nt2); + +SELECT nt2.k AS key FROM nt1 natural join nt2 ORDER BY key; + +SELECT nt1.k, nt2.k FROM nt1 natural join nt2 where k = "one"; + +SELECT * FROM (SELECT * FROM nt1 natural join nt2); + +SELECT * FROM (SELECT nt1.*, nt2.* FROM nt1 natural join nt2); + +SELECT * FROM (SELECT nt1.v1, nt2.k FROM nt1 natural join nt2); + +SELECT nt2.k FROM (SELECT * FROM nt1 natural join nt2); + +SELECT * FROM nt1 natural join nt2 natural join nt3; + +SELECT nt1.*, nt2.*, nt3.* FROM nt1 natural join nt2 natural join nt3; + +SELECT nt1.*, nt2.*, nt3.* FROM nt1 natural join nt2 join nt3 on nt2.k = nt3.k; + +SELECT * FROM nt1 natural join nt2 join nt3 on nt1.k = nt3.k; + +SELECT * FROM nt1 natural join nt2 join nt3 on nt2.k = nt3.k; + +SELECT nt1.*, nt2.*, nt3.*, nt4.* FROM nt1 natural join nt2 natural join nt3 natural join nt4; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/non-excludable-rule.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/non-excludable-rule.sql new file mode 100644 index 000000000000..c02bc8c6063f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/non-excludable-rule.sql @@ -0,0 +1,19 @@ +-- SPARK-39448 +SET spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ReplaceCTERefWithRepartition; +SELECT + (SELECT min(id) FROM range(10)), + (SELECT sum(id) FROM range(10)), + (SELECT count(distinct id) FROM range(10)); + +-- SPARK-39444 +SET spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.Optimizer$OptimizeSubqueries; +WITH tmp AS ( + SELECT id FROM range(2) + INTERSECT + SELECT id FROM range(4) +) +SELECT id FROM range(3) WHERE id > (SELECT max(id) FROM tmp); + +-- SPARK-36979 +SET spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.RewriteLateralSubquery; +SELECT * FROM testData, LATERAL (SELECT * FROM testData) LIMIT 1; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/array.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/array.sql new file mode 100644 index 000000000000..b04abe57cb86 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/array.sql @@ -0,0 +1 @@ +--IMPORT array.sql \ No newline at end of file diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/cast.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/cast.sql new file mode 100644 index 000000000000..b16ee89ac67e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/cast.sql @@ -0,0 +1 @@ +--IMPORT cast.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/conditional-functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/conditional-functions.sql new file mode 100644 index 000000000000..c4b3418cc6d8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/conditional-functions.sql @@ -0,0 +1 @@ +--IMPORT conditional-functions.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/date.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/date.sql new file mode 100644 index 000000000000..1f6e366eba94 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/date.sql @@ -0,0 +1 @@ +--IMPORT date.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/datetime-parsing-invalid.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/datetime-parsing-invalid.sql new file mode 100644 index 000000000000..70022f33337d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/datetime-parsing-invalid.sql @@ -0,0 +1,2 @@ +--IMPORT datetime-parsing-invalid.sql + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/datetime-special.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/datetime-special.sql new file mode 100644 index 000000000000..87a5986ae9a9 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/datetime-special.sql @@ -0,0 +1 @@ +--IMPORT datetime-special.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/decimalArithmeticOperations.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/decimalArithmeticOperations.sql new file mode 100644 index 000000000000..7d6c336df452 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/decimalArithmeticOperations.sql @@ -0,0 +1 @@ +--IMPORT decimalArithmeticOperations.sql \ No newline at end of file diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/double-quoted-identifiers.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/double-quoted-identifiers.sql new file mode 100644 index 000000000000..7ccd294e709b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/double-quoted-identifiers.sql @@ -0,0 +1 @@ +--IMPORT double-quoted-identifiers.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/higher-order-functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/higher-order-functions.sql new file mode 100644 index 000000000000..1e2424fe47ca --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/higher-order-functions.sql @@ -0,0 +1 @@ +--IMPORT higher-order-functions.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/interval.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/interval.sql new file mode 100644 index 000000000000..215ce9658e1a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/interval.sql @@ -0,0 +1 @@ +--IMPORT interval.sql \ No newline at end of file diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/keywords.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/keywords.sql new file mode 100644 index 000000000000..ebc9345c3a73 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/keywords.sql @@ -0,0 +1 @@ +--IMPORT keywords.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/literals.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/literals.sql new file mode 100644 index 000000000000..698e8fa88630 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/literals.sql @@ -0,0 +1,2 @@ +--- malformed interval literal with ansi mode +--IMPORT literals.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/map.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/map.sql new file mode 100644 index 000000000000..23e5b9562973 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/map.sql @@ -0,0 +1 @@ +--IMPORT map.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/math.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/math.sql new file mode 100644 index 000000000000..5ee19c28ca6b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/math.sql @@ -0,0 +1 @@ +--IMPORT math.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/parse-schema-string.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/parse-schema-string.sql new file mode 100644 index 000000000000..42775102e650 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/parse-schema-string.sql @@ -0,0 +1 @@ +--IMPORT parse-schema-string.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/string-functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/string-functions.sql new file mode 100644 index 000000000000..dd28e9b97fb2 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/string-functions.sql @@ -0,0 +1 @@ +--IMPORT string-functions.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/timestamp.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/timestamp.sql new file mode 100644 index 000000000000..79193c900d04 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/timestamp.sql @@ -0,0 +1 @@ +--IMPORT timestamp.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/try_aggregates.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/try_aggregates.sql new file mode 100644 index 000000000000..f5b44d2b2aa7 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/try_aggregates.sql @@ -0,0 +1 @@ +--IMPORT try_aggregates.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/try_arithmetic.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/try_arithmetic.sql new file mode 100644 index 000000000000..a0165b9784ae --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/try_arithmetic.sql @@ -0,0 +1 @@ +--IMPORT try_arithmetic.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/try_datetime_functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/try_datetime_functions.sql new file mode 100644 index 000000000000..ede47f3eecb9 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/try_datetime_functions.sql @@ -0,0 +1 @@ +--IMPORT try_datetime_functions.sql \ No newline at end of file diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/try_element_at.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/try_element_at.sql new file mode 100644 index 000000000000..64d6197fa205 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/nonansi/try_element_at.sql @@ -0,0 +1 @@ +--IMPORT try_element_at.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/null-handling.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/null-handling.sql new file mode 100644 index 000000000000..dcdf241df73d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/null-handling.sql @@ -0,0 +1,53 @@ +-- Create a test table with data +create table t1(a int, b int, c int) using parquet; +insert into t1 values(1,0,0); +insert into t1 values(2,0,1); +insert into t1 values(3,1,0); +insert into t1 values(4,1,1); +insert into t1 values(5,null,0); +insert into t1 values(6,null,1); +insert into t1 values(7,null,null); + +-- Adding anything to null gives null +select a, b+c from t1; +select b + 0 from t1 where a = 5; +select -100 + b + 100 from t1 where a = 5; + +-- Multiplying null by zero gives null +select a+10, b*0 from t1; + +-- nulls are NOT distinct in SELECT DISTINCT +select distinct b from t1; + +-- nulls are NOT distinct in UNION +select b from t1 union select b from t1; + +-- CASE WHEN null THEN 1 ELSE 0 END is 0 +select a+20, case b when c then 1 else 0 end from t1; +select a+30, case c when b then 1 else 0 end from t1; +select a+40, case when b<>0 then 1 else 0 end from t1; +select a+50, case when not b<>0 then 1 else 0 end from t1; +select a+60, case when b<>0 and c<>0 then 1 else 0 end from t1; + +-- "not (null AND false)" is true +select a+70, case when not (b<>0 and c<>0) then 1 else 0 end from t1; + +-- "null OR true" is true +select a+80, case when b<>0 or c<>0 then 1 else 0 end from t1; +select a+90, case when not (b<>0 or c<>0) then 1 else 0 end from t1; + +-- null with aggregate operators +select count(*), count(b), sum(b), avg(b), min(b), max(b) from t1; + +-- Check the behavior of NULLs in WHERE clauses +select a+100 from t1 where b<10; +select a+110 from t1 where not b>10; +select a+120 from t1 where b<10 OR c=1; +select a+130 from t1 where b<10 AND c=1; +select a+140 from t1 where not (b<10 AND c=1); +select a+150 from t1 where not (c=1 AND b<10); + +-- null-safe comparisons +select b, c, equal_null(b, c), equal_null(c, b) from t1; + +drop table t1; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/null-propagation.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/null-propagation.sql new file mode 100644 index 000000000000..66549da7971d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/null-propagation.sql @@ -0,0 +1,9 @@ + +-- count(null) should be 0 +SELECT COUNT(NULL) FROM VALUES 1, 2, 3; +SELECT COUNT(1 + NULL) FROM VALUES 1, 2, 3; + +-- count(null) on window should be 0 +SELECT COUNT(NULL) OVER () FROM VALUES 1, 2, 3; +SELECT COUNT(1 + NULL) OVER () FROM VALUES 1, 2, 3; + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/operators.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/operators.sql new file mode 100644 index 000000000000..33274665f55c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/operators.sql @@ -0,0 +1,105 @@ + +-- unary minus and plus +select -100; +select +230; +select -5.2; +select +6.8e0; +select -key, +key from testdata where key = 2; +select -(key + 1), - key + 1, +(key + 5) from testdata where key = 1; +select -max(key), +max(key) from testdata; +select - (-10); +select + (-key) from testdata where key = 32; +select - (+max(key)) from testdata; +select - - 3; +select - + 20; +select + + 100; +select - - max(key) from testdata; +select + - key from testdata where key = 33; + +-- division +select 5 / 2; +select 5 / 0; +select 5 / null; +select null / 5; + +-- integral div +select 5 div 2; +select 5 div 0; +select 5 div null; +select null div 5; +select cast(51 as decimal(10, 0)) div cast(2 as decimal(2, 0)); +select cast(5 as decimal(1, 0)) div cast(0 as decimal(2, 0)); +select cast(5 as decimal(1, 0)) div cast(null as decimal(2, 0)); +select cast(null as decimal(1, 0)) div cast(5 as decimal(2, 0)); + +-- other arithmetics +select 1 + 2; +select 1 - 2; +select 2 * 5; +select 5 % 3; +select pmod(-7, 3); + +-- math functions +select sec(1); +select sec(null); +select sec(0); +select sec(-1); +select csc(1); +select csc(null); +select csc(0); +select csc(-1); +select cot(1); +select cot(null); +select cot(0); +select cot(-1); + +-- ceil and ceiling +select ceiling(0); +select ceiling(1); +select ceil(1234567890123456); +select ceiling(1234567890123456); +select ceil(0.01); +select ceiling(-0.10); + +-- floor +select floor(0); +select floor(1); +select floor(1234567890123456); +select floor(0.01); +select floor(-0.10); + +-- comparison operator +select 1 > 0.00001; + +-- mod +select mod(7, 2), mod(7, 0), mod(0, 2), mod(7, null), mod(null, 2), mod(null, null); + +-- length +select BIT_LENGTH('abc'); +select CHAR_LENGTH('abc'); +select CHARACTER_LENGTH('abc'); +select OCTET_LENGTH('abc'); + +-- abs +select abs(-3.13), abs('-2.19'); + +-- positive/negative +select positive('-1.11'), positive(-1.11), negative('-1.11'), negative(-1.11); + +-- pmod +select pmod(-7, 2), pmod(0, 2), pmod(7, 0), pmod(7, null), pmod(null, 2), pmod(null, null); +select pmod(cast(3.13 as decimal), cast(0 as decimal)), pmod(cast(2 as smallint), cast(0 as smallint)); + +-- width_bucket +select width_bucket(5.35, 0.024, 10.06, 5); +select width_bucket(5.35, 0.024, 10.06, 3 + 2); +select width_bucket('5.35', '0.024', '10.06', '5'); +select width_bucket(5.35, 0.024, 10.06, 2.5); +select width_bucket(5.35, 0.024, 10.06, 0.5); +select width_bucket(null, 0.024, 10.06, 5); +select width_bucket(5.35, null, 10.06, 5); +select width_bucket(5.35, 0.024, null, -5); +select width_bucket(5.35, 0.024, 10.06, null); +select width_bucket(5.35, 0.024, 10.06, -5); +select width_bucket(5.35, 0.024, 10.06, 9223372036854775807L); -- long max value +select width_bucket(5.35, 0.024, 10.06, 9223372036854775807L - 1); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/order-by-all.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/order-by-all.sql new file mode 100644 index 000000000000..56472cad7644 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/order-by-all.sql @@ -0,0 +1,47 @@ +create temporary view data as select * from values + (0, 1), + (0, 2), + (1, 3), + (1, NULL) + as data(g, i); + +-- most basic test with only 1 column +select g from data order by all; + +-- two columns +select * from data order by all; + +-- test case insensitive +select * from data order by aLl; + +-- asc/desc +select * from data order by all asc; +select * from data order by all desc; + +-- nulls first / last +select * from data order by all nulls first; +select * from data order by all nulls last; + +-- combining nulls first/last/asc/desc +select * from data order by all asc nulls first; +select * from data order by all desc nulls first; +select * from data order by all asc nulls last; +select * from data order by all desc nulls last; + +-- set operations from duckdb +select * from data union all select * from data order by all; +select * from data union select * from data order by all; + +-- limit +select * from data order by all limit 2; + +-- precedence: if there's a column already named all, reference that, instead of expanding. +-- result should be 1, 2, 3, and not 3, 2, 1 +select * from values("z", 1), ("y", 2), ("x", 3) AS T(col1, all) order by all; + +-- shouldn't work in window functions +select name, dept, rank() over (partition by dept order by all) as rank +from values('Lisa', 'Sales', 10000, 35) as T(name, dept, salary, age); + +-- ORDER BY column can't reference an outer scope +SELECT all, (SELECT col2 FROM VALUES (2, 2) ORDER BY ALL) FROM VALUES (1) AS t1 (all) diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/order-by-nulls-ordering.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/order-by-nulls-ordering.sql new file mode 100644 index 000000000000..ad3977465c83 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/order-by-nulls-ordering.sql @@ -0,0 +1,88 @@ +-- Test sort operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +-- Q1. testing window functions with order by +create table spark_10747(col1 int, col2 int, col3 int) using parquet; + +-- Q2. insert to tables +INSERT INTO spark_10747 VALUES (6, 12, 10), (6, 11, 4), (6, 9, 10), (6, 15, 8), +(6, 15, 8), (6, 7, 4), (6, 7, 8), (6, 13, null), (6, 10, null); + +-- Q3. windowing with order by DESC NULLS LAST +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 desc nulls last, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2; + +-- Q4. windowing with order by DESC NULLS FIRST +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 desc nulls first, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2; + +-- Q5. windowing with order by ASC NULLS LAST +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 asc nulls last, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2; + +-- Q6. windowing with order by ASC NULLS FIRST +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 asc nulls first, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2; + +-- Q7. Regular query with ORDER BY ASC NULLS FIRST +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 ASC NULLS FIRST, COL2; + +-- Q8. Regular query with ORDER BY ASC NULLS LAST +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 NULLS LAST, COL2; + +-- Q9. Regular query with ORDER BY DESC NULLS FIRST +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS FIRST, COL2; + +-- Q10. Regular query with ORDER BY DESC NULLS LAST +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS LAST, COL2; + +-- drop the test table +drop table spark_10747; + +-- Q11. mix datatype for ORDER BY NULLS FIRST|LAST +create table spark_10747_mix( +col1 string, +col2 int, +col3 double, +col4 decimal(10,2), +col5 decimal(20,1)) +using parquet; + +-- Q12. Insert to the table +INSERT INTO spark_10747_mix VALUES +('b', 2, 1.0, 1.00, 10.0), +('d', 3, 2.0, 3.00, 0.0), +('c', 3, 2.0, 2.00, 15.1), +('d', 3, 0.0, 3.00, 1.0), +(null, 3, 0.0, 3.00, 1.0), +('d', 3, null, 4.00, 1.0), +('a', 1, 1.0, 1.00, null), +('c', 3, 2.0, 2.00, null); + +-- Q13. Regular query with 2 NULLS LAST columns +select * from spark_10747_mix order by col1 nulls last, col5 nulls last; + +-- Q14. Regular query with 2 NULLS FIRST columns +select * from spark_10747_mix order by col1 desc nulls first, col5 desc nulls first; + +-- Q15. Regular query with mixed NULLS FIRST|LAST +select * from spark_10747_mix order by col5 desc nulls first, col3 desc nulls last; + +-- drop the test table +drop table spark_10747_mix; + + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/order-by-ordinal.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/order-by-ordinal.sql new file mode 100644 index 000000000000..8d733e77fa8d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/order-by-ordinal.sql @@ -0,0 +1,36 @@ +-- order by and sort by ordinal positions + +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b); + +select * from data order by 1 desc; + +-- mix ordinal and column name +select * from data order by 1 desc, b desc; + +-- order by multiple ordinals +select * from data order by 1 desc, 2 desc; + +-- 1 + 0 is considered a constant (not an ordinal) and thus ignored +select * from data order by 1 + 0 desc, b desc; + +-- negative cases: ordinal position out of range +select * from data order by 0; +select * from data order by -1; +select * from data order by 3; + +-- sort by ordinal +select * from data sort by 1 desc; + +-- turn off order by ordinal +set spark.sql.orderByOrdinal=false; + +-- 0 is now a valid literal +select * from data order by 0; +select * from data sort by 0; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/order-by.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/order-by.sql new file mode 100644 index 000000000000..d67987bcb51c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/order-by.sql @@ -0,0 +1,34 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b); + +-- ORDER BY a column from a child's output +SELECT a FROM testData ORDER BY a; +SELECT a FROM testData GROUP BY a, b ORDER BY a; + +-- ORDER BY a column from an output below a child's one +SELECT b FROM testData WHERE a > 1 ORDER BY a; + +-- ORDER BY a column from grouping expressions list +SELECT b FROM testData GROUP BY a, b ORDER BY a; + +-- ORDER BY a nested column from an output below a child's one +SELECT col1 FROM VALUES (1, named_struct('f1', 1)) ORDER BY col2.f1; +SELECT col1 FROM VALUES (1, named_struct('f1', named_struct('f2', 1))) ORDER BY col2.f1.f2; + +-- ORDER BY column can't reference an outer scope +SELECT a, (SELECT b FROM testData GROUP BY b HAVING b > 1 ORDER BY a) FROM testData; + +-- Column resolution from the child's output takes the precedence over `ORDER BY ALL` +SELECT a, (SELECT b FROM VALUES (1, 2) AS innerTestData (all, b) ORDER BY ALL) FROM testData; + +-- ORDER BY with scalar subqueries +SELECT * FROM testData ORDER BY (SELECT a FROM testData ORDER BY b); +SELECT * FROM testData ORDER BY (SELECT * FROM testData ORDER BY (SELECT a FROM testData ORDER BY b)); + +-- Fails because correlation is not allowed in ORDER BY +SELECT * FROM testData ORDER BY (SELECT a FROM VALUES (1)); + +-- Clean up +DROP VIEW IF EXISTS testData; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/outer-join.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/outer-join.sql new file mode 100644 index 000000000000..a43b0eadf2b4 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/outer-join.sql @@ -0,0 +1,45 @@ +-- There are 2 dimensions we want to test +-- 1. run with broadcast hash join, sort merge join or shuffle hash join. +-- 2. run with whole-stage-codegen, operator codegen or no codegen. + +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=10485760 +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.forceApplyShuffledHashJoin=true + +--CONFIG_DIM2 spark.sql.codegen.wholeStage=true +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +-- SPARK-17099: Incorrect result when HAVING clause is added to group by query +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(-234), (145), (367), (975), (298) +as t1(int_col1); + +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES +(-769, -244), (-800, -409), (940, 86), (-507, 304), (-367, 158) +as t2(int_col0, int_col1); + +SELECT + (SUM(COALESCE(t1.int_col1, t2.int_col0))), + ((COALESCE(t1.int_col1, t2.int_col0)) * 2) +FROM t1 +RIGHT JOIN t2 + ON (t2.int_col0) = (t1.int_col1) +GROUP BY GREATEST(COALESCE(t2.int_col1, 109), COALESCE(t1.int_col1, -449)), + COALESCE(t1.int_col1, t2.int_col0) +HAVING (SUM(COALESCE(t1.int_col1, t2.int_col0))) + > ((COALESCE(t1.int_col1, t2.int_col0)) * 2); + + +-- SPARK-17120: Analyzer incorrectly optimizes plan to empty LocalRelation +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (97) as t1(int_col1); + +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (0) as t2(int_col1); + +SELECT * +FROM ( +SELECT + COALESCE(t2.int_col1, t1.int_col1) AS int_col + FROM t1 + LEFT JOIN t2 ON false +) t where (t.int_col) is not null; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/parse-query-correctness-old-behavior.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/parse-query-correctness-old-behavior.sql new file mode 100644 index 000000000000..bece56f575cb --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/parse-query-correctness-old-behavior.sql @@ -0,0 +1,2 @@ +--SET spark.sql.legacy.parseQueryWithoutEof=true +--IMPORT parse-query-correctness.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/parse-query-correctness.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/parse-query-correctness.sql new file mode 100644 index 000000000000..8884e51f052e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/parse-query-correctness.sql @@ -0,0 +1,286 @@ +-- This test suite checks the correctness of AbstractSqlParser.parseQuery + +-- SPARK-51113, view read path + +-- UNION + +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num + UNION + SELECT 2 + UNION + SELECT 3 + UNION + SELECT 4 +; +SELECT * FROM v1; + +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 + UNION + SELECT 2 + UNION + SELECT 3 + UNION + SELECT 4 +; +SELECT * FROM v1; + +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1, 'a' + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +; +SELECT * FROM v1; + +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num, 'a' + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +; +SELECT * FROM v1; + +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1, 'a' AS a + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +; +SELECT * FROM v1; + +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num, 'a' AS a + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +; +SELECT * FROM v1; + +-- EXCEPT + +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num + EXCEPT + SELECT 2 + EXCEPT + SELECT 1 + EXCEPT + SELECT 2 +; +SELECT * FROM v1; + +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 + EXCEPT + SELECT 2 + EXCEPT + SELECT 1 + EXCEPT + SELECT 2 +; +SELECT * FROM v1; + +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1, 'a' + EXCEPT + SELECT 2, 'b' + EXCEPT + SELECT 1, 'a' + EXCEPT + SELECT 2, 'b' +; +SELECT * FROM v1; + +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num, 'a' + EXCEPT + SELECT 2, 'b' + EXCEPT + SELECT 1, 'a' + EXCEPT + SELECT 2, 'b' +; +SELECT * FROM v1; + +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1, 'a' AS a + EXCEPT + SELECT 2, 'b' + EXCEPT + SELECT 1, 'a' + EXCEPT + SELECT 2, 'b' +; +SELECT * FROM v1; + +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num, 'a' AS a + EXCEPT + SELECT 2, 'b' + EXCEPT + SELECT 1, 'a' + EXCEPT + SELECT 2, 'b' +; +SELECT * FROM v1; + +-- INTERSECT + +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num + INTERSECT + SELECT 1 + INTERSECT + SELECT 2 + INTERSECT + SELECT 2 +; +SELECT * FROM v1; + +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 + INTERSECT + SELECT 1 + INTERSECT + SELECT 2 + INTERSECT + SELECT 2 +; +SELECT * FROM v1; + +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1, 'a' + INTERSECT + SELECT 1, 'a' + INTERSECT + SELECT 2, 'b' + INTERSECT + SELECT 2, 'b' +; +SELECT * FROM v1; + +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num, 'a' + INTERSECT + SELECT 1, 'a' + INTERSECT + SELECT 2, 'b' + INTERSECT + SELECT 2, 'b' +; +SELECT * FROM v1; + +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1, 'a' AS a + INTERSECT + SELECT 1, 'a' + INTERSECT + SELECT 2, 'b' + INTERSECT + SELECT 2, 'b' +; +SELECT * FROM v1; + +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num, 'a' AS a + INTERSECT + SELECT 1, 'a' + INTERSECT + SELECT 2, 'b' + INTERSECT + SELECT 2, 'b' +; +SELECT * FROM v1; + +DROP VIEW v1; + +-- Views with schema evolution would give incorrect result. +-- The last column would be named 'UNION' on read, but schema evolution +-- would allow the 'a' -> 'UNION' rename on read. + +CREATE OR REPLACE VIEW v1 WITH SCHEMA EVOLUTION AS + SELECT 1, 'a' + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +; +SELECT * FROM v1; + +CREATE OR REPLACE VIEW v1 WITH SCHEMA EVOLUTION AS + SELECT 1 AS num, 'a' + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +; +SELECT * FROM v1; + +CREATE OR REPLACE VIEW v1 WITH SCHEMA EVOLUTION AS + SELECT 1 AS num, 'a' AS a + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +; +SELECT * FROM v1; + +DROP VIEW v1; + +-- SPARK-51113, EXECUTE IMMEDIATE ... INTO + +DECLARE v1 INT; + +EXECUTE IMMEDIATE 'SELECT 1 UNION SELECT 2 UNION SELECT 3' INTO v1; +SELECT v1; + +EXECUTE IMMEDIATE 'SELECT 1 UNION SELECT 1 UNION SELECT 1' INTO v1; +SELECT v1; + +EXECUTE IMMEDIATE 'SELECT 1 EXCEPT SELECT 2 EXCEPT SELECT 3' INTO v1; +SELECT v1; + +EXECUTE IMMEDIATE 'SELECT 1 EXCEPT SELECT 1 EXCEPT SELECT 1' INTO v1; +SELECT v1; + +EXECUTE IMMEDIATE 'SELECT 1 INTERSECT SELECT 2 INTERSECT SELECT 3' INTO v1; +SELECT v1; + +EXECUTE IMMEDIATE 'SELECT 1 INTERSECT SELECT 1 INTERSECT SELECT 1' INTO v1; +SELECT v1; + +-- Garbage queries + +EXECUTE IMMEDIATE 'SELECT 1 JOIN SELECT 2' INTO v1; +SELECT v1; + +EXECUTE IMMEDIATE 'SELECT 1 VALUES (1)' INTO v1; +SELECT v1; + +EXECUTE IMMEDIATE 'SELECT 1 alias garbage garbage garbage' INTO v1; +SELECT v1; + +EXECUTE IMMEDIATE 'SELECT 1 WITH abc' INTO v1; +SELECT v1; + +DROP TEMPORARY VARIABLE v1; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/parse-schema-string.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/parse-schema-string.sql new file mode 100644 index 000000000000..c67d45139fd6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/parse-schema-string.sql @@ -0,0 +1,5 @@ +-- Use keywords as attribute names +select from_csv('1', 'create INT'); +select from_csv('1', 'cube INT'); +select from_json('{"create":1}', 'create INT'); +select from_json('{"cube":1}', 'cube INT'); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/percentiles.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/percentiles.sql new file mode 100644 index 000000000000..4b3e8708222a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/percentiles.sql @@ -0,0 +1,429 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW aggr AS SELECT * FROM VALUES +(0, 0), (0, 10), (0, 20), (0, 30), (0, 40), (1, 10), (1, 20), (2, 10), (2, 20), (2, 25), (2, 30), (3, 60), (4, null) +AS aggr(k, v); + +CREATE OR REPLACE TEMPORARY VIEW basic_pays AS SELECT * FROM VALUES +('Diane Murphy','Accounting',8435), +('Mary Patterson','Accounting',9998), +('Jeff Firrelli','Accounting',8992), +('William Patterson','Accounting',8870), +('Gerard Bondur','Accounting',11472), +('Anthony Bow','Accounting',6627), +('Leslie Jennings','IT',8113), +('Leslie Thompson','IT',5186), +('Julie Firrelli','Sales',9181), +('Steve Patterson','Sales',9441), +('Foon Yue Tseng','Sales',6660), +('George Vanauf','Sales',10563), +('Loui Bondur','SCM',10449), +('Gerard Hernandez','SCM',6949), +('Pamela Castillo','SCM',11303), +('Larry Bott','SCM',11798), +('Barry Jones','SCM',10586) +AS basic_pays(employee_name, department, salary); + +SELECT + percentile_cont(0.25) WITHIN GROUP (ORDER BY v), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr; + +SELECT + k, + percentile_cont(0.25) WITHIN GROUP (ORDER BY v), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr +GROUP BY k +ORDER BY k; + +SELECT + percentile_disc(0.25) WITHIN GROUP (ORDER BY v), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr; + +SELECT + k, + percentile_disc(0.25) WITHIN GROUP (ORDER BY v), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr +GROUP BY k +ORDER BY k; + +SELECT + median(v), + percentile(v, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY v) +FROM aggr; + +SELECT + round(v, 0) WITHIN GROUP (ORDER BY v) +FROM aggr; + +SELECT + round(v, 0) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY k) +FROM aggr; + +SELECT + percentile(v, 0.5) WITHIN GROUP (ORDER BY v) +FROM aggr; + +SELECT + percentile(v, 0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY k) +FROM aggr; + +SELECT + percentile_cont(DISTINCT 0.5) WITHIN GROUP (ORDER BY v) +FROM aggr; + +SELECT + percentile_cont(DISTINCT 0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY k) +FROM aggr; + +SELECT + percentile_cont() WITHIN GROUP (ORDER BY v) +FROM aggr; + +SELECT + percentile_cont() WITHIN GROUP (ORDER BY v) OVER (PARTITION BY k) +FROM aggr; + +SELECT + percentile_cont(0.5) +FROM aggr; + +SELECT + percentile_cont(0.5) OVER (PARTITION BY k) +FROM aggr; + +SELECT + percentile_cont(0.5) WITHIN GROUP (ORDER BY k, v) +FROM aggr; + +SELECT + percentile_cont(k, 0.5) WITHIN GROUP (ORDER BY v) +FROM aggr; + +SELECT + k, + median(v), + percentile(v, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY v) +FROM aggr +GROUP BY k +ORDER BY k; + +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department), + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department) +FROM basic_pays +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary), + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ORDER BY salary) +FROM basic_pays +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ORDER BY salary) +FROM basic_pays +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + median(salary) OVER (PARTITION BY department ORDER BY salary) +FROM basic_pays +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING), + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +FROM basic_pays +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +FROM basic_pays +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +FROM basic_pays +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department) +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + median(salary) OVER w, + percentile_cont(0.5) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.5) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.5) WITHIN GROUP (ORDER BY salary DESC) OVER w, + percentile_disc(0.5) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WHERE salary > 8900 +WINDOW w AS (PARTITION BY department) +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ORDER BY salary) +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ORDER BY salary) +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + median(salary) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ORDER BY salary) +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary; + +SELECT + employee_name, + department, + salary, + median(salary) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary; + +CREATE OR REPLACE TEMPORARY VIEW intervals AS SELECT * FROM VALUES +(0, INTERVAL '0' MONTH, INTERVAL '0' SECOND, INTERVAL '0' MINUTE), +(0, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(0, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(0, INTERVAL '30' MONTH, INTERVAL '30' SECOND, INTERVAL '30' MINUTE), +(0, INTERVAL '40' MONTH, INTERVAL '40' SECOND, INTERVAL '40' MINUTE), +(1, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(1, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(2, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(2, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(2, INTERVAL '25' MONTH, INTERVAL '25' SECOND, INTERVAL '25' MINUTE), +(2, INTERVAL '30' MONTH, INTERVAL '30' SECOND, INTERVAL '30' MINUTE), +(3, INTERVAL '60' MONTH, INTERVAL '60' SECOND, INTERVAL '60' MINUTE), +(4, null, null, null) +AS intervals(k, dt, ym, dt2); + +SELECT + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt), + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt DESC) +FROM intervals; + +SELECT + k, + percentile_cont(0.25) WITHIN GROUP (ORDER BY ym), + percentile_cont(0.25) WITHIN GROUP (ORDER BY ym DESC) +FROM intervals +GROUP BY k +ORDER BY k; + +SELECT + k, + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt2), + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt2 DESC) +FROM intervals +GROUP BY k +ORDER BY k; + +SELECT + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt), + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt DESC) +FROM intervals; + +SELECT + k, + percentile_disc(0.25) WITHIN GROUP (ORDER BY ym), + percentile_disc(0.25) WITHIN GROUP (ORDER BY ym DESC) +FROM intervals +GROUP BY k +ORDER BY k; + +SELECT + k, + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt2), + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt2 DESC) +FROM intervals +GROUP BY k +ORDER BY k; + +SELECT + median(dt), + percentile(dt, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY dt) +FROM intervals; + +SELECT + k, + median(ym), + percentile(ym, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY ym) +FROM intervals +GROUP BY k +ORDER BY k; + +SELECT + k, + median(dt2), + percentile(dt2, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY dt2) +FROM intervals +GROUP BY k +ORDER BY k; + +-- SPARK-44871: Fix percentile_disc behaviour +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0) AS v(a); + +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0), (1) AS v(a); + +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0), (1), (2) AS v(a); + +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0), (1), (2), (3), (4) AS v(a); + +SET spark.sql.legacy.percentileDiscCalculation = true; + +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0), (1), (2), (3), (4) AS v(a); + +SELECT + percentile_cont(b) WITHIN GROUP (ORDER BY a DESC) as p0 +FROM values (12, 0.25), (13, 0.25), (22, 0.25) as v(a, b); + +SET spark.sql.legacy.percentileDiscCalculation = false; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/pipe-operators.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/pipe-operators.sql new file mode 100644 index 000000000000..ce70cea0e6e0 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/pipe-operators.sql @@ -0,0 +1,1763 @@ +-- Prepare some test data. +-------------------------- +drop table if exists t; +create table t(x int, y string) using csv; +insert into t values (0, 'abc'), (1, 'def'); + +drop table if exists other; +create table other(a int, b int) using json; +insert into other values (1, 1), (1, 2), (2, 4); + +drop table if exists st; +create table st(x int, col struct) using parquet; +insert into st values (1, (2, 3)); + +create temporary view courseSales as select * from values + ("dotNET", 2012, 10000), + ("Java", 2012, 20000), + ("dotNET", 2012, 5000), + ("dotNET", 2013, 48000), + ("Java", 2013, 30000) + as courseSales(course, year, earnings); + +create temporary view courseEarnings as select * from values + ("dotNET", 15000, 48000, 22500), + ("Java", 20000, 30000, NULL) + as courseEarnings(course, `2012`, `2013`, `2014`); + +create temporary view courseEarningsAndSales as select * from values + ("dotNET", 15000, NULL, 48000, 1, 22500, 1), + ("Java", 20000, 1, 30000, 2, NULL, NULL) + as courseEarningsAndSales( + course, earnings2012, sales2012, earnings2013, sales2013, earnings2014, sales2014); + +create temporary view yearsWithComplexTypes as select * from values + (2012, array(1, 1), map('1', 1), struct(1, 'a')), + (2013, array(2, 2), map('2', 2), struct(2, 'b')) + as yearsWithComplexTypes(y, a, m, s); + +create temporary view join_test_t1 as select * from values (1) as grouping(a); +create temporary view join_test_t2 as select * from values (1) as grouping(a); +create temporary view join_test_t3 as select * from values (1) as grouping(a); +create temporary view join_test_empty_table as select a from join_test_t2 where false; + +create temporary view lateral_test_t1(c1, c2) + as values (0, 1), (1, 2); +create temporary view lateral_test_t2(c1, c2) + as values (0, 2), (0, 3); +create temporary view lateral_test_t3(c1, c2) + as values (0, array(0, 1)), (1, array(2)), (2, array()), (null, array(4)); +create temporary view lateral_test_t4(c1, c2) + as values (0, 1), (0, 2), (1, 1), (1, 3); + +create temporary view natural_join_test_t1 as select * from values + ("one", 1), ("two", 2), ("three", 3) as natural_join_test_t1(k, v1); + +create temporary view natural_join_test_t2 as select * from values + ("one", 1), ("two", 22), ("one", 5) as natural_join_test_t2(k, v2); + +create temporary view natural_join_test_t3 as select * from values + ("one", 4), ("two", 5), ("one", 6) as natural_join_test_t3(k, v3); + +create temporary view windowTestData as select * from values + (null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), + (1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), + (1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), + (2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), + (1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), + (2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), + (3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), + (null, null, null, null, null, null), + (3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) + AS testData(val, val_long, val_double, val_date, val_timestamp, cate); + +-- FROM operators: positive tests. +---------------------------------- + +-- FromClause alone. +from t; + +-- Table alone. +table t; + +-- Selecting from a constant. +from t +|> select 1 as x; + +-- Selecting using a table alias. +from t as t_alias +|> select t_alias.x; + +-- Selecting using a table alias. +from t as t_alias +|> select t_alias.x as tx, t_alias.y as ty +|> where ty = 'def' +|> select tx; + +-- Selecting from multiple relations. +from t, other +|> select t.x + other.a as z; + +-- Selecting from multiple relations with join. +from t join other on (t.x = other.a) +|> select t.x + other.a as z; + +-- Selecting from lateral view. +from t lateral view explode(array(100, 101)) as ly +|> select t.x + ly as z; + +-- Selecting struct fields. +from st +|> select col.i1; + +-- Selecting struct fields using a table alias. +from st as st_alias +|> select st_alias.col.i1; + +-- Selecting from a VALUES list. +from values (0), (1) tab(col) +|> select col as x; + +-- FROM operators: negative tests. +---------------------------------- + +-- It is not possible to use the FROM operator accepting an input relation. +from t +|> from t; + +-- SELECT operators: positive tests. +--------------------------------------- + +-- Selecting a constant. +table t +|> select 1 as x; + +-- Selecting attributes. +table t +|> select x, y; + +-- Chained pipe SELECT operators. +table t +|> select x, y +|> select x + length(y) as z; + +-- Using the VALUES list as the source relation. +values (0), (1) tab(col) +|> select col * 2 as result; + +-- Using a table subquery as the source relation. +(select * from t union all select * from t) +|> select x + length(y) as result; + +-- Enclosing the result of a pipe SELECT operation in a table subquery. +(table t + |> select x, y + |> select x) +union all +select x from t where x < 1; + +-- Selecting struct fields. +(select col from st) +|> select col.i1; + +table st +|> select st.col.i1; + +-- Expression subqueries in the pipe operator SELECT list. +table t +|> select (select a from other where x = a limit 1) as result; + +-- Pipe operator SELECT inside expression subqueries. +select (values (0) tab(col) |> select col) as result; + +-- Aggregations are allowed within expression subqueries in the pipe operator SELECT list as long as +-- no aggregate functions exist in the top-level select list. +table t +|> select (select any_value(a) from other where x = a limit 1) as result; + +-- Lateral column aliases in the pipe operator SELECT list. +table t +|> select x + length(x) as z, z + 1 as plus_one; + +-- Window functions are allowed in the pipe operator SELECT list. +table t +|> select first_value(x) over (partition by y) as result; + +select 1 x, 2 y, 3 z +|> select 1 + sum(x) over (), + avg(y) over (), + x, + avg(x+1) over (partition by y order by z) AS a2 +|> select a2; + +table t +|> select x, count(*) over () +|> select x; + +-- DISTINCT is supported. +table t +|> select distinct x, y; + +-- SELECT * is supported. +table t +|> select *; + +table t +|> select * except (y); + +-- Hints are supported. +table t +|> select /*+ repartition(3) */ *; + +table t +|> select /*+ repartition(3) */ distinct x; + +table t +|> select /*+ repartition(3) */ all x; + +-- SELECT operators: negative tests. +--------------------------------------- + +-- Aggregate functions are not allowed in the pipe operator SELECT list. +table t +|> select sum(x) as result; + +table t +|> select y, length(y) + sum(x) as result; + +from t +|> select sum(x); + +from t as t_alias +|> select y, sum(x); + +from t as t_alias +|> select y, sum(x) group by y; + +-- EXTEND operators: positive tests. +------------------------------------ + +-- Extending with a constant. +table t +|> extend 1 as z; + +-- Extending without an explicit alias. +table t +|> extend 1; + +-- Extending with an attribute. +table t +|> extend x as z; + +-- Extending with an expression. +table t +|> extend x + length(y) as z; + +-- Extending two times. +table t +|> extend x + length(y) as z, x + 1 as zz; + +-- Extending two times in sequence. +table t +|> extend x + length(y) as z +|> extend z + 1 as zz; + +-- Extending with a struct field. +select col from st +|> extend col.i1 as z; + +-- Extending with a subquery. +table t +|> extend (select a from other where x = a limit 1) as z; + +-- Extending with a correlated reference. +table t +|> where exists ( + table other + |> extend t.x + |> select * except (a, b)); + +-- Extending with a column name that already exists in the input relation. +table t +|> extend 1 as x; + +-- Window functions are allowed in the pipe operator EXTEND list. +table t +|> extend first_value(x) over (partition by y) as result; + +-- Lateral column aliases in the pipe operator EXTEND list. +table t +|> extend x + length(y) as z, z + 1 as plus_one; + +-- EXTEND operators: negative tests. +------------------------------------ + +-- Aggregations are not allowed. +table t +|> extend sum(x) as z; + +-- DISTINCT is not supported. +table t +|> extend distinct x as z; + +-- EXTEND * is not supported. +table t +|> extend *; + +-- SET operators: positive tests. +--------------------------------- + +-- Setting with a constant. +-- The indicated column is not the last column in the table, and the SET operator will replace it +-- with the new value in its existing position. +table t +|> set x = 1; + +-- Setting with an attribute. +table t +|> set y = x; + +-- Setting with an expression. +table t +|> extend 1 as z +|> set z = x + length(y); + +-- Setting two times. +table t +|> extend 1 as z +|> extend 2 as zz +|> set z = x + length(y), zz = x + 1; + +table other +|> extend 3 as c +|> set a = b, b = c; + +-- Setting two times with a lateral reference. +table t +|> extend 1 as z +|> extend 2 as zz +|> set z = x + length(y), zz = z + 1; + +-- Setting two times in sequence. +table t +|> extend 1 as z +|> set z = x + length(y) +|> set z = z + 1; + +-- SET assignments with duplicate keys. This is supported, and we can update the column as we go. +table t +|> extend 1 as z +|> set z = x + length(y), z = z + 1; + +-- Setting with a struct field. +select col from st +|> extend 1 as z +|> set z = col.i1; + +-- Setting with a subquery. +table t +|> set y = (select a from other where x = a limit 1); + +-- Setting with a backquoted column name with a dot inside. +table t +|> extend 1 as `x.y.z` +|> set `x.y.z` = x + length(y); + +-- Window functions are allowed in the pipe operator SET list. +table t +|> extend 1 as z +|> set z = first_value(x) over (partition by y); + +-- Any prior table aliases remain visible after a SET operator. +values (0), (1) lhs(a) +|> inner join values (1), (2) rhs(a) using (a) +|> extend lhs.a + rhs.a as z1 +|> extend lhs.a - rhs.a as z2 +|> drop z1 +|> where z2 = 0 +|> order by lhs.a, rhs.a, z2 +|> set z2 = 4 +|> limit 2 +|> select lhs.a, rhs.a, z2; + +-- SET operators: negative tests. +--------------------------------- + +-- SET with a column name that does not exist in the input relation. +table t +|> set z = 1; + +-- SET with an alias. +table t +|> set x = 1 as z; + +-- Setting nested fields in structs is not supported. +select col from st +|> set col.i1 = 42; + +-- DROP operators: positive tests. +------------------------------------ + +-- Dropping a column. +table t +|> drop y; + +-- Dropping two times. +select 1 as x, 2 as y, 3 as z +|> drop z, y; + +-- Dropping two times in sequence. +select 1 as x, 2 as y, 3 as z +|> drop z +|> drop y; + +-- Dropping all columns in the input relation. +select x from t +|> drop x; + +-- Dropping a backquoted column name with a dot inside. +table t +|> extend 1 as `x.y.z` +|> drop `x.y.z`; + +-- DROP operators: negative tests. +---------------------------------- + +-- Dropping a column that is not present in the input relation. +table t +|> drop z; + +-- Attempting to drop a struct field. +table st +|> drop col.i1; + +table st +|> drop `col.i1`; + +-- Duplicate fields in the drop list. +select 1 as x, 2 as y, 3 as z +|> drop z, y, z; + +-- AS operators: positive tests. +-------------------------------- + +-- Renaming a table. +table t +|> as u +|> select u.x, u.y; + +-- Renaming an input relation that is not a table. +select 1 as x, 2 as y +|> as u +|> select u.x, u.y; + +-- Renaming as a backquoted name including a period. +table t +|> as `u.v` +|> select `u.v`.x, `u.v`.y; + +-- Renaming two times. +table t +|> as u +|> as v +|> select v.x, v.y; + +-- Filtering by referring to the table or table subquery alias. +table t +|> as u +|> where u.x = 1; + +-- AS operators: negative tests. +-------------------------------- + +-- Multiple aliases are not supported. +table t +|> as u, v; + +-- Expressions are not supported. +table t +|> as 1 + 2; + +-- Renaming as an invalid name. +table t +|> as u-v; + +table t +|> as u@v; + +table t +|> as u#######v; + +-- WHERE operators: positive tests. +----------------------------------- + +-- Filtering with a constant predicate. +table t +|> where true; + +-- Filtering with a predicate based on attributes from the input relation. +table t +|> where x + length(y) < 4; + +-- Two consecutive filters are allowed. +table t +|> where x + length(y) < 4 +|> where x + length(y) < 3; + +-- It is possible to use the WHERE operator instead of the HAVING clause when processing the result +-- of aggregations. For example, this WHERE operator is equivalent to the normal SQL "HAVING x = 1". +(select x, sum(length(y)) as sum_len from t group by x) +|> where x = 1; + +-- Filtering by referring to the table or table subquery alias. +table t +|> where t.x = 1; + +table t +|> where spark_catalog.default.t.x = 1; + +-- Filtering using struct fields. +(select col from st) +|> where col.i1 = 1; + +table st +|> where st.col.i1 = 2; + +-- Expression subqueries in the WHERE clause. +table t +|> where exists (select a from other where x = a limit 1); + +-- Aggregations are allowed within expression subqueries in the pipe operator WHERE clause as long +-- no aggregate functions exist in the top-level expression predicate. +table t +|> where (select any_value(a) from other where x = a limit 1) = 1; + +-- WHERE operators: negative tests. +----------------------------------- + +-- Aggregate functions are not allowed in the top-level WHERE predicate. +-- (Note: to implement this behavior, perform the aggregation first separately and then add a +-- pipe-operator WHERE clause referring to the result of aggregate expression(s) therein). +table t +|> where sum(x) = 1; + +table t +|> where y = 'abc' or length(y) + sum(x) = 1; + +-- Window functions are not allowed in the WHERE clause (pipe operators or otherwise). +-- (Note: to implement this behavior, perform the window function first separately in a SELECT +-- clause and then add a pipe-operator WHERE clause referring to the result of the window function +-- expression(s) therein). +table t +|> where sum(x) over (partition by y) = 1; + +table t +|> where sum(x) over w = 1 + window w as (partition by y); + +select * from t where sum(x) over (partition by y) = 1; + +-- Pipe operators may only refer to attributes produced as output from the directly-preceding +-- pipe operator, not from earlier ones. +table t +|> select x, length(y) as z +|> where x + length(y) < 4; + +table t +|> select x, length(y) as z +|> limit 1000 +|> where x + length(y) < 4; + +table t +|> select x, length(y) as z +|> limit 1000 offset 1 +|> where x + length(y) < 4; + +table t +|> select x, length(y) as z +|> order by x, y +|> where x + length(y) < 4; + +-- If the WHERE clause wants to filter rows produced by an aggregation, it is not valid to try to +-- refer to the aggregate functions directly; it is necessary to use aliases instead. +(select x, sum(length(y)) as sum_len from t group by x) +|> where sum(length(y)) = 3; + +-- Pivot and unpivot operators: positive tests. +----------------------------------------------- + +table courseSales +|> select `year`, course, earnings +|> pivot ( + sum(earnings) + for course in ('dotNET', 'Java') + ); + +table courseSales +|> select `year` as y, course as c, earnings as e +|> pivot ( + sum(e) as s, avg(e) as a + for y in (2012 as firstYear, 2013 as secondYear) + ); + +-- Pivot on multiple pivot columns with aggregate columns of complex data types. +select course, `year`, y, a +from courseSales +join yearsWithComplexTypes on `year` = y +|> pivot ( + max(a) + for (y, course) in ((2012, 'dotNET'), (2013, 'Java')) + ); + +-- Pivot on pivot column of struct type. +select earnings, `year`, s +from courseSales +join yearsWithComplexTypes on `year` = y +|> pivot ( + sum(earnings) + for s in ((1, 'a'), (2, 'b')) + ); + +table courseEarnings +|> unpivot ( + earningsYear for `year` in (`2012`, `2013`, `2014`) + ); + +table courseEarnings +|> unpivot include nulls ( + earningsYear for `year` in (`2012`, `2013`, `2014`) + ); + +table courseEarningsAndSales +|> unpivot include nulls ( + (earnings, sales) for `year` in ( + (earnings2012, sales2012) as `2012`, + (earnings2013, sales2013) as `2013`, + (earnings2014, sales2014) as `2014`) + ); + +-- Pivot and unpivot operators: negative tests. +----------------------------------------------- + +-- The PIVOT operator refers to a column 'year' is not available in the input relation. +table courseSales +|> select course, earnings +|> pivot ( + sum(earnings) + for `year` in (2012, 2013) + ); + +-- Non-literal PIVOT values are not supported. +table courseSales +|> pivot ( + sum(earnings) + for `year` in (course, 2013) + ); + +-- The PIVOT and UNPIVOT clauses are mutually exclusive. +table courseSales +|> select course, earnings +|> pivot ( + sum(earnings) + for `year` in (2012, 2013) + ) + unpivot ( + earningsYear for `year` in (`2012`, `2013`, `2014`) + ); + +table courseSales +|> select course, earnings +|> unpivot ( + earningsYear for `year` in (`2012`, `2013`, `2014`) + ) + pivot ( + sum(earnings) + for `year` in (2012, 2013) + ); + +-- Multiple PIVOT and/or UNPIVOT clauses are not supported in the same pipe operator. +table courseSales +|> select course, earnings +|> pivot ( + sum(earnings) + for `year` in (2012, 2013) + ) + pivot ( + sum(earnings) + for `year` in (2012, 2013) + ); + +table courseSales +|> select course, earnings +|> unpivot ( + earningsYear for `year` in (`2012`, `2013`, `2014`) + ) + unpivot ( + earningsYear for `year` in (`2012`, `2013`, `2014`) + ) + pivot ( + sum(earnings) + for `year` in (2012, 2013) + ); + +-- Sampling operators: positive tests. +-------------------------------------- + +-- We will use the REPEATABLE clause and/or adjust the sampling options to either remove no rows or +-- all rows to help keep the tests deterministic. +table t +|> tablesample (100 percent) repeatable (0); + +table t +|> tablesample (2 rows) repeatable (0); + +table t +|> tablesample (bucket 1 out of 1) repeatable (0); + +table t +|> tablesample (100 percent) repeatable (0) +|> tablesample (5 rows) repeatable (0) +|> tablesample (bucket 1 out of 1) repeatable (0); + +-- Sampling operators: negative tests. +-------------------------------------- + +-- The sampling method is required. +table t +|> tablesample (); + +-- Negative sampling options are not supported. +table t +|> tablesample (-100 percent) repeatable (0); + +table t +|> tablesample (-5 rows); + +-- The sampling method may not refer to attribute names from the input relation. +table t +|> tablesample (x rows); + +-- The bucket number is invalid. +table t +|> tablesample (bucket 2 out of 1); + +-- Byte literals are not supported. +table t +|> tablesample (200b) repeatable (0); + +-- Invalid byte literal syntax. +table t +|> tablesample (200) repeatable (0); + +-- JOIN operators: positive tests. +---------------------------------- + +table join_test_t1 +|> inner join join_test_empty_table; + +table join_test_t1 +|> cross join join_test_empty_table; + +table join_test_t1 +|> left outer join join_test_empty_table; + +table join_test_t1 +|> right outer join join_test_empty_table; + +table join_test_t1 +|> full outer join join_test_empty_table using (a); + +table join_test_t1 +|> full outer join join_test_empty_table on (join_test_t1.a = join_test_empty_table.a); + +table join_test_t1 +|> left semi join join_test_empty_table; + +table join_test_t1 +|> left anti join join_test_empty_table; + +select * from join_test_t1 where true +|> inner join join_test_empty_table; + +select 1 as x, 2 as y +|> inner join (select 1 as x, 4 as y) using (x); + +table join_test_t1 +|> inner join (join_test_t2 jt2 inner join join_test_t3 jt3 using (a)) using (a) +|> select a, join_test_t1.a, jt2.a, jt3.a; + +table join_test_t1 +|> inner join join_test_t2 tablesample (100 percent) repeatable (0) jt2 using (a); + +table join_test_t1 +|> inner join (select 1 as a) tablesample (100 percent) repeatable (0) jt2 using (a); + +table join_test_t1 +|> join join_test_t1 using (a); + +-- Lateral joins. +table lateral_test_t1 +|> join lateral (select c1); + +table lateral_test_t1 +|> join lateral (select c1 from lateral_test_t2); + +table lateral_test_t1 +|> join lateral (select lateral_test_t1.c1 from lateral_test_t2); + +table lateral_test_t1 +|> join lateral (select lateral_test_t1.c1 + t2.c1 from lateral_test_t2 t2); + +table lateral_test_t1 +|> join lateral (select *); + +table lateral_test_t1 +|> join lateral (select * from lateral_test_t2); + +table lateral_test_t1 +|> join lateral (select lateral_test_t1.* from lateral_test_t2); + +table lateral_test_t1 +|> join lateral (select lateral_test_t1.*, t2.* from lateral_test_t2 t2); + +table lateral_test_t1 +|> join lateral_test_t2 +|> join lateral (select lateral_test_t1.c2 + lateral_test_t2.c2); + +-- Natural joins. +table natural_join_test_t1 +|> natural join natural_join_test_t2 +|> where k = "one"; + +table natural_join_test_t1 +|> natural join natural_join_test_t2 nt2 +|> select natural_join_test_t1.*; + +table natural_join_test_t1 +|> natural join natural_join_test_t2 nt2 +|> natural join natural_join_test_t3 nt3 +|> select natural_join_test_t1.*, nt2.*, nt3.*; + +-- JOIN operators: negative tests. +---------------------------------- + +-- Multiple joins within the same pipe operator are not supported without parentheses. +table join_test_t1 +|> inner join join_test_empty_table + inner join join_test_empty_table; + +-- The join pipe operator can only refer to column names from the previous relation. +table join_test_t1 +|> select 1 + 2 as result +|> full outer join join_test_empty_table on (join_test_t1.a = join_test_empty_table.a); + +-- The table from the pipe input is not visible as a table name in the right side. +table join_test_t1 jt +|> cross join (select * from jt); + +-- Set operations: positive tests. +---------------------------------- + +-- Union all. +table t +|> union all table t; + +-- Union distinct. +table t +|> union table t; + +-- Union all with a table subquery. +(select * from t) +|> union all table t; + +-- Union distinct with a table subquery. +(select * from t) +|> union table t; + +-- Union all with a VALUES list. +values (0, 'abc') tab(x, y) +|> union all table t; + +-- Union distinct with a VALUES list. +-- The |> WHERE operator applies to the result of the |> UNION operator, not to the "table t" input. +values (2, 'xyz') tab(x, y) +|> union table t +|> where x = 0; + +-- Union distinct with a VALUES list. +-- The |> DROP operator applies to the result of the |> UNION operator, not to the "table t" input. +values (2, 'xyz') tab(x, y) +|> union table t +|> drop x; + +-- Union all with a table subquery on both the source and target sides. +(select * from t) +|> union all (select * from t); + +-- Except all. +table t +|> except all table t; + +-- Except distinct. +table t +|> except table t; + +-- Intersect all. +table t +|> intersect all table t; + +-- Intersect distinct. +table t +|> intersect table t; + +-- Minus all. +table t +|> minus all table t; + +-- Minus distinct. +table t +|> minus table t; + +-- Set operations: negative tests. +---------------------------------- + +-- The UNION operator requires the same number of columns in the input relations. +table t +|> select x +|> union all table t; + +-- The UNION operator requires the column types to be compatible. +table t +|> union all table st; + +-- Sorting and repartitioning operators: positive tests. +-------------------------------------------------------- + +-- Order by. +table t +|> order by x; + +-- Order by with a table subquery. +(select * from t) +|> order by x; + +-- Order by with a VALUES list. +values (0, 'abc') tab(x, y) +|> order by x; + +-- Limit. +table t +|> order by x +|> limit 1; + +-- Limit with offset. +table t +|> where x = 1 +|> select y +|> limit 2 offset 1; + +-- Offset is allowed without limit. +table t +|> where x = 1 +|> select y +|> offset 1; + +-- LIMIT ALL and OFFSET 0 are equivalent to no LIMIT or OFFSET clause, respectively. +table t +|> limit all offset 0; + +-- Distribute by. +table t +|> distribute by x; + +-- Cluster by. +table t +|> cluster by x; + +-- Sort and distribute by. +table t +|> sort by x distribute by x; + +-- It is possible to apply a final ORDER BY clause on the result of a query containing pipe +-- operators. +table t +|> order by x desc +order by y; + +-- Sorting and repartitioning operators: negative tests. +-------------------------------------------------------- + +-- Multiple order by clauses are not supported in the same pipe operator. +-- We add an extra "ORDER BY y" clause at the end in this test to show that the "ORDER BY x + y" +-- clause was consumed end the of the final query, not as part of the pipe operator. +table t +|> order by x desc order by x + y +order by y; + +-- The ORDER BY clause may only refer to column names from the previous input relation. +table t +|> select 1 + 2 as result +|> order by x; + +-- The DISTRIBUTE BY clause may only refer to column names from the previous input relation. +table t +|> select 1 + 2 as result +|> distribute by x; + +-- Combinations of multiple ordering and limit clauses are not supported. +table t +|> order by x limit 1; + +-- ORDER BY and SORT BY are not supported at the same time. +table t +|> order by x sort by x; + +-- Aggregation operators: positive tests. +----------------------------------------- + +-- Basic aggregation with a GROUP BY clause. The resulting table contains all the attributes from +-- the grouping keys followed by all the attributes from the aggregate functions, in order. +table other +|> aggregate sum(b) as result group by a; + +-- Basic aggregation with a GROUP BY clause, followed by a SELECT of just the aggregate function. +-- This restricts the output attributes to just the aggregate function. +table other +|> aggregate sum(b) as result group by a +|> select result; + +-- Basic aggregation with a GROUP BY clause, followed by a SELECT of just the grouping expression. +-- This restricts the output attributes to just the grouping expression. Note that we must use an +-- alias for the grouping expression to refer to it in the SELECT clause. +table other +|> aggregate sum(b) group by a + 1 as gkey +|> select gkey; + +-- Basic aggregation on a constant table. +select 1 as x, 2 as y +|> aggregate group by x, y; + +-- Basic aggregation with group by ordinals. +select 3 as x, 4 as y +|> aggregate group by 1, 2; + +values (3, 4) as tab(x, y) +|> aggregate sum(y) group by 1; + +values (3, 4), (5, 4) as tab(x, y) +|> aggregate sum(y) group by 1; + +select 3 as x, 4 as y +|> aggregate sum(y) group by 1, 1; + +select 1 as `1`, 2 as `2` +|> aggregate sum(`2`) group by `1`; + +select 3 as x, 4 as y +|> aggregate sum(y) group by 2; + +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 2; + +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 3; + +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 2, 3; + +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 1, 2, 3; + +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by x, 2, 3; + +-- Basic table aggregation. +table t +|> aggregate sum(x); + +-- Basic table aggregation with an alias. +table t +|> aggregate sum(x) + 1 as result_plus_one; + +-- Grouping with no aggregate functions. +table other +|> aggregate group by a +|> where a = 1; + +-- Group by an expression on columns, all of which are already grouped. +select 1 as x, 2 as y, 3 as z +|> aggregate group by x, y, x + y as z; + +-- Group by an expression on columns, some of which (y) aren't already grouped. +select 1 as x, 2 as y, 3 as z +|> aggregate group by x as z, x + y as z; + +-- We get an output column for each item in GROUP BY, even when they are duplicate expressions. +select 1 as x, 2 as y, named_struct('z', 3) as st +|> aggregate group by x, y, x, x, st.z, (st).z, 1 + x, 2 + x; + +-- Chained aggregates. +select 1 x, 2 y, 3 z +|> aggregate sum(z) z group by x, y +|> aggregate avg(z) z group by x +|> aggregate count(distinct z) c; + +-- Ambiguous name from duplicate GROUP BY item. This is generally allowed. +select 1 x, 3 z +|> aggregate count(*) group by x, z, x +|> select x; + +-- Aggregate expressions may contain a mix of aggregate functions and grouping expressions. +table other +|> aggregate a + count(b) group by a; + +-- Aggregation operators: negative tests. +----------------------------------------- + +-- All aggregate expressions must contain at least one aggregate function. +table other +|> aggregate a group by a; + +-- GROUP BY ALL is not currently supported. +select 3 as x, 4 as y +|> aggregate group by all; + +-- GROUP BY ROLLUP is not supported yet. +table courseSales +|> aggregate sum(earnings) group by rollup(course, `year`) +|> where course = 'dotNET' and `year` = '2013'; + +-- GROUP BY CUBE is not supported yet. +table courseSales +|> aggregate sum(earnings) group by cube(course, `year`) +|> where course = 'dotNET' and `year` = '2013'; + +-- GROUPING SETS is not supported yet. +table courseSales +|> aggregate sum(earnings) group by course, `year` grouping sets(course, `year`) +|> where course = 'dotNET' and `year` = '2013'; + +-- GROUPING/GROUPING_ID is not supported yet. +table courseSales +|> aggregate sum(earnings), grouping(course) + 1 + group by course +|> where course = 'dotNET' and `year` = '2013'; + +-- GROUPING/GROUPING_ID is not supported yet. +table courseSales +|> aggregate sum(earnings), grouping_id(course) + group by course +|> where course = 'dotNET' and `year` = '2013'; + +-- GROUP BY () is not valid syntax. +select 1 as x, 2 as y +|> aggregate group by (); + +-- Non-aggregate expressions are not allowed in place of aggregate functions. +table other +|> aggregate a; + +-- Using aggregate functions without the AGGREGATE keyword is not allowed. +table other +|> select sum(a) as result; + +-- The AGGREGATE keyword requires a GROUP BY clause and/or aggregation function(s). +table other +|> aggregate; + +-- The AGGREGATE GROUP BY list cannot be empty. +table other +|> aggregate group by; + +-- The AGGREGATE keyword is required to perform grouping. +table other +|> group by a; + +-- Window functions are not allowed in the AGGREGATE expression list. +table other +|> aggregate sum(a) over () group by b; + +-- Ambiguous name from AGGREGATE list vs GROUP BY. +select 1 x, 2 y, 3 z +|> aggregate count(*) AS c, sum(x) AS x group by x +|> where c = 1 +|> where x = 1; + +-- WINDOW operators (within SELECT): positive tests. +--------------------------------------------------- + +-- SELECT with a WINDOW clause. +table windowTestData +|> select cate, sum(val) over w + window w as (partition by cate order by val); + +-- SELECT with RANGE BETWEEN as part of the window definition. +table windowTestData +|> select cate, sum(val) over w + window w as (order by val_timestamp range between unbounded preceding and current row); + +-- SELECT with a WINDOW clause not being referred in the SELECT list. +table windowTestData +|> select cate, val + window w as (partition by cate order by val); + +-- multiple SELECT clauses, each with a WINDOW clause (with the same window definition names). +table windowTestData +|> select cate, val, sum(val) over w as sum_val + window w as (partition by cate) +|> select cate, val, sum_val, first_value(cate) over w + window w as (order by val); + +-- SELECT with a WINDOW clause for multiple window definitions. +table windowTestData +|> select cate, val, sum(val) over w1, first_value(cate) over w2 + window w1 as (partition by cate), w2 as (order by val); + +-- SELECT with a WINDOW clause for multiple window functions over one window definition +table windowTestData +|> select cate, val, sum(val) over w, first_value(val) over w + window w1 as (partition by cate order by val); + +-- SELECT with a WINDOW clause, using struct fields. +(select col from st) +|> select col.i1, sum(col.i2) over w + window w as (partition by col.i1 order by col.i2); + +table st +|> select st.col.i1, sum(st.col.i2) over w + window w as (partition by st.col.i1 order by st.col.i2); + +table st +|> select spark_catalog.default.st.col.i1, sum(spark_catalog.default.st.col.i2) over w + window w as (partition by spark_catalog.default.st.col.i1 order by spark_catalog.default.st.col.i2); + +-- SELECT with one WINDOW definition shadowing a column name. +table windowTestData +|> select cate, sum(val) over val + window val as (partition by cate order by val); + +-- WINDOW operators (within SELECT): negative tests. +--------------------------------------------------- + +-- WINDOW without definition is not allowed in the pipe operator SELECT clause. +table windowTestData +|> select cate, sum(val) over w; + +-- Multiple WINDOW clauses are not supported in the pipe operator SELECT clause. +table windowTestData +|> select cate, val, sum(val) over w1, first_value(cate) over w2 + window w1 as (partition by cate) + window w2 as (order by val); + +-- WINDOW definition cannot be referred across different pipe operator SELECT clauses. +table windowTestData +|> select cate, val, sum(val) over w as sum_val + window w as (partition by cate order by val) +|> select cate, val, sum_val, first_value(cate) over w; + +table windowTestData +|> select cate, val, first_value(cate) over w as first_val +|> select cate, val, sum(val) over w as sum_val + window w as (order by val); + +-- Exercise SQL compilation using a subset of TPC-DS table schemas. +------------------------------------------------------------------- + +-- Q1 +with customer_total_return as +(select + sr_customer_sk as ctr_customer_sk, + sr_store_sk as ctr_store_sk, + sum(sr_return_amt) as ctr_total_return + from store_returns, date_dim + where sr_returned_date_sk = d_date_sk and d_year = 2000 + group by sr_customer_sk, sr_store_sk) +select c_customer_id +from customer_total_return ctr1, store, customer +where ctr1.ctr_total_return > + (select avg(ctr_total_return) * 1.2 + from customer_total_return ctr2 + where ctr1.ctr_store_sk = ctr2.ctr_store_sk) + and s_store_sk = ctr1.ctr_store_sk + and s_state = 'tn' + and ctr1.ctr_customer_sk = c_customer_sk +order by c_customer_id +limit 100; + +with customer_total_return as + (from store_returns + |> join date_dim + |> where sr_returned_date_sk = d_date_sk and d_year = 2000 + |> aggregate sum(sr_return_amt) as ctr_total_return + group by sr_customer_sk as ctr_customer_sk, sr_store_sk as ctr_store_sk) +from customer_total_return ctr1 +|> join store +|> join customer +|> where ctr1.ctr_total_return > + (table customer_total_return + |> as ctr2 + |> where ctr1.ctr_store_sk = ctr2.ctr_store_sk + |> aggregate avg(ctr_total_return) * 1.2) + and s_store_sk = ctr1.ctr_store_sk + and s_state = 'tn' + and ctr1.ctr_customer_sk = c_customer_sk +|> order by c_customer_id +|> limit 100 +|> select c_customer_id; + +-- Q2 +with wscs as +( select + sold_date_sk, + sales_price + from (select + ws_sold_date_sk sold_date_sk, + ws_ext_sales_price sales_price + from web_sales) x + union all + (select + cs_sold_date_sk sold_date_sk, + cs_ext_sales_price sales_price + from catalog_sales)), + wswscs as + ( select + d_week_seq, + sum(case when (d_day_name = 'sunday') + then sales_price + else null end) + sun_sales, + sum(case when (d_day_name = 'monday') + then sales_price + else null end) + mon_sales, + sum(case when (d_day_name = 'tuesday') + then sales_price + else null end) + tue_sales, + sum(case when (d_day_name = 'wednesday') + then sales_price + else null end) + wed_sales, + sum(case when (d_day_name = 'thursday') + then sales_price + else null end) + thu_sales, + sum(case when (d_day_name = 'friday') + then sales_price + else null end) + fri_sales, + sum(case when (d_day_name = 'saturday') + then sales_price + else null end) + sat_sales + from wscs, date_dim + where d_date_sk = sold_date_sk + group by d_week_seq) +select + d_week_seq1, + round(sun_sales1 / sun_sales2, 2), + round(mon_sales1 / mon_sales2, 2), + round(tue_sales1 / tue_sales2, 2), + round(wed_sales1 / wed_sales2, 2), + round(thu_sales1 / thu_sales2, 2), + round(fri_sales1 / fri_sales2, 2), + round(sat_sales1 / sat_sales2, 2) +from + (select + wswscs.d_week_seq d_week_seq1, + sun_sales sun_sales1, + mon_sales mon_sales1, + tue_sales tue_sales1, + wed_sales wed_sales1, + thu_sales thu_sales1, + fri_sales fri_sales1, + sat_sales sat_sales1 + from wswscs, date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and d_year = 2001) y, + (select + wswscs.d_week_seq d_week_seq2, + sun_sales sun_sales2, + mon_sales mon_sales2, + tue_sales tue_sales2, + wed_sales wed_sales2, + thu_sales thu_sales2, + fri_sales fri_sales2, + sat_sales sat_sales2 + from wswscs, date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and d_year = 2001 + 1) z +where d_week_seq1 = d_week_seq2 - 53 +order by d_week_seq1; + +with wscs as + (table web_sales + |> select + ws_sold_date_sk sold_date_sk, + ws_ext_sales_price sales_price + |> as x + |> union all ( + table catalog_sales + |> select + cs_sold_date_sk sold_date_sk, + cs_ext_sales_price sales_price) + |> select + sold_date_sk, + sales_price), +wswscs as + (table wscs + |> join date_dim + |> where d_date_sk = sold_date_sk + |> aggregate + sum(case when (d_day_name = 'sunday') + then sales_price + else null end) + sun_sales, + sum(case when (d_day_name = 'monday') + then sales_price + else null end) + mon_sales, + sum(case when (d_day_name = 'tuesday') + then sales_price + else null end) + tue_sales, + sum(case when (d_day_name = 'wednesday') + then sales_price + else null end) + wed_sales, + sum(case when (d_day_name = 'thursday') + then sales_price + else null end) + thu_sales, + sum(case when (d_day_name = 'friday') + then sales_price + else null end) + fri_sales, + sum(case when (d_day_name = 'saturday') + then sales_price + else null end) + sat_sales + group by d_week_seq) +table wswscs +|> join date_dim +|> where date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001 +|> select + wswscs.d_week_seq d_week_seq1, + sun_sales sun_sales1, + mon_sales mon_sales1, + tue_sales tue_sales1, + wed_sales wed_sales1, + thu_sales thu_sales1, + fri_sales fri_sales1, + sat_sales sat_sales1 +|> as y +|> join ( + table wswscs + |> join date_dim + |> where date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001 + 1 + |> select + wswscs.d_week_seq d_week_seq2, + sun_sales sun_sales2, + mon_sales mon_sales2, + tue_sales tue_sales2, + wed_sales wed_sales2, + thu_sales thu_sales2, + fri_sales fri_sales2, + sat_sales sat_sales2 + |> as z) +|> where d_week_seq1 = d_week_seq2 - 53 +|> order by d_week_seq1 +|> select + d_week_seq1, + round(sun_sales1 / sun_sales2, 2), + round(mon_sales1 / mon_sales2, 2), + round(tue_sales1 / tue_sales2, 2), + round(wed_sales1 / wed_sales2, 2), + round(thu_sales1 / thu_sales2, 2), + round(fri_sales1 / fri_sales2, 2), + round(sat_sales1 / sat_sales2, 2); + +-- Q3 +select + dt.d_year, + item.i_brand_id brand_id, + item.i_brand brand, + sum(ss_ext_sales_price) sum_agg +from date_dim dt, store_sales, item +where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manufact_id = 128 + and dt.d_moy = 11 +group by dt.d_year, item.i_brand, item.i_brand_id +order by dt.d_year, sum_agg desc, brand_id +limit 100; + +table date_dim +|> as dt +|> join store_sales +|> join item +|> where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manufact_id = 128 + and dt.d_moy = 11 +|> aggregate sum(ss_ext_sales_price) sum_agg + group by dt.d_year d_year, item.i_brand_id brand_id, item.i_brand brand +|> order by d_year, sum_agg desc, brand_id +|> limit 100; + +-- Q12 +select + i_item_desc, + i_category, + i_class, + i_current_price, + sum(ws_ext_sales_price) as itemrevenue, + sum(ws_ext_sales_price) * 100 / sum(sum(ws_ext_sales_price)) + over + (partition by i_class) as revenueratio +from + web_sales, item, date_dim +where + ws_item_sk = i_item_sk + and i_category in ('sports', 'books', 'home') + and ws_sold_date_sk = d_date_sk + and d_date between cast('1999-02-22' as date) + and (cast('1999-02-22' as date) + interval 30 days) +group by + i_item_id, i_item_desc, i_category, i_class, i_current_price +order by + i_category, i_class, i_item_id, i_item_desc, revenueratio +limit 100; + +table web_sales +|> join item +|> join date_dim +|> where ws_item_sk = i_item_sk + and i_category in ('sports', 'books', 'home') + and ws_sold_date_sk = d_date_sk + and d_date between cast('1999-02-22' as date) + and (cast('1999-02-22' as date) + interval 30 days) +|> aggregate sum(ws_ext_sales_price) AS itemrevenue + group by i_item_id, i_item_desc, i_category, i_class, i_current_price +|> extend + itemrevenue * 100 / sum(itemrevenue) + over (partition by i_class) as revenueratio +|> order by i_category, i_class, i_item_id, i_item_desc, revenueratio +|> select i_item_desc, i_category, i_class, i_current_price, itemrevenue, revenueratio +|> limit 100; + +-- Q44 +select + asceding.rnk, + i1.i_product_name best_performing, + i2.i_product_name worst_performing +from (select * +from (select + item_sk, + rank() + over ( + order by rank_col asc) rnk +from (select + ss_item_sk item_sk, + avg(ss_net_profit) rank_col +from store_sales ss1 +where ss_store_sk = 4 +group by ss_item_sk +having avg(ss_net_profit) > 0.9 * (select avg(ss_net_profit) rank_col +from store_sales +where ss_store_sk = 4 + and ss_addr_sk is null +group by ss_store_sk)) v1) v11 +where rnk < 11) asceding, + (select * + from (select + item_sk, + rank() + over ( + order by rank_col desc) rnk + from (select + ss_item_sk item_sk, + avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 4 + group by ss_item_sk + having avg(ss_net_profit) > 0.9 * (select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 4 + and ss_addr_sk is null + group by ss_store_sk)) v2) v21 + where rnk < 11) descending, + item i1, item i2 +where asceding.rnk = descending.rnk + and i1.i_item_sk = asceding.item_sk + and i2.i_item_sk = descending.item_sk +order by asceding.rnk +limit 100; + +from store_sales ss1 +|> where ss_store_sk = 4 +|> aggregate avg(ss_net_profit) rank_col + group by ss_item_sk as item_sk +|> where rank_col > 0.9 * ( + from store_sales + |> where ss_store_sk = 4 + and ss_addr_sk is null + |> aggregate avg(ss_net_profit) rank_col + group by ss_store_sk + |> select rank_col) +|> as v1 +|> select + item_sk, + rank() over ( + order by rank_col asc) rnk +|> as v11 +|> where rnk < 11 +|> as asceding +|> join ( + from store_sales ss1 + |> where ss_store_sk = 4 + |> aggregate avg(ss_net_profit) rank_col + group by ss_item_sk as item_sk + |> where rank_col > 0.9 * ( + table store_sales + |> where ss_store_sk = 4 + and ss_addr_sk is null + |> aggregate avg(ss_net_profit) rank_col + group by ss_store_sk + |> select rank_col) + |> as v2 + |> select + item_sk, + rank() over ( + order by rank_col asc) rnk + |> as v21 + |> where rnk < 11) descending +|> join item i1 +|> join item i2 +|> where asceding.rnk = descending.rnk + and i1.i_item_sk = asceding.item_sk + and i2.i_item_sk = descending.item_sk +|> order by asceding.rnk +|> select + asceding.rnk, + i1.i_product_name best_performing, + i2.i_product_name worst_performing; + +-- Q51 +with web_v1 as ( + select + ws_item_sk item_sk, + d_date, + sum(sum(ws_sales_price)) + over (partition by ws_item_sk + order by d_date + rows between unbounded preceding and current row) cume_sales + from web_sales, date_dim + where ws_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ws_item_sk is not null + group by ws_item_sk, d_date), + store_v1 as ( + select + ss_item_sk item_sk, + d_date, + sum(sum(ss_sales_price)) + over (partition by ss_item_sk + order by d_date + rows between unbounded preceding and current row) cume_sales + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ss_item_sk is not null + group by ss_item_sk, d_date) +select * +from (select + item_sk, + d_date, + web_sales, + store_sales, + max(web_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) web_cumulative, + max(store_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) store_cumulative +from (select + case when web.item_sk is not null + then web.item_sk + else store.item_sk end item_sk, + case when web.d_date is not null + then web.d_date + else store.d_date end d_date, + web.cume_sales web_sales, + store.cume_sales store_sales +from web_v1 web full outer join store_v1 store on (web.item_sk = store.item_sk + and web.d_date = store.d_date) + ) x) y +where web_cumulative > store_cumulative +order by item_sk, d_date +limit 100; + +with web_v1 as ( + table web_sales + |> join date_dim + |> where ws_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ws_item_sk is not null + |> aggregate sum(ws_sales_price) as sum_ws_sales_price + group by ws_item_sk as item_sk, d_date + |> extend sum(sum_ws_sales_price) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) + as cume_sales), +store_v1 as ( + table store_sales + |> join date_dim + |> where ss_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ss_item_sk is not null + |> aggregate sum(ss_sales_price) as sum_ss_sales_price + group by ss_item_sk as item_sk, d_date + |> extend sum(sum_ss_sales_price) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) + as cume_sales) +table web_v1 +|> as web +|> full outer join store_v1 store + on (web.item_sk = store.item_sk and web.d_date = store.d_date) +|> select + case when web.item_sk is not null + then web.item_sk + else store.item_sk end item_sk, + case when web.d_date is not null + then web.d_date + else store.d_date end d_date, + web.cume_sales web_sales, + store.cume_sales store_sales +|> as x +|> select + item_sk, + d_date, + web_sales, + store_sales, + max(web_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) web_cumulative, + max(store_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) store_cumulative +|> as y +|> where web_cumulative > store_cumulative +|> order by item_sk, d_date +|> limit 100; + +-- Cleanup. +----------- +drop table t; +drop table other; +drop table st; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/pivot.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/pivot.sql new file mode 100644 index 000000000000..c2ecd97e2b02 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/pivot.sql @@ -0,0 +1,300 @@ +create temporary view courseSales as select * from values + ("dotNET", 2012, 10000), + ("Java", 2012, 20000), + ("dotNET", 2012, 5000), + ("dotNET", 2013, 48000), + ("Java", 2013, 30000) + as courseSales(course, year, earnings); + +create temporary view years as select * from values + (2012, 1), + (2013, 2) + as years(y, s); + +create temporary view yearsWithComplexTypes as select * from values + (2012, array(1, 1), map('1', 1), struct(1, 'a')), + (2013, array(2, 2), map('2', 2), struct(2, 'b')) + as yearsWithComplexTypes(y, a, m, s); + +-- pivot courses +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(earnings) + FOR course IN ('dotNET', 'Java') +); + +-- pivot years with no subquery +SELECT * FROM courseSales +PIVOT ( + sum(earnings) + FOR year IN (2012, 2013) +); + +-- pivot courses with multiple aggregations +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(earnings), avg(earnings) + FOR course IN ('dotNET', 'Java') +); + +-- pivot with no group by column +SELECT * FROM ( + SELECT course, earnings FROM courseSales +) +PIVOT ( + sum(earnings) + FOR course IN ('dotNET', 'Java') +); + +-- pivot with no group by column and with multiple aggregations on different columns +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(earnings), min(year) + FOR course IN ('dotNET', 'Java') +); + +-- pivot on join query with multiple group by columns +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + sum(earnings) + FOR s IN (1, 2) +); + +-- pivot on join query with multiple aggregations on different columns +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + sum(earnings), min(s) + FOR course IN ('dotNET', 'Java') +); + +-- pivot on join query with multiple columns in one aggregation +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + sum(earnings * s) + FOR course IN ('dotNET', 'Java') +); + +-- pivot with aliases and projection +SELECT 2012_s, 2013_s, 2012_a, 2013_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + sum(e) s, avg(e) a + FOR y IN (2012, 2013) +); + +-- pivot with projection and value aliases +SELECT firstYear_s, secondYear_s, firstYear_a, secondYear_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + sum(e) s, avg(e) a + FOR y IN (2012 as firstYear, 2013 secondYear) +); + +-- pivot years with non-aggregate function +SELECT * FROM courseSales +PIVOT ( + abs(earnings) + FOR year IN (2012, 2013) +); + +-- pivot with one of the expressions as non-aggregate function +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(earnings), year + FOR course IN ('dotNET', 'Java') +); + +-- pivot with unresolvable columns +SELECT * FROM ( + SELECT course, earnings FROM courseSales +) +PIVOT ( + sum(earnings) + FOR year IN (2012, 2013) +); + +-- pivot with complex aggregate expressions +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + ceil(sum(earnings)), avg(earnings) + 1 as a1 + FOR course IN ('dotNET', 'Java') +); + +-- pivot with invalid arguments in aggregate expressions +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(avg(earnings)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot on multiple pivot columns +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + sum(earnings) + FOR (course, year) IN (('dotNET', 2012), ('Java', 2013)) +); + +-- pivot on multiple pivot columns with aliased values +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + sum(earnings) + FOR (course, s) IN (('dotNET', 2) as c1, ('Java', 1) as c2) +); + +-- pivot on multiple pivot columns with values of wrong data types +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + sum(earnings) + FOR (course, year) IN ('dotNET', 'Java') +); + +-- pivot with unresolvable values +SELECT * FROM courseSales +PIVOT ( + sum(earnings) + FOR year IN (s, 2013) +); + +-- pivot with non-literal values +SELECT * FROM courseSales +PIVOT ( + sum(earnings) + FOR year IN (course, 2013) +); + +-- pivot on join query with columns of complex data types +SELECT * FROM ( + SELECT course, year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + min(a) + FOR course IN ('dotNET', 'Java') +); + +-- pivot on multiple pivot columns with agg columns of complex data types +SELECT * FROM ( + SELECT course, year, y, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + max(a) + FOR (y, course) IN ((2012, 'dotNET'), (2013, 'Java')) +); + +-- pivot on pivot column of array type +SELECT * FROM ( + SELECT earnings, year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + sum(earnings) + FOR a IN (array(1, 1), array(2, 2)) +); + +-- pivot on multiple pivot columns containing array type +SELECT * FROM ( + SELECT course, earnings, year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + sum(earnings) + FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2))) +); + +-- pivot on pivot column of struct type +SELECT * FROM ( + SELECT earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + sum(earnings) + FOR s IN ((1, 'a'), (2, 'b')) +); + +-- pivot on multiple pivot columns containing struct type +SELECT * FROM ( + SELECT course, earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + sum(earnings) + FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b'))) +); + +-- pivot on pivot column of map type +SELECT * FROM ( + SELECT earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + sum(earnings) + FOR m IN (map('1', 1), map('2', 2)) +); + +-- pivot on multiple pivot columns containing map type +SELECT * FROM ( + SELECT course, earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + sum(earnings) + FOR (course, m) IN (('dotNET', map('1', 1)), ('Java', map('2', 2))) +); + +-- grouping columns output in the same order as input +-- correctly handle pivot columns with different cases +SELECT * FROM ( + SELECT course, earnings, "a" as a, "z" as z, "b" as b, "y" as y, "c" as c, "x" as x, "d" as d, "w" as w + FROM courseSales +) +PIVOT ( + sum(Earnings) + FOR Course IN ('dotNET', 'Java') +); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/aggregates_part1.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/aggregates_part1.sql new file mode 100644 index 000000000000..1152d77da0cf --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/aggregates_part1.sql @@ -0,0 +1,153 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- AGGREGATES [Part 1] +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L1-L143 + +-- avoid bit-exact output here because operations may not be bit-exact. +-- SET extra_float_digits = 0; + +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +SELECT avg(four) AS avg_1 FROM onek; + +SELECT avg(a) AS avg_32 FROM aggtest WHERE a < 100; + +-- In 7.1, avg(float4) is computed using float8 arithmetic. +-- Round the result to 3 digits to avoid platform-specific results. + +select CAST(avg(b) AS Decimal(10,3)) AS avg_107_943 FROM aggtest; +-- `student` has a column with data type POINT, which is not supported by Spark [SPARK-27766] +-- SELECT avg(gpa) AS avg_3_4 FROM ONLY student; + +SELECT sum(four) AS sum_1500 FROM onek; +SELECT sum(a) AS sum_198 FROM aggtest; +SELECT sum(b) AS avg_431_773 FROM aggtest; +-- `student` has a column with data type POINT, which is not supported by Spark [SPARK-27766] +-- SELECT sum(gpa) AS avg_6_8 FROM ONLY student; + +SELECT max(four) AS max_3 FROM onek; +SELECT max(a) AS max_100 FROM aggtest; +SELECT max(aggtest.b) AS max_324_78 FROM aggtest; +-- `student` has a column with data type POINT, which is not supported by Spark [SPARK-27766] +-- SELECT max(student.gpa) AS max_3_7 FROM student; + +SELECT stddev_pop(b) FROM aggtest; +SELECT stddev_samp(b) FROM aggtest; +SELECT var_pop(b) FROM aggtest; +SELECT var_samp(b) FROM aggtest; + +SELECT stddev_pop(CAST(b AS Decimal(38,0))) FROM aggtest; +SELECT stddev_samp(CAST(b AS Decimal(38,0))) FROM aggtest; +SELECT var_pop(CAST(b AS Decimal(38,0))) FROM aggtest; +SELECT var_samp(CAST(b AS Decimal(38,0))) FROM aggtest; + +-- population variance is defined for a single tuple, sample variance +-- is not +SELECT var_pop(1.0), var_samp(2.0); +SELECT stddev_pop(CAST(3.0 AS Decimal(38,0))), stddev_samp(CAST(4.0 AS Decimal(38,0))); + + +-- verify correct results for null and NaN inputs +select sum(CAST(null AS int)) from range(1,4); +select sum(CAST(null AS long)) from range(1,4); +select sum(CAST(null AS Decimal(38,0))) from range(1,4); +select sum(CAST(null AS DOUBLE)) from range(1,4); +select avg(CAST(null AS int)) from range(1,4); +select avg(CAST(null AS long)) from range(1,4); +select avg(CAST(null AS Decimal(38,0))) from range(1,4); +select avg(CAST(null AS DOUBLE)) from range(1,4); +select sum(CAST('NaN' AS DOUBLE)) from range(1,4); +select avg(CAST('NaN' AS DOUBLE)) from range(1,4); + +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES (CAST('1' AS DOUBLE)), (CAST('infinity' AS DOUBLE))) v(x); +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES ('infinity'), ('1')) v(x); +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES ('infinity'), ('infinity')) v(x); +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES ('-infinity'), ('infinity')) v(x); + +-- test accuracy with a large input offset +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x); +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES (7000000000005), (7000000000007)) v(x); + +-- SQL2003 binary aggregates [SPARK-23907] +SELECT regr_count(b, a) FROM aggtest; +SELECT regr_sxx(b, a) FROM aggtest; +SELECT regr_syy(b, a) FROM aggtest; +SELECT regr_sxy(b, a) FROM aggtest; +SELECT regr_avgx(b, a), regr_avgy(b, a) FROM aggtest; +SELECT regr_r2(b, a) FROM aggtest; +SELECT regr_slope(b, a), regr_intercept(b, a) FROM aggtest; +SELECT covar_pop(b, a), covar_samp(b, a) FROM aggtest; +SELECT corr(b, a) FROM aggtest; + + +-- test accum and combine functions directly [SPARK-23907] +CREATE TEMPORARY VIEW regr_test AS SELECT * FROM VALUES (10,150),(20,250),(30,350),(80,540),(100,200) AS regr_test (x, y); +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test WHERE x IN (10,20,30,80); +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test; +-- SELECT float8_accum('{4,140,2900}'::float8[], 100); +-- SELECT float8_regr_accum('{4,140,2900,1290,83075,15050}'::float8[], 200, 100); +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test WHERE x IN (10,20,30); +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test WHERE x IN (80,100); +-- SELECT float8_combine('{3,60,200}'::float8[], '{0,0,0}'::float8[]); +-- SELECT float8_combine('{0,0,0}'::float8[], '{2,180,200}'::float8[]); +-- SELECT float8_combine('{3,60,200}'::float8[], '{2,180,200}'::float8[]); +-- SELECT float8_regr_combine('{3,60,200,750,20000,2000}'::float8[], +-- '{0,0,0,0,0,0}'::float8[]); +-- SELECT float8_regr_combine('{0,0,0,0,0,0}'::float8[], +-- '{2,180,200,740,57800,-3400}'::float8[]); +-- SELECT float8_regr_combine('{3,60,200,750,20000,2000}'::float8[], +-- '{2,180,200,740,57800,-3400}'::float8[]); +DROP VIEW regr_test; + + +-- test count, distinct +SELECT count(four) AS cnt_1000 FROM onek; +SELECT count(DISTINCT four) AS cnt_4 FROM onek; + +select ten, count(*), sum(four) from onek +group by ten order by ten; + +select ten, count(four), sum(DISTINCT four) from onek +group by ten order by ten; + +-- user-defined aggregates +-- SELECT newavg(four) AS avg_1 FROM onek; +-- SELECT newsum(four) AS sum_1500 FROM onek; +-- SELECT newcnt(four) AS cnt_1000 FROM onek; +-- SELECT newcnt(*) AS cnt_1000 FROM onek; +-- SELECT oldcnt(*) AS cnt_1000 FROM onek; +-- SELECT sum2(q1,q2) FROM int8_tbl; + +-- test for outer-level aggregates + +-- this should work +select ten, sum(distinct four) from onek a +group by ten +having exists (select 1 from onek b where sum(distinct a.four) = b.four); + +-- this should fail because subquery has an agg of its own in WHERE +select ten, sum(distinct four) from onek a +group by ten +having exists (select 1 from onek b + where sum(distinct a.four + b.four) = b.four); + +-- [SPARK-27769] Test handling of sublinks within outer-level aggregates. +-- Per bug report from Daniel Grace. +select + (select max((select i.unique2 from tenk1 i where i.unique1 = o.unique1))) +from tenk1 o; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/aggregates_part2.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/aggregates_part2.sql new file mode 100644 index 000000000000..a8af1db77563 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/aggregates_part2.sql @@ -0,0 +1,218 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- AGGREGATES [Part 2] +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L145-L350 + +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1); + +-- Test handling of Params within aggregate arguments in hashed aggregation. +-- Per bug report from Jeevan Chalke. +-- [SPARK-27877] Implement SQL-standard LATERAL subqueries +-- explain (verbose, costs off) +-- select s1, s2, sm +-- from generate_series(1, 3) s1, +-- lateral (select s2, sum(s1 + s2) sm +-- from generate_series(1, 3) s2 group by s2) ss +-- order by 1, 2; +-- select s1, s2, sm +-- from generate_series(1, 3) s1, +-- lateral (select s2, sum(s1 + s2) sm +-- from generate_series(1, 3) s2 group by s2) ss +-- order by 1, 2; + +-- [SPARK-27878] Support ARRAY(sub-SELECT) expressions +-- explain (verbose, costs off) +-- select array(select sum(x+y) s +-- from generate_series(1,3) y group by y order by s) +-- from generate_series(1,3) x; +-- select array(select sum(x+y) s +-- from generate_series(1,3) y group by y order by s) +-- from generate_series(1,3) x; + +-- [SPARK-27879] Implement bitwise integer aggregates(BIT_AND and BIT_OR) +-- +-- test for bitwise integer aggregates +-- +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (3, 3, 3, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4); + +-- empty case +SELECT BIT_AND(b1) AS n1, BIT_OR(b2) AS n2 FROM bitwise_test where 1 = 0; + +-- null case +SELECT BIT_AND(b4) AS n1, BIT_OR(b4) AS n2 FROM bitwise_test where b4 is null; + +SELECT + BIT_AND(cast(b1 as tinyint)) AS a1, + BIT_AND(cast(b2 as smallint)) AS b1, + BIT_AND(b3) AS c1, + BIT_AND(b4) AS d1, + BIT_OR(cast(b1 as tinyint)) AS e7, + BIT_OR(cast(b2 as smallint)) AS f7, + BIT_OR(b3) AS g7, + BIT_OR(b4) AS h3 +FROM bitwise_test; + +-- group by +SELECT b1 , bit_and(b2), bit_or(b4) FROM bitwise_test GROUP BY b1; + +--having +SELECT b1, bit_and(b2) FROM bitwise_test GROUP BY b1 HAVING bit_and(b2) < 7; + +-- window +SELECT b1, b2, bit_and(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test; +SELECT b1, b2, bit_or(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test; + +-- +-- test boolean aggregates +-- +-- first test all possible transition and final states + +-- The result is inconsistent with PostgreSQL because our AND does not have strict mode +SELECT + -- boolean and transitions + -- null because strict + (NULL AND NULL) IS NULL AS `t`, + (TRUE AND NULL) IS NULL AS `t`, + (FALSE AND NULL) IS NULL AS `t`, + (NULL AND TRUE) IS NULL AS `t`, + (NULL AND FALSE) IS NULL AS `t`, + -- and actual computations + (TRUE AND TRUE) AS `t`, + NOT (TRUE AND FALSE) AS `t`, + NOT (FALSE AND TRUE) AS `t`, + NOT (FALSE AND FALSE) AS `t`; + +-- The result is inconsistent with PostgreSQL because our OR does not have strict mode +SELECT + -- boolean or transitions + -- null because strict + (NULL OR NULL) IS NULL AS `t`, + (TRUE OR NULL) IS NULL AS `t`, + (FALSE OR NULL) IS NULL AS `t`, + (NULL OR TRUE) IS NULL AS `t`, + (NULL OR FALSE) IS NULL AS `t`, + -- actual computations + (TRUE OR TRUE) AS `t`, + (TRUE OR FALSE) AS `t`, + (FALSE OR TRUE) AS `t`, + NOT (FALSE OR FALSE) AS `t`; + +-- [SPARK-27880] Implement boolean aggregates(BOOL_AND, BOOL_OR and EVERY) +CREATE OR REPLACE TEMPORARY VIEW bool_test AS SELECT * FROM VALUES + (TRUE, null, FALSE, null), + (FALSE, TRUE, null, null), + (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4); + +-- empty case +SELECT BOOL_AND(b1) AS n1, BOOL_OR(b3) AS n2 FROM bool_test WHERE 1 = 0; + +SELECT + BOOL_AND(b1) AS f1, + BOOL_AND(b2) AS t2, + BOOL_AND(b3) AS f3, + BOOL_AND(b4) AS n4, + BOOL_AND(NOT b2) AS f5, + BOOL_AND(NOT b3) AS t6 +FROM bool_test; + +SELECT + EVERY(b1) AS f1, + EVERY(b2) AS t2, + EVERY(b3) AS f3, + EVERY(b4) AS n4, + EVERY(NOT b2) AS f5, + EVERY(NOT b3) AS t6 +FROM bool_test; + +SELECT + BOOL_OR(b1) AS t1, + BOOL_OR(b2) AS t2, + BOOL_OR(b3) AS f3, + BOOL_OR(b4) AS n4, + BOOL_OR(NOT b2) AS f5, + BOOL_OR(NOT b3) AS t6 +FROM bool_test; + +-- +-- Test cases that should be optimized into indexscans instead of +-- the generic aggregate implementation. +-- + +-- Basic cases +-- explain +-- select min(unique1) from tenk1; +select min(unique1) from tenk1; +-- explain +-- select max(unique1) from tenk1; +select max(unique1) from tenk1; +-- explain +-- select max(unique1) from tenk1 where unique1 < 42; +select max(unique1) from tenk1 where unique1 < 42; +-- explain +-- select max(unique1) from tenk1 where unique1 > 42; +select max(unique1) from tenk1 where unique1 > 42; + +-- the planner may choose a generic aggregate here if parallel query is +-- enabled, since that plan will be parallel safe and the "optimized" +-- plan, which has almost identical cost, will not be. we want to test +-- the optimized plan, so temporarily disable parallel query. +-- begin; +-- set local max_parallel_workers_per_gather = 0; +-- explain +-- select max(unique1) from tenk1 where unique1 > 42000; +select max(unique1) from tenk1 where unique1 > 42000; +-- rollback; + +-- multi-column index (uses tenk1_thous_tenthous) +-- explain +-- select max(tenthous) from tenk1 where thousand = 33; +select max(tenthous) from tenk1 where thousand = 33; +-- explain +-- select min(tenthous) from tenk1 where thousand = 33; +select min(tenthous) from tenk1 where thousand = 33; + +-- [SPARK-17348] Correlated column is not allowed in a non-equality predicate +-- check parameter propagation into an indexscan subquery +-- explain +-- select f1, (select min(unique1) from tenk1 where unique1 > f1) AS gt +-- from int4_tbl; +-- select f1, (select min(unique1) from tenk1 where unique1 > f1) AS gt +-- from int4_tbl; + +-- check some cases that were handled incorrectly in 8.3.0 +-- explain +-- select distinct max(unique2) from tenk1; +select distinct max(unique2) from tenk1; +-- explain +-- select max(unique2) from tenk1 order by 1; +select max(unique2) from tenk1 order by 1; +-- explain +-- select max(unique2) from tenk1 order by max(unique2); +select max(unique2) from tenk1 order by max(unique2); +-- explain +-- select max(unique2) from tenk1 order by max(unique2)+1; +select max(unique2) from tenk1 order by max(unique2)+1; +-- explain +-- select max(unique2), generate_series(1,3) as g from tenk1 order by g desc; +select t1.max_unique2, g from (select max(unique2) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc; + +-- interesting corner case: constant gets optimized into a seqscan +-- explain +-- select max(100) from tenk1; +select max(100) from tenk1; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/aggregates_part3.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/aggregates_part3.sql new file mode 100644 index 000000000000..41fd4de2a097 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/aggregates_part3.sql @@ -0,0 +1,276 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- AGGREGATES [Part 3] +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L352-L605 + +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +-- [SPARK-28865] Table inheritance +-- try it on an inheritance tree +-- create table minmaxtest(f1 int); +-- create table minmaxtest1() inherits (minmaxtest); +-- create table minmaxtest2() inherits (minmaxtest); +-- create table minmaxtest3() inherits (minmaxtest); +-- create index minmaxtesti on minmaxtest(f1); +-- create index minmaxtest1i on minmaxtest1(f1); +-- create index minmaxtest2i on minmaxtest2(f1 desc); +-- create index minmaxtest3i on minmaxtest3(f1) where f1 is not null; + +-- insert into minmaxtest values(11), (12); +-- insert into minmaxtest1 values(13), (14); +-- insert into minmaxtest2 values(15), (16); +-- insert into minmaxtest3 values(17), (18); + +-- explain (costs off) +-- select min(f1), max(f1) from minmaxtest; +-- select min(f1), max(f1) from minmaxtest; + +-- DISTINCT doesn't do anything useful here, but it shouldn't fail +-- explain (costs off) +-- select distinct min(f1), max(f1) from minmaxtest; +-- select distinct min(f1), max(f1) from minmaxtest; + +-- drop table minmaxtest cascade; + +-- check for correct detection of nested-aggregate errors +select max(min(unique1)) from tenk1; +-- select (select max(min(unique1)) from int8_tbl) from tenk1; + +-- These tests only test the explain. Skip these tests. +-- +-- Test removal of redundant GROUP BY columns +-- + +-- create temp table t1 (a int, b int, c int, d int, primary key (a, b)); +-- create temp table t2 (x int, y int, z int, primary key (x, y)); +-- create temp table t3 (a int, b int, c int, primary key(a, b) deferrable); + +-- Non-primary-key columns can be removed from GROUP BY +-- explain (costs off) select * from t1 group by a,b,c,d; + +-- No removal can happen if the complete PK is not present in GROUP BY +-- explain (costs off) select a,c from t1 group by a,c,d; + +-- Test removal across multiple relations +-- explain (costs off) select * +-- from t1 inner join t2 on t1.a = t2.x and t1.b = t2.y +-- group by t1.a,t1.b,t1.c,t1.d,t2.x,t2.y,t2.z; + +-- Test case where t1 can be optimized but not t2 +-- explain (costs off) select t1.*,t2.x,t2.z +-- from t1 inner join t2 on t1.a = t2.x and t1.b = t2.y +-- group by t1.a,t1.b,t1.c,t1.d,t2.x,t2.z; + +-- Cannot optimize when PK is deferrable +-- explain (costs off) select * from t3 group by a,b,c; + +-- drop table t1; +-- drop table t2; +-- drop table t3; + +-- [SPARK-28664] ORDER BY in aggregate function +-- +-- Test combinations of DISTINCT and/or ORDER BY +-- + +-- select array_agg(a order by b) +-- from (values (1,4),(2,3),(3,1),(4,2)) v(a,b); +-- select array_agg(a order by a) +-- from (values (1,4),(2,3),(3,1),(4,2)) v(a,b); +-- select array_agg(a order by a desc) +-- from (values (1,4),(2,3),(3,1),(4,2)) v(a,b); +-- select array_agg(b order by a desc) +-- from (values (1,4),(2,3),(3,1),(4,2)) v(a,b); + +-- select array_agg(distinct a) +-- from (values (1),(2),(1),(3),(null),(2)) v(a); +-- select array_agg(distinct a order by a) +-- from (values (1),(2),(1),(3),(null),(2)) v(a); +-- select array_agg(distinct a order by a desc) +-- from (values (1),(2),(1),(3),(null),(2)) v(a); +-- select array_agg(distinct a order by a desc nulls last) +-- from (values (1),(2),(1),(3),(null),(2)) v(a); + +-- Skip the test below because it requires 4 UDAFs: aggf_trans, aggfns_trans, aggfstr, and aggfns +-- multi-arg aggs, strict/nonstrict, distinct/order by + +-- select aggfstr(a,b,c) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c); +-- select aggfns(a,b,c) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c); + +-- select aggfstr(distinct a,b,c) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,3) i; +-- select aggfns(distinct a,b,c) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,3) i; + +-- select aggfstr(distinct a,b,c order by b) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,3) i; +-- select aggfns(distinct a,b,c order by b) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,3) i; + +-- test specific code paths + +-- [SPARK-28768] Implement more text pattern operators +-- select aggfns(distinct a,a,c order by c using ~<~,a) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,2) i; +-- select aggfns(distinct a,a,c order by c using ~<~) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,2) i; +-- select aggfns(distinct a,a,c order by a) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,2) i; +-- select aggfns(distinct a,b,c order by a,c using ~<~,b) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,2) i; + +-- check node I/O via view creation and usage, also deparsing logic + +-- create view agg_view1 as +-- select aggfns(a,b,c) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c); + +-- select * from agg_view1; +-- select pg_get_viewdef('agg_view1'::regclass); + +-- create or replace view agg_view1 as +-- select aggfns(distinct a,b,c) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,3) i; + +-- select * from agg_view1; +-- select pg_get_viewdef('agg_view1'::regclass); + +-- create or replace view agg_view1 as +-- select aggfns(distinct a,b,c order by b) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,3) i; + +-- select * from agg_view1; +-- select pg_get_viewdef('agg_view1'::regclass); + +-- create or replace view agg_view1 as +-- select aggfns(a,b,c order by b+1) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c); + +-- select * from agg_view1; +-- select pg_get_viewdef('agg_view1'::regclass); + +-- create or replace view agg_view1 as +-- select aggfns(a,a,c order by b) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c); + +-- select * from agg_view1; +-- select pg_get_viewdef('agg_view1'::regclass); + +-- create or replace view agg_view1 as +-- select aggfns(a,b,c order by c using ~<~) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c); + +-- select * from agg_view1; +-- select pg_get_viewdef('agg_view1'::regclass); + +-- create or replace view agg_view1 as +-- select aggfns(distinct a,b,c order by a,c using ~<~,b) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,2) i; + +-- select * from agg_view1; +-- select pg_get_viewdef('agg_view1'::regclass); + +-- drop view agg_view1; + +-- incorrect DISTINCT usage errors + +-- select aggfns(distinct a,b,c order by i) +-- from (values (1,1,'foo')) v(a,b,c), generate_series(1,2) i; +-- select aggfns(distinct a,b,c order by a,b+1) +-- from (values (1,1,'foo')) v(a,b,c), generate_series(1,2) i; +-- select aggfns(distinct a,b,c order by a,b,i,c) +-- from (values (1,1,'foo')) v(a,b,c), generate_series(1,2) i; +-- select aggfns(distinct a,a,c order by a,b) +-- from (values (1,1,'foo')) v(a,b,c), generate_series(1,2) i; + +-- [SPARK-27978] Add built-in Aggregate Functions: string_agg +-- string_agg tests +-- select string_agg(a,',') from (values('aaaa'),('bbbb'),('cccc')) g(a); +-- select string_agg(a,',') from (values('aaaa'),(null),('bbbb'),('cccc')) g(a); +-- select string_agg(a,'AB') from (values(null),(null),('bbbb'),('cccc')) g(a); +-- select string_agg(a,',') from (values(null),(null)) g(a); + +-- check some implicit casting cases, as per bug #5564 +-- select string_agg(distinct f1, ',' order by f1) from varchar_tbl; -- ok +-- select string_agg(distinct f1::text, ',' order by f1) from varchar_tbl; -- not ok +-- select string_agg(distinct f1, ',' order by f1::text) from varchar_tbl; -- not ok +-- select string_agg(distinct f1::text, ',' order by f1::text) from varchar_tbl; -- ok + +-- [SPARK-28121] decode can not accept 'hex' as charset +-- string_agg bytea tests +-- CREATE TABLE bytea_test_table(v BINARY) USING parquet; + +-- select string_agg(v, '') from bytea_test_table; + +-- insert into bytea_test_table values(decode('ff','hex')); + +-- select string_agg(v, '') from bytea_test_table; + +-- insert into bytea_test_table values(decode('aa','hex')); + +-- select string_agg(v, '') from bytea_test_table; +-- select string_agg(v, NULL) from bytea_test_table; +-- select string_agg(v, decode('ee', 'hex')) from bytea_test_table; + +-- drop table bytea_test_table; + +-- FILTER tests + +select min(unique1) filter (where unique1 > 100) from tenk1; + +select sum(1/ten) filter (where ten > 0) from tenk1; + +-- select ten, sum(distinct four) filter (where four::text ~ '123') from onek a +-- group by ten; + +select ten, sum(distinct four) filter (where four > 10) from onek a +group by ten +having exists (select 1 from onek b where sum(distinct a.four) = b.four); + +-- [SPARK-28682] ANSI SQL: Collation Support +-- select max(foo COLLATE "C") filter (where (bar collate "POSIX") > '0') +-- from (values ('a', 'b')) AS v(foo,bar); + +-- outer reference in FILTER (PostgreSQL extension) +select (select count(*) + from (values (1)) t0(inner_c)) +from (values (2),(3)) t1(outer_c); -- inner query is aggregation query +-- [SPARK-30219] Support Filter expression reference the outer query +-- select (select count(*) filter (where outer_c <> 0) +-- from (values (1)) t0(inner_c)) +-- from (values (2),(3)) t1(outer_c); -- outer query is aggregation query +-- select (select count(inner_c) filter (where outer_c <> 0) +-- from (values (1)) t0(inner_c)) +-- from (values (2),(3)) t1(outer_c); -- inner query is aggregation query +-- select +-- (select max((select i.unique2 from tenk1 i where i.unique1 = o.unique1)) +-- filter (where o.unique1 < 10)) +-- from tenk1 o; -- outer query is aggregation query + +-- [SPARK-30220] Support Filter expression uses IN/EXISTS predicate sub-queries +-- subquery in FILTER clause (PostgreSQL extension) +-- select sum(unique1) FILTER (WHERE +-- unique1 IN (SELECT unique1 FROM onek where unique1 < 100)) FROM tenk1; + +-- exercise lots of aggregate parts with FILTER +-- select aggfns(distinct a,b,c order by a,c using ~<~,b) filter (where a > 1) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,2) i; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/aggregates_part4.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/aggregates_part4.sql new file mode 100644 index 000000000000..3cf57b1f0e50 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/aggregates_part4.sql @@ -0,0 +1,424 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- AGGREGATES [Part 4] +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L607-L997 + +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +-- [SPARK-27980] Ordered-Set Aggregate Functions +-- ordered-set aggregates + +-- select p, percentile_cont(p) within group (order by x::float8) +-- from generate_series(1,5) x, +-- (values (0::float8),(0.1),(0.25),(0.4),(0.5),(0.6),(0.75),(0.9),(1)) v(p) +-- group by p order by p; + +-- select p, percentile_cont(p order by p) within group (order by x) -- error +-- from generate_series(1,5) x, +-- (values (0::float8),(0.1),(0.25),(0.4),(0.5),(0.6),(0.75),(0.9),(1)) v(p) +-- group by p order by p; + +-- select p, sum() within group (order by x::float8) -- error +-- from generate_series(1,5) x, +-- (values (0::float8),(0.1),(0.25),(0.4),(0.5),(0.6),(0.75),(0.9),(1)) v(p) +-- group by p order by p; + +-- select p, percentile_cont(p,p) -- error +-- from generate_series(1,5) x, +-- (values (0::float8),(0.1),(0.25),(0.4),(0.5),(0.6),(0.75),(0.9),(1)) v(p) +-- group by p order by p; + +select percentile_cont(0.5) within group (order by b) from aggtest; +select percentile_cont(0.5) within group (order by b), sum(b) from aggtest; +select percentile_cont(0.5) within group (order by thousand) from tenk1; +select percentile_disc(0.5) within group (order by thousand) from tenk1; +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- select rank(3) within group (order by x) +-- from (values (1),(1),(2),(2),(3),(3),(4)) v(x); +-- select cume_dist(3) within group (order by x) +-- from (values (1),(1),(2),(2),(3),(3),(4)) v(x); +-- select percent_rank(3) within group (order by x) +-- from (values (1),(1),(2),(2),(3),(3),(4),(5)) v(x); +-- select dense_rank(3) within group (order by x) +-- from (values (1),(1),(2),(2),(3),(3),(4)) v(x); + +-- [SPARK-27980] Ordered-Set Aggregate Functions +-- select percentile_disc(array[0,0.1,0.25,0.5,0.75,0.9,1]) within group (order by thousand) +-- from tenk1; +-- select percentile_cont(array[0,0.25,0.5,0.75,1]) within group (order by thousand) +-- from tenk1; +-- select percentile_disc(array[[null,1,0.5],[0.75,0.25,null]]) within group (order by thousand) +-- from tenk1; +-- select percentile_cont(array[0,1,0.25,0.75,0.5,1,0.3,0.32,0.35,0.38,0.4]) within group (order by x) +-- from generate_series(1,6) x; + +-- [SPARK-27980] Ordered-Set Aggregate Functions +-- [SPARK-28382] Array Functions: unnest +-- select ten, mode() within group (order by string4) from tenk1 group by ten; + +-- select percentile_disc(array[0.25,0.5,0.75]) within group (order by x) +-- from unnest('{fred,jim,fred,jack,jill,fred,jill,jim,jim,sheila,jim,sheila}'::text[]) u(x); + +-- [SPARK-28669] System Information Functions +-- check collation propagates up in suitable cases: +-- select pg_collation_for(percentile_disc(1) within group (order by x collate "POSIX")) +-- from (values ('fred'),('jim')) v(x); + +-- test_rank and test_percentile_disc function created by create_aggregate.sql +-- ordered-set aggs created with CREATE AGGREGATE +-- select test_rank(3) within group (order by x) +-- from (values (1),(1),(2),(2),(3),(3),(4)) v(x); +-- select test_percentile_disc(0.5) within group (order by thousand) from tenk1; + +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- ordered-set aggs can't use ungrouped vars in direct args: +-- select rank(x) within group (order by x) from generate_series(1,5) x; + +-- [SPARK-27980] Ordered-Set Aggregate Functions +-- outer-level agg can't use a grouped arg of a lower level, either: +-- select array(select percentile_disc(a) within group (order by x) +-- from (values (0.3),(0.7)) v(a) group by a) +-- from generate_series(1,5) g(x); + +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- agg in the direct args is a grouping violation, too: +--select rank(sum(x)) within group (order by x) from generate_series(1,5) x; + +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- hypothetical-set type unification and argument-count failures: +-- select rank(3) within group (order by x) from (values ('fred'),('jim')) v(x); +-- select rank(3) within group (order by stringu1,stringu2) from tenk1; +-- select rank('fred') within group (order by x) from generate_series(1,5) x; +-- select rank('adam'::text collate "C") within group (order by x collate "POSIX") +-- from (values ('fred'),('jim')) v(x); +-- hypothetical-set type unification successes: +-- select rank('adam'::varchar) within group (order by x) from (values ('fred'),('jim')) v(x); +-- select rank('3') within group (order by x) from generate_series(1,5) x; + +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- divide by zero check +-- select percent_rank(0) within group (order by x) from generate_series(1,0) x; + +-- [SPARK-27980] Ordered-Set Aggregate Functions +-- deparse and multiple features: +-- create view aggordview1 as +-- select ten, +-- percentile_disc(0.5) within group (order by thousand) as p50, +-- percentile_disc(0.5) within group (order by thousand) filter (where hundred=1) as px, +-- rank(5,'AZZZZ',50) within group (order by hundred, string4 desc, hundred) +-- from tenk1 +-- group by ten order by ten; + +-- select pg_get_viewdef('aggordview1'); +-- select * from aggordview1 order by ten; +-- drop view aggordview1; + +-- least_agg created by create_aggregate.sql +-- variadic aggregates +-- select least_agg(q1,q2) from int8_tbl; +-- select least_agg(variadic array[q1,q2]) from int8_tbl; + + +-- Skip these tests because we do not support create type +-- test aggregates with common transition functions share the same states +-- begin work; + +-- create type avg_state as (total bigint, count bigint); + +-- create or replace function avg_transfn(state avg_state, n int) returns avg_state as +-- $$ +-- declare new_state avg_state; +-- begin +-- raise notice 'avg_transfn called with %', n; +-- if state is null then +-- if n is not null then +-- new_state.total := n; +-- new_state.count := 1; +-- return new_state; +-- end if; +-- return null; +-- elsif n is not null then +-- state.total := state.total + n; +-- state.count := state.count + 1; +-- return state; +-- end if; +-- +-- return null; +-- end +-- $$ language plpgsql; + +-- create function avg_finalfn(state avg_state) returns int4 as +-- $$ +-- begin +-- if state is null then +-- return NULL; +-- else +-- return state.total / state.count; +-- end if; +-- end +-- $$ language plpgsql; + +-- create function sum_finalfn(state avg_state) returns int4 as +-- $$ +-- begin +-- if state is null then +-- return NULL; +-- else +-- return state.total; +-- end if; +-- end +-- $$ language plpgsql; + +-- create aggregate my_avg(int4) +-- ( +-- stype = avg_state, +-- sfunc = avg_transfn, +-- finalfunc = avg_finalfn +-- ); +-- +-- create aggregate my_sum(int4) +-- ( +-- stype = avg_state, +-- sfunc = avg_transfn, +-- finalfunc = sum_finalfn +-- ); + +-- aggregate state should be shared as aggs are the same. +-- select my_avg(one),my_avg(one) from (values(1),(3)) t(one); + +-- aggregate state should be shared as transfn is the same for both aggs. +-- select my_avg(one),my_sum(one) from (values(1),(3)) t(one); + +-- same as previous one, but with DISTINCT, which requires sorting the input. +-- select my_avg(distinct one),my_sum(distinct one) from (values(1),(3),(1)) t(one); + +-- shouldn't share states due to the distinctness not matching. +-- select my_avg(distinct one),my_sum(one) from (values(1),(3)) t(one); + +-- shouldn't share states due to the filter clause not matching. +-- select my_avg(one) filter (where one > 1),my_sum(one) from (values(1),(3)) t(one); + +-- this should not share the state due to different input columns. +-- select my_avg(one),my_sum(two) from (values(1,2),(3,4)) t(one,two); + +-- [SPARK-27980] Ordered-Set Aggregate Functions +-- exercise cases where OSAs share state +-- select +-- percentile_cont(0.5) within group (order by a), +-- percentile_disc(0.5) within group (order by a) +-- from (values(1::float8),(3),(5),(7)) t(a); + +-- select +-- percentile_cont(0.25) within group (order by a), +-- percentile_disc(0.5) within group (order by a) +-- from (values(1::float8),(3),(5),(7)) t(a); + +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- these can't share state currently +-- select +-- rank(4) within group (order by a), +-- dense_rank(4) within group (order by a) +-- from (values(1),(3),(5),(7)) t(a); + +-- test that aggs with the same sfunc and initcond share the same agg state +-- create aggregate my_sum_init(int4) +-- ( +-- stype = avg_state, +-- sfunc = avg_transfn, +-- finalfunc = sum_finalfn, +-- initcond = '(10,0)' +-- ); + +-- create aggregate my_avg_init(int4) +-- ( +-- stype = avg_state, +-- sfunc = avg_transfn, +-- finalfunc = avg_finalfn, +-- initcond = '(10,0)' +-- ); + +-- create aggregate my_avg_init2(int4) +-- ( +-- stype = avg_state, +-- sfunc = avg_transfn, +-- finalfunc = avg_finalfn, +-- initcond = '(4,0)' +-- ); + +-- state should be shared if INITCONDs are matching +-- select my_sum_init(one),my_avg_init(one) from (values(1),(3)) t(one); + +-- Varying INITCONDs should cause the states not to be shared. +-- select my_sum_init(one),my_avg_init2(one) from (values(1),(3)) t(one); + +-- rollback; + +-- test aggregate state sharing to ensure it works if one aggregate has a +-- finalfn and the other one has none. +-- begin work; + +-- create or replace function sum_transfn(state int4, n int4) returns int4 as +-- $$ +-- declare new_state int4; +-- begin +-- raise notice 'sum_transfn called with %', n; +-- if state is null then +-- if n is not null then +-- new_state := n; +-- return new_state; +-- end if; +-- return null; +-- elsif n is not null then +-- state := state + n; +-- return state; +-- end if; +-- +-- return null; +-- end +-- $$ language plpgsql; + +-- create function halfsum_finalfn(state int4) returns int4 as +-- $$ +-- begin +-- if state is null then +-- return NULL; +-- else +-- return state / 2; +-- end if; +-- end +-- $$ language plpgsql; + +-- create aggregate my_sum(int4) +-- ( +-- stype = int4, +-- sfunc = sum_transfn +-- ); + +-- create aggregate my_half_sum(int4) +-- ( +-- stype = int4, +-- sfunc = sum_transfn, +-- finalfunc = halfsum_finalfn +-- ); + +-- Agg state should be shared even though my_sum has no finalfn +-- select my_sum(one),my_half_sum(one) from (values(1),(2),(3),(4)) t(one); + +-- rollback; + + +-- test that the aggregate transition logic correctly handles +-- transition / combine functions returning NULL + +-- First test the case of a normal transition function returning NULL +-- BEGIN; +-- CREATE FUNCTION balkifnull(int8, int4) +-- RETURNS int8 +-- STRICT +-- LANGUAGE plpgsql AS $$ +-- BEGIN +-- IF $1 IS NULL THEN +-- RAISE 'erroneously called with NULL argument'; +-- END IF; +-- RETURN NULL; +-- END$$; + +-- CREATE AGGREGATE balk(int4) +-- ( +-- SFUNC = balkifnull(int8, int4), +-- STYPE = int8, +-- PARALLEL = SAFE, +-- INITCOND = '0' +-- ); + +-- SELECT balk(hundred) FROM tenk1; + +-- ROLLBACK; + +-- Secondly test the case of a parallel aggregate combiner function +-- returning NULL. For that use normal transition function, but a +-- combiner function returning NULL. +-- BEGIN ISOLATION LEVEL REPEATABLE READ; +-- CREATE FUNCTION balkifnull(int8, int8) +-- RETURNS int8 +-- PARALLEL SAFE +-- STRICT +-- LANGUAGE plpgsql AS $$ +-- BEGIN +-- IF $1 IS NULL THEN +-- RAISE 'erroneously called with NULL argument'; +-- END IF; +-- RETURN NULL; +-- END$$; + +-- CREATE AGGREGATE balk(int4) +-- ( +-- SFUNC = int4_sum(int8, int4), +-- STYPE = int8, +-- COMBINEFUNC = balkifnull(int8, int8), +-- PARALLEL = SAFE, +-- INITCOND = '0' +-- ); + +-- force use of parallelism +-- ALTER TABLE tenk1 set (parallel_workers = 4); +-- SET LOCAL parallel_setup_cost=0; +-- SET LOCAL max_parallel_workers_per_gather=4; + +-- EXPLAIN (COSTS OFF) SELECT balk(hundred) FROM tenk1; +-- SELECT balk(hundred) FROM tenk1; + +-- ROLLBACK; + +-- test coverage for aggregate combine/serial/deserial functions +-- BEGIN ISOLATION LEVEL REPEATABLE READ; + +-- SET parallel_setup_cost = 0; +-- SET parallel_tuple_cost = 0; +-- SET min_parallel_table_scan_size = 0; +-- SET max_parallel_workers_per_gather = 4; +-- SET enable_indexonlyscan = off; + +-- [SPARK-28663] Aggregate Functions for Statistics +-- variance(int4) covers numeric_poly_combine +-- sum(int8) covers int8_avg_combine +-- regr_count(float8, float8) covers int8inc_float8_float8 and aggregates with > 1 arg +-- EXPLAIN (COSTS OFF, VERBOSE) +-- SELECT variance(unique1::int4), sum(unique1::int8), regr_count(unique1::float8, unique1::float8) FROM tenk1; + +-- SELECT variance(unique1::int4), sum(unique1::int8), regr_count(unique1::float8, unique1::float8) FROM tenk1; + +-- ROLLBACK; + +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- test coverage for dense_rank +-- SELECT dense_rank(x) WITHIN GROUP (ORDER BY x) FROM (VALUES (1),(1),(2),(2),(3),(3)) v(x) GROUP BY (x) ORDER BY 1; + + +-- [SPARK-28664] ORDER BY in aggregate function +-- Ensure that the STRICT checks for aggregates does not take NULLness +-- of ORDER BY columns into account. See bug report around +-- 2a505161-2727-2473-7c46-591ed108ac52@email.cz +-- SELECT min(x ORDER BY y) FROM (VALUES(1, NULL)) AS d(x,y); +-- SELECT min(x ORDER BY y) FROM (VALUES(1, 2)) AS d(x,y); + +-- [SPARK-28382] Array Functions: unnest +-- check collation-sensitive matching between grouping expressions +-- select v||'a', case v||'a' when 'aa' then 1 else 0 end, count(*) +-- from unnest(array['a','b']) u(v) +-- group by v||'a' order by 1; +-- select v||'a', case when v||'a' = 'aa' then 1 else 0 end, count(*) +-- from unnest(array['a','b']) u(v) +-- group by v||'a' order by 1; + +-- Make sure that generation of HashAggregate for uniqification purposes +-- does not lead to array overflow due to unexpected duplicate hash keys +-- see CAFeeJoKKu0u+A_A9R9316djW-YW3-+Gtgvy3ju655qRHR3jtdA@mail.gmail.com +-- explain (costs off) +-- select 1 from tenk1 +-- where (hundred, thousand) in (select twothousand, twothousand from onek); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/boolean.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/boolean.sql new file mode 100644 index 000000000000..3a949c834deb --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/boolean.sql @@ -0,0 +1,281 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- BOOLEAN +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/boolean.sql + +-- +-- sanity check - if this fails go insane! +-- +SELECT 1 AS one; + + +-- ******************testing built-in type bool******************** + +-- check bool input syntax + +SELECT true AS true; + +-- [SPARK-28349] We do not need to follow PostgreSQL to support reserved words in column alias +SELECT false AS `false`; + +SELECT boolean('t') AS true; + +SELECT boolean(' f ') AS `false`; + +SELECT boolean('true') AS true; + +-- [SPARK-27923] PostgreSQL does not accept 'test' but Spark SQL accepts it and sets it to NULL +SELECT boolean('test') AS error; + +SELECT boolean('false') AS `false`; + +-- [SPARK-27923] PostgreSQL does not accept 'foo' but Spark SQL accepts it and sets it to NULL +SELECT boolean('foo') AS error; + +SELECT boolean('y') AS true; + +SELECT boolean('yes') AS true; + +-- [SPARK-27923] PostgreSQL does not accept 'yeah' but Spark SQL accepts it and sets it to NULL +SELECT boolean('yeah') AS error; + +SELECT boolean('n') AS `false`; + +SELECT boolean('no') AS `false`; + +-- [SPARK-27923] PostgreSQL does not accept 'nay' but Spark SQL accepts it and sets it to NULL +SELECT boolean('nay') AS error; + +SELECT boolean('on') AS true; + +SELECT boolean('off') AS `false`; + +SELECT boolean('of') AS `false`; + +-- [SPARK-27923] PostgreSQL does not accept 'o' but Spark SQL accepts it and sets it to NULL +SELECT boolean('o') AS error; + +-- [SPARK-27923] PostgreSQL does not accept 'on_' but Spark SQL accepts it and sets it to NULL +SELECT boolean('on_') AS error; + +-- [SPARK-27923] PostgreSQL does not accept 'off_' but Spark SQL accepts it and sets it to NULL +SELECT boolean('off_') AS error; + +SELECT boolean('1') AS true; + +-- [SPARK-27923] PostgreSQL does not accept '11' but Spark SQL accepts it and sets it to NULL +SELECT boolean('11') AS error; + +SELECT boolean('0') AS `false`; + +-- [SPARK-27923] PostgreSQL does not accept '000' but Spark SQL accepts it and sets it to NULL +SELECT boolean('000') AS error; + +-- [SPARK-27923] PostgreSQL does not accept '' but Spark SQL accepts it and sets it to NULL +SELECT boolean('') AS error; + +-- and, or, not in qualifications + +SELECT boolean('t') or boolean('f') AS true; + +SELECT boolean('t') and boolean('f') AS `false`; + +SELECT not boolean('f') AS true; + +SELECT boolean('t') = boolean('f') AS `false`; + +SELECT boolean('t') <> boolean('f') AS true; + +SELECT boolean('t') > boolean('f') AS true; + +SELECT boolean('t') >= boolean('f') AS true; + +SELECT boolean('f') < boolean('t') AS true; + +SELECT boolean('f') <= boolean('t') AS true; + +-- explicit casts to/from text +SELECT boolean(string('TrUe')) AS true, boolean(string('fAlse')) AS `false`; +SELECT boolean(string(' true ')) AS true, + boolean(string(' FALSE')) AS `false`; +SELECT string(boolean(true)) AS true, string(boolean(false)) AS `false`; + +-- [SPARK-27923] PostgreSQL does not accept ' tru e ' but Spark SQL accepts it and sets it to NULL +SELECT boolean(string(' tru e ')) AS invalid; -- error +-- [SPARK-27923] PostgreSQL does not accept '' but Spark SQL accepts it and sets it to NULL +SELECT boolean(string('')) AS invalid; -- error + +CREATE TABLE BOOLTBL1 (f1 boolean) USING parquet; + +INSERT INTO BOOLTBL1 VALUES (cast('t' as boolean)); + +INSERT INTO BOOLTBL1 VALUES (cast('True' as boolean)); + +INSERT INTO BOOLTBL1 VALUES (cast('true' as boolean)); + + +-- BOOLTBL1 should be full of true's at this point +SELECT '' AS t_3, BOOLTBL1.* FROM BOOLTBL1; + + +SELECT '' AS t_3, BOOLTBL1.* + FROM BOOLTBL1 + WHERE f1 = boolean('true'); + + +SELECT '' AS t_3, BOOLTBL1.* + FROM BOOLTBL1 + WHERE f1 <> boolean('false'); + +SELECT '' AS zero, BOOLTBL1.* + FROM BOOLTBL1 + WHERE booleq(boolean('false'), f1); + +INSERT INTO BOOLTBL1 VALUES (boolean('f')); + +SELECT '' AS f_1, BOOLTBL1.* + FROM BOOLTBL1 + WHERE f1 = boolean('false'); + + +CREATE TABLE BOOLTBL2 (f1 boolean) USING parquet; + +INSERT INTO BOOLTBL2 VALUES (boolean('f')); + +INSERT INTO BOOLTBL2 VALUES (boolean('false')); + +INSERT INTO BOOLTBL2 VALUES (boolean('False')); + +INSERT INTO BOOLTBL2 VALUES (boolean('FALSE')); + +-- [SPARK-27923] PostgreSQL does not accept 'XXX' but Spark SQL accepts it and sets it to NULL +-- This is now an invalid expression +-- For pre-v6.3 this evaluated to false - thomas 1997-10-23 +INSERT INTO BOOLTBL2 + VALUES (boolean('XXX')); + +-- BOOLTBL2 should be full of false's at this point +SELECT '' AS f_4, BOOLTBL2.* FROM BOOLTBL2; + + +SELECT '' AS tf_12, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE BOOLTBL2.f1 <> BOOLTBL1.f1; + + +SELECT '' AS tf_12, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE boolne(BOOLTBL2.f1,BOOLTBL1.f1); + + +SELECT '' AS ff_4, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE BOOLTBL2.f1 = BOOLTBL1.f1 and BOOLTBL1.f1 = boolean('false'); + + +SELECT '' AS tf_12_ff_4, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE BOOLTBL2.f1 = BOOLTBL1.f1 or BOOLTBL1.f1 = boolean('true') + ORDER BY BOOLTBL1.f1, BOOLTBL2.f1; + +-- [SPARK-27924] E061-14: Search Conditions +-- +-- SQL syntax +-- Try all combinations to ensure that we get nothing when we expect nothing +-- - thomas 2000-01-04 +-- + +SELECT '' AS True, f1 + FROM BOOLTBL1 + WHERE f1 IS TRUE; + +SELECT '' AS `Not False`, f1 + FROM BOOLTBL1 + WHERE f1 IS NOT FALSE; + +SELECT '' AS `False`, f1 + FROM BOOLTBL1 + WHERE f1 IS FALSE; + +SELECT '' AS `Not True`, f1 + FROM BOOLTBL1 + WHERE f1 IS NOT TRUE; + +SELECT '' AS `True`, f1 + FROM BOOLTBL2 + WHERE f1 IS TRUE; + +SELECT '' AS `Not False`, f1 + FROM BOOLTBL2 + WHERE f1 IS NOT FALSE; + +SELECT '' AS `False`, f1 + FROM BOOLTBL2 + WHERE f1 IS FALSE; + +SELECT '' AS `Not True`, f1 + FROM BOOLTBL2 + WHERE f1 IS NOT TRUE; + +-- +-- Tests for BooleanTest +-- +CREATE TABLE BOOLTBL3 (d string, b boolean, o int) USING parquet; +INSERT INTO BOOLTBL3 VALUES ('true', true, 1); +INSERT INTO BOOLTBL3 VALUES ('false', false, 2); +INSERT INTO BOOLTBL3 VALUES ('null', null, 3); + +-- [SPARK-27924] E061-14: Search Conditions +SELECT + d, + b IS TRUE AS istrue, + b IS NOT TRUE AS isnottrue, + b IS FALSE AS isfalse, + b IS NOT FALSE AS isnotfalse, + b IS UNKNOWN AS isunknown, + b IS NOT UNKNOWN AS isnotunknown +FROM booltbl3 ORDER BY o; + + +-- Test to make sure short-circuiting and NULL handling is +-- correct. Use a table as source to prevent constant simplification +-- to interfer. +CREATE TABLE booltbl4(isfalse boolean, istrue boolean, isnul boolean) USING parquet; +INSERT INTO booltbl4 VALUES (false, true, null); +-- \pset null '(null)' + +-- AND expression need to return null if there's any nulls and not all +-- of the value are true +SELECT istrue AND isnul AND istrue FROM booltbl4; +SELECT istrue AND istrue AND isnul FROM booltbl4; +SELECT isnul AND istrue AND istrue FROM booltbl4; +SELECT isfalse AND isnul AND istrue FROM booltbl4; +SELECT istrue AND isfalse AND isnul FROM booltbl4; +SELECT isnul AND istrue AND isfalse FROM booltbl4; + +-- OR expression need to return null if there's any nulls and none +-- of the value is true +SELECT isfalse OR isnul OR isfalse FROM booltbl4; +SELECT isfalse OR isfalse OR isnul FROM booltbl4; +SELECT isnul OR isfalse OR isfalse FROM booltbl4; +SELECT isfalse OR isnul OR istrue FROM booltbl4; +SELECT istrue OR isfalse OR isnul FROM booltbl4; +SELECT isnul OR istrue OR isfalse FROM booltbl4; + + +-- +-- Clean up +-- Many tables are retained by the regression test, but these do not seem +-- particularly useful so just get rid of them for now. +-- - thomas 1997-11-30 +-- + +DROP TABLE BOOLTBL1; + +DROP TABLE BOOLTBL2; + +DROP TABLE BOOLTBL3; + +DROP TABLE BOOLTBL4; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/case.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/case.sql new file mode 100644 index 000000000000..e8129b8cfaf6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/case.sql @@ -0,0 +1,262 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- CASE +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/case.sql +-- Test the CASE statement +-- +CREATE TABLE CASE_TBL ( + i integer, + f double +) USING parquet; + +CREATE TABLE CASE2_TBL ( + i integer, + j integer +) USING parquet; + +INSERT INTO CASE_TBL VALUES (1, 10.1); +INSERT INTO CASE_TBL VALUES (2, 20.2); +INSERT INTO CASE_TBL VALUES (3, -30.3); +INSERT INTO CASE_TBL VALUES (4, NULL); + +INSERT INTO CASE2_TBL VALUES (1, -1); +INSERT INTO CASE2_TBL VALUES (2, -2); +INSERT INTO CASE2_TBL VALUES (3, -3); +INSERT INTO CASE2_TBL VALUES (2, -4); +INSERT INTO CASE2_TBL VALUES (1, NULL); +INSERT INTO CASE2_TBL VALUES (NULL, -6); + +-- +-- Simplest examples without tables +-- + +SELECT '3' AS `One`, + CASE + WHEN 1 < 2 THEN 3 + END AS `Simple WHEN`; + +SELECT '' AS `One`, + CASE + WHEN 1 > 2 THEN 3 + END AS `Simple default`; + +SELECT '3' AS `One`, + CASE + WHEN 1 < 2 THEN 3 + ELSE 4 + END AS `Simple ELSE`; + +SELECT '4' AS `One`, + CASE + WHEN 1 > 2 THEN 3 + ELSE 4 + END AS `ELSE default`; + +SELECT '6' AS `One`, + CASE + WHEN 1 > 2 THEN 3 + WHEN 4 < 5 THEN 6 + ELSE 7 + END AS `Two WHEN with default`; + +SELECT '7' AS `None`, + CASE WHEN rand() < 0 THEN 1 + END AS `NULL on no matches`; + +-- Constant-expression folding shouldn't evaluate unreachable subexpressions +SELECT CASE WHEN 1=0 THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END; +SELECT CASE 1 WHEN 0 THEN 1/0 WHEN 1 THEN 1 ELSE 2/0 END; + +-- However we do not currently suppress folding of potentially +-- reachable subexpressions +SELECT CASE WHEN i > 100 THEN 1/0 ELSE 0 END FROM case_tbl; + +-- Test for cases involving untyped literals in test expression +SELECT CASE 'a' WHEN 'a' THEN 1 ELSE 2 END; + +-- +-- Examples of targets involving tables +-- + +SELECT '' AS `Five`, + CASE + WHEN i >= 3 THEN i + END AS `>= 3 or Null` + FROM CASE_TBL; + +SELECT '' AS `Five`, + CASE WHEN i >= 3 THEN (i + i) + ELSE i + END AS `Simplest Math` + FROM CASE_TBL; + +SELECT '' AS `Five`, i AS `Value`, + CASE WHEN (i < 0) THEN 'small' + WHEN (i = 0) THEN 'zero' + WHEN (i = 1) THEN 'one' + WHEN (i = 2) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL; + +SELECT '' AS `Five`, + CASE WHEN ((i < 0) or (i < 0)) THEN 'small' + WHEN ((i = 0) or (i = 0)) THEN 'zero' + WHEN ((i = 1) or (i = 1)) THEN 'one' + WHEN ((i = 2) or (i = 2)) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL; + +-- +-- Examples of qualifications involving tables +-- + +-- +-- NULLIF() and COALESCE() +-- Shorthand forms for typical CASE constructs +-- defined in the SQL standard. +-- + +SELECT * FROM CASE_TBL WHERE COALESCE(f,i) = 4; + +SELECT * FROM CASE_TBL WHERE NULLIF(f,i) = 2; + +SELECT COALESCE(a.f, b.i, b.j) + FROM CASE_TBL a, CASE2_TBL b; + +SELECT * + FROM CASE_TBL a, CASE2_TBL b + WHERE COALESCE(a.f, b.i, b.j) = 2; + +SELECT '' AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, + NULLIF(b.i, 4) AS `NULLIF(b.i,4)` + FROM CASE_TBL a, CASE2_TBL b; + +SELECT '' AS `Two`, * + FROM CASE_TBL a, CASE2_TBL b + WHERE COALESCE(f,b.i) = 2; + +-- We don't support update now. +-- +-- Examples of updates involving tables +-- + +-- UPDATE CASE_TBL +-- SET i = CASE WHEN i >= 3 THEN (- i) +-- ELSE (2 * i) END; + +-- SELECT * FROM CASE_TBL; + +-- UPDATE CASE_TBL +-- SET i = CASE WHEN i >= 2 THEN (2 * i) +-- ELSE (3 * i) END; + +-- SELECT * FROM CASE_TBL; + +-- UPDATE CASE_TBL +-- SET i = CASE WHEN b.i >= 2 THEN (2 * j) +-- ELSE (3 * j) END +-- FROM CASE2_TBL b +-- WHERE j = -CASE_TBL.i; + +-- SELECT * FROM CASE_TBL; + +-- +-- Nested CASE expressions +-- + +-- This test exercises a bug caused by aliasing econtext->caseValue_isNull +-- with the isNull argument of the inner CASE's CaseExpr evaluation. After +-- evaluating the vol(null) expression in the inner CASE's second WHEN-clause, +-- the isNull flag for the case test value incorrectly became true, causing +-- the third WHEN-clause not to match. The volatile function calls are needed +-- to prevent constant-folding in the planner, which would hide the bug. + +-- Wrap this in a single transaction so the transient '=' operator doesn't +-- cause problems in concurrent sessions +-- BEGIN; + +-- CREATE FUNCTION vol(text) returns text as +-- 'begin return $1; end' language plpgsql volatile; + +SELECT CASE + (CASE vol('bar') + WHEN 'foo' THEN 'it was foo!' + WHEN vol(null) THEN 'null input' + WHEN 'bar' THEN 'it was bar!' END + ) + WHEN 'it was foo!' THEN 'foo recognized' + WHEN 'it was bar!' THEN 'bar recognized' + ELSE 'unrecognized' END; + +-- We don't support the features below: +-- 1. CREATE DOMAIN ... +-- 2. CREATE OPERATOR ... +-- 3. CREATE TYPE ... + +-- In this case, we can't inline the SQL function without confusing things. +-- CREATE DOMAIN foodomain AS text; + +-- CREATE FUNCTION volfoo(text) returns foodomain as +-- 'begin return $1::foodomain; end' language plpgsql volatile; + +-- CREATE FUNCTION inline_eq(foodomain, foodomain) returns boolean as +-- 'SELECT CASE $2::text WHEN $1::text THEN true ELSE false END' language sql; + +-- CREATE OPERATOR = (procedure = inline_eq, +-- leftarg = foodomain, rightarg = foodomain); + +-- SELECT CASE volfoo('bar') WHEN 'foo'::foodomain THEN 'is foo' ELSE 'is not foo' END; + +-- ROLLBACK; + +-- Test multiple evaluation of a CASE arg that is a read/write object (#14472) +-- Wrap this in a single transaction so the transient '=' operator doesn't +-- cause problems in concurrent sessions +-- BEGIN; + +-- CREATE DOMAIN arrdomain AS int[]; + +-- CREATE FUNCTION make_ad(int,int) returns arrdomain as +-- 'declare x arrdomain; +-- begin +-- x := array[$1,$2]; +-- return x; +-- end' language plpgsql volatile; + +-- CREATE FUNCTION ad_eq(arrdomain, arrdomain) returns boolean as +-- 'begin return array_eq($1, $2); end' language plpgsql; + +-- CREATE OPERATOR = (procedure = ad_eq, +-- leftarg = arrdomain, rightarg = arrdomain); + +-- SELECT CASE make_ad(1,2) +-- WHEN array[2,4]::arrdomain THEN 'wrong' +-- WHEN array[2,5]::arrdomain THEN 'still wrong' +-- WHEN array[1,2]::arrdomain THEN 'right' +-- END; + +-- ROLLBACK; + +-- Test interaction of CASE with ArrayCoerceExpr (bug #15471) +-- BEGIN; + +-- CREATE TYPE casetestenum AS ENUM ('e', 'f', 'g'); + +-- SELECT +-- CASE 'foo'::text +-- WHEN 'foo' THEN ARRAY['a', 'b', 'c', 'd'] || enum_range(NULL::casetestenum)::text[] +-- ELSE ARRAY['x', 'y'] +-- END; + +-- ROLLBACK; + +-- +-- Clean up +-- + +DROP TABLE CASE_TBL; +DROP TABLE CASE2_TBL; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/comments.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/comments.sql new file mode 100644 index 000000000000..b4614bf2e469 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/comments.sql @@ -0,0 +1,49 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- COMMENTS +-- https://github.com/postgres/postgres/blob/REL_12_BETA3/src/test/regress/sql/comments.sql +-- + +SELECT 'trailing' AS first; -- trailing single line +SELECT /* embedded single line */ 'embedded' AS `second`; +SELECT /* both embedded and trailing single line */ 'both' AS third; -- trailing single line + +SELECT 'before multi-line' AS fourth; +--QUERY-DELIMITER-START +/* This is an example of SQL which should not execute: + * select 'multi-line'; + */ +SELECT 'after multi-line' AS fifth; +--QUERY-DELIMITER-END + +-- +-- Nested comments +-- +--QUERY-DELIMITER-START +/* +SELECT 'trailing' as x1; -- inside block comment +*/ + +/* This block comment surrounds a query which itself has a block comment... +SELECT /* embedded single line */ 'embedded' AS x2; +*/ + +SELECT -- continued after the following block comments... +/* Deeply nested comment. + This includes a single apostrophe to make sure we aren't decoding this part as a string. +SELECT 'deep nest' AS n1; +/* Second level of nesting... +SELECT 'deeper nest' as n2; +/* Third level of nesting... +SELECT 'deepest nest' as n3; +*/ +Hoo boy. Still two deep... +*/ +Now just one deep... +*/ +'deeply nested example' AS sixth; +--QUERY-DELIMITER-END +-- [SPARK-30824] Support submit sql content only contains comments. +-- /* and this is the end of the file */ diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/create_view.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/create_view.sql new file mode 100644 index 000000000000..2889941c1fcc --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/create_view.sql @@ -0,0 +1,779 @@ +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- CREATE VIEW +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/create_view.sql + +-- [SPARK-27764] Support geometric types +-- CREATE VIEW street AS +-- SELECT r.name, r.thepath, c.cname AS cname +-- FROM ONLY road r, real_city c +-- WHERE c.outline ## r.thepath; + +-- [SPARK-27764] Support geometric types +-- CREATE VIEW iexit AS +-- SELECT ih.name, ih.thepath, +-- interpt_pp(ih.thepath, r.thepath) AS exit +-- FROM ihighway ih, ramp r +-- WHERE ih.thepath ## r.thepath; + +CREATE TABLE emp ( + name string, + age int, + -- [SPARK-27764] Support geometric types + -- location point + salary int, + manager string +) USING parquet; + +CREATE VIEW toyemp AS + SELECT name, age, /* location ,*/ 12*salary AS annualsal + FROM emp; + +-- [SPARK-29659] Support COMMENT ON syntax +-- Test comments +-- COMMENT ON VIEW noview IS 'no view'; +-- COMMENT ON VIEW toyemp IS 'is a view'; +-- COMMENT ON VIEW toyemp IS NULL; + +DROP VIEW toyemp; +DROP TABLE emp; + +-- These views are left around mainly to exercise special cases in pg_dump. + +-- [SPARK-19842] Informational Referential Integrity Constraints Support in Spark +CREATE TABLE view_base_table (key int /* PRIMARY KEY */, data varchar(20)) USING PARQUET; +-- +CREATE VIEW key_dependent_view AS + SELECT * FROM view_base_table GROUP BY key; +-- +-- [SPARK-19842] Informational Referential Integrity Constraints Support in Spark +-- ALTER TABLE view_base_table DROP CONSTRAINT view_base_table_pkey; -- fails + +CREATE VIEW key_dependent_view_no_cols AS + SELECT FROM view_base_table GROUP BY key HAVING length(data) > 0; + +-- +-- CREATE OR REPLACE VIEW +-- + +CREATE TABLE viewtest_tbl (a int, b int) using parquet; +-- [SPARK-29386] Copy data between a file and a table +-- COPY viewtest_tbl FROM stdin; +-- 5 10 +-- 10 15 +-- 15 20 +-- 20 25 +-- \. +INSERT INTO viewtest_tbl VALUES (5, 10), (10, 15), (15, 20), (20, 25); + +CREATE OR REPLACE VIEW viewtest AS + SELECT * FROM viewtest_tbl; + +CREATE OR REPLACE VIEW viewtest AS + SELECT * FROM viewtest_tbl WHERE a > 10; + +SELECT * FROM viewtest; + +CREATE OR REPLACE VIEW viewtest AS + SELECT a, b FROM viewtest_tbl WHERE a > 5 ORDER BY b DESC; + +SELECT * FROM viewtest; + +-- should fail +-- [SPARK-29660] Dropping columns and changing column names/types are prohibited in VIEW definition +CREATE OR REPLACE VIEW viewtest AS + SELECT a FROM viewtest_tbl WHERE a <> 20; + +-- should fail +-- [SPARK-29660] Dropping columns and changing column names/types are prohibited in VIEW definition +CREATE OR REPLACE VIEW viewtest AS + SELECT 1, * FROM viewtest_tbl; + +-- should fail +-- [SPARK-29660] Dropping columns and changing column names/types are prohibited in VIEW definition +CREATE OR REPLACE VIEW viewtest AS + SELECT a, decimal(b) FROM viewtest_tbl; + +-- should work +CREATE OR REPLACE VIEW viewtest AS + SELECT a, b, 0 AS c FROM viewtest_tbl; + +DROP VIEW viewtest; +DROP TABLE viewtest_tbl; + +-- tests for temporary views + +-- [SPARK-29661] Support cascaded syntax in CREATE SCHEMA +-- CREATE SCHEMA temp_view_test +-- CREATE TABLE base_table (a int, id int) using parquet +-- CREATE TABLE base_table2 (a int, id int) using parquet; +CREATE SCHEMA temp_view_test; +CREATE TABLE temp_view_test.base_table (a int, id int) using parquet; +CREATE TABLE temp_view_test.base_table2 (a int, id int) using parquet; + +-- Replace SET with USE +-- SET search_path TO temp_view_test, public; +USE temp_view_test; + +-- Since Spark doesn't support CREATE TEMPORARY TABLE, we used CREATE TEMPORARY VIEW instead +-- CREATE TEMPORARY TABLE temp_table (a int, id int); +CREATE TEMPORARY VIEW temp_table AS SELECT * FROM VALUES + (1, 1) as temp_table(a, id); + +-- should be created in temp_view_test schema +CREATE VIEW v1 AS SELECT * FROM base_table; +DESC TABLE EXTENDED v1; +-- should be created in temp object schema +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW v1_temp AS SELECT * FROM temp_table; +-- should be created in temp object schema +CREATE TEMP VIEW v2_temp AS SELECT * FROM base_table; +DESC TABLE EXTENDED v2_temp; +-- should be created in temp_views schema +CREATE VIEW temp_view_test.v2 AS SELECT * FROM base_table; +DESC TABLE EXTENDED temp_view_test.v2; +-- should fail +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW temp_view_test.v3_temp AS SELECT * FROM temp_table; +-- should fail +-- [SPARK-29661] Support cascaded syntax in CREATE SCHEMA +-- CREATE SCHEMA test_view_schema +-- CREATE TEMP VIEW testview AS SELECT 1; + +-- joins: if any of the join relations are temporary, the view +-- should also be temporary + +-- should be non-temp +CREATE VIEW v3 AS + SELECT t1.a AS t1_a, t2.a AS t2_a + FROM base_table t1, base_table2 t2 + WHERE t1.id = t2.id; +DESC TABLE EXTENDED v3; +-- should be temp (one join rel is temp) +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW v4_temp AS + SELECT t1.a AS t1_a, t2.a AS t2_a + FROM base_table t1, temp_table t2 + WHERE t1.id = t2.id; +-- should be temp +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW v5_temp AS + SELECT t1.a AS t1_a, t2.a AS t2_a, t3.a AS t3_a + FROM base_table t1, base_table2 t2, temp_table t3 + WHERE t1.id = t2.id and t2.id = t3.id; + +-- subqueries +CREATE VIEW v4 AS SELECT * FROM base_table WHERE id IN (SELECT id FROM base_table2); +DESC TABLE EXTENDED v4; +CREATE VIEW v5 AS SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM base_table2) t2; +DESC TABLE EXTENDED v5; +CREATE VIEW v6 AS SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM base_table2); +DESC TABLE EXTENDED v6; +CREATE VIEW v7 AS SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM base_table2); +DESC TABLE EXTENDED v7; +CREATE VIEW v8 AS SELECT * FROM base_table WHERE EXISTS (SELECT 1); +DESC TABLE EXTENDED v8; + +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW v6_temp AS SELECT * FROM base_table WHERE id IN (SELECT id FROM temp_table); +CREATE VIEW v7_temp AS SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM temp_table) t2; +CREATE VIEW v8_temp AS SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM temp_table); +CREATE VIEW v9_temp AS SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM temp_table); + +-- a view should also be temporary if it references a temporary view +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW v10_temp AS SELECT * FROM v7_temp; +CREATE VIEW v11_temp AS SELECT t1.id, t2.a FROM base_table t1, v10_temp t2; +CREATE VIEW v12_temp AS SELECT true FROM v11_temp; + +-- [SPARK-27764] Support ANSI SQL CREATE SEQUENCE +-- a view should also be temporary if it references a temporary sequence +-- CREATE SEQUENCE seq1; +-- CREATE TEMPORARY SEQUENCE seq1_temp; +-- CREATE VIEW v9 AS SELECT seq1.is_called FROM seq1; +-- CREATE VIEW v13_temp AS SELECT seq1_temp.is_called FROM seq1_temp; + +-- Skip the tests below because of PostgreSQL specific cases +-- SELECT relname FROM pg_class +-- WHERE relname LIKE 'v_' +-- AND relnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'temp_view_test') +-- ORDER BY relname; +-- SELECT relname FROM pg_class +-- WHERE relname LIKE 'v%' +-- AND relnamespace IN (SELECT oid FROM pg_namespace WHERE nspname LIKE 'pg_temp%') +-- ORDER BY relname; + +CREATE SCHEMA testviewschm2; +-- Replace SET with USE +-- SET search_path TO testviewschm2, public; +USE testviewschm2; + +CREATE TABLE t1 (num int, name string) using parquet; +CREATE TABLE t2 (num2 int, value string) using parquet; +-- Since Spark doesn't support CREATE TEMPORARY TABLE, we used CREATE TEMPORARY VIEW instead +-- CREATE TEMP TABLE tt (num2 int, value string); +CREATE TEMP VIEW tt AS SELECT * FROM VALUES + (1, 'a') AS tt(num2, value); + +CREATE VIEW nontemp1 AS SELECT * FROM t1 CROSS JOIN t2; +DESC TABLE EXTENDED nontemp1; +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW temporal1 AS SELECT * FROM t1 CROSS JOIN tt; +CREATE VIEW nontemp2 AS SELECT * FROM t1 INNER JOIN t2 ON t1.num = t2.num2; +DESC TABLE EXTENDED nontemp2; +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW temporal2 AS SELECT * FROM t1 INNER JOIN tt ON t1.num = tt.num2; +CREATE VIEW nontemp3 AS SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2; +DESC TABLE EXTENDED nontemp3; +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW temporal3 AS SELECT * FROM t1 LEFT JOIN tt ON t1.num = tt.num2; +CREATE VIEW nontemp4 AS SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 AND t2.value = 'xxx'; +DESC TABLE EXTENDED nontemp4; +-- [SPARK-29628] Forcibly create a temporary view in CREATE VIEW if referencing a temporary view +CREATE VIEW temporal4 AS SELECT * FROM t1 LEFT JOIN tt ON t1.num = tt.num2 AND tt.value = 'xxx'; +CREATE VIEW temporal5 AS SELECT * FROM t1 WHERE num IN (SELECT num FROM t1 WHERE EXISTS (SELECT 1 FROM tt)); + +-- Skip the tests below because of PostgreSQL specific cases +-- SELECT relname FROM pg_class +-- WHERE relname LIKE 'nontemp%' +-- AND relnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'testviewschm2') +-- ORDER BY relname; +-- SELECT relname FROM pg_class +-- WHERE relname LIKE 'temporal%' +-- AND relnamespace IN (SELECT oid FROM pg_namespace WHERE nspname LIKE 'pg_temp%') +-- ORDER BY relname; + +CREATE TABLE tbl1 ( a int, b int) using parquet; +CREATE TABLE tbl2 (c int, d int) using parquet; +CREATE TABLE tbl3 (e int, f int) using parquet; +CREATE TABLE tbl4 (g int, h int) using parquet; +-- Since Spark doesn't support CREATE TEMPORARY TABLE, we used CREATE TABLE instead +-- CREATE TEMP TABLE tmptbl (i int, j int); +CREATE TABLE tmptbl (i int, j int) using parquet; +INSERT INTO tmptbl VALUES (1, 1); + +--Should be in testviewschm2 +CREATE VIEW pubview AS SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f); +DESC TABLE EXTENDED pubview; + +-- Skip the test below because of PostgreSQL specific cases +-- SELECT count(*) FROM pg_class where relname = 'pubview' +-- AND relnamespace IN (SELECT OID FROM pg_namespace WHERE nspname = 'testviewschm2'); + +--Should be in temp object schema +CREATE VIEW mytempview AS SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) +AND NOT EXISTS (SELECT g FROM tbl4 LEFT JOIN tmptbl ON tbl4.h = tmptbl.j); +DESC TABLE EXTENDED mytempview; + +-- Skip the test below because of PostgreSQL specific cases +-- SELECT count(*) FROM pg_class where relname LIKE 'mytempview' +-- And relnamespace IN (SELECT OID FROM pg_namespace WHERE nspname LIKE 'pg_temp%'); + +-- +-- CREATE VIEW and WITH(...) clause +-- CREATE VIEW mysecview1 +-- AS SELECT * FROM tbl1 WHERE a = 0; +-- +-- Skip the tests below because Spark doesn't support `WITH options` +-- CREATE VIEW mysecview2 WITH (security_barrier=true) +-- AS SELECT * FROM tbl1 WHERE a > 0; +-- CREATE VIEW mysecview3 WITH (security_barrier=false) +-- AS SELECT * FROM tbl1 WHERE a < 0; +-- CREATE VIEW mysecview4 WITH (security_barrier) +-- AS SELECT * FROM tbl1 WHERE a <> 0; +-- Spark cannot support options in WITH clause +-- CREATE VIEW mysecview5 WITH (security_barrier=100) -- Error +-- AS SELECT * FROM tbl1 WHERE a > 100; +-- CREATE VIEW mysecview6 WITH (invalid_option) -- Error +-- AS SELECT * FROM tbl1 WHERE a < 100; +-- Skip the test below because of PostgreSQL specific cases +-- SELECT relname, relkind, reloptions FROM pg_class +-- WHERE oid in ('mysecview1'::regclass, 'mysecview2'::regclass, +-- 'mysecview3'::regclass, 'mysecview4'::regclass) +-- ORDER BY relname; + +-- CREATE OR REPLACE VIEW mysecview1 +-- AS SELECT * FROM tbl1 WHERE a = 256; +-- CREATE OR REPLACE VIEW mysecview2 +-- AS SELECT * FROM tbl1 WHERE a > 256; +-- CREATE OR REPLACE VIEW mysecview3 WITH (security_barrier=true) +-- AS SELECT * FROM tbl1 WHERE a < 256; +-- CREATE OR REPLACE VIEW mysecview4 WITH (security_barrier=false) +-- AS SELECT * FROM tbl1 WHERE a <> 256; +-- Skip the test below because of PostgreSQL specific cases +-- SELECT relname, relkind, reloptions FROM pg_class +-- WHERE oid in ('mysecview1'::regclass, 'mysecview2'::regclass, +-- 'mysecview3'::regclass, 'mysecview4'::regclass) +-- ORDER BY relname; + +-- Check that unknown literals are converted to "text" in CREATE VIEW, +-- so that we don't end up with unknown-type columns. + +-- Skip the tests below because of PostgreSQL specific cases +-- CREATE VIEW unspecified_types AS +-- SELECT 42 as i, 42.5 as num, 'foo' as u, 'foo'::unknown as u2, null as n; +-- \d+ unspecified_types +-- SELECT * FROM unspecified_types; + +-- This test checks that proper typmods are assigned in a multi-row VALUES + +CREATE VIEW tt1 AS + SELECT * FROM ( + VALUES + ('abc', '0123456789', 42, 'abcd'), + ('0123456789', 'abc', 42.12, 'abc') + ) vv(a,b,c,d); +-- Replace the PostgreSQL meta command `\d` with `DESC` +-- \d+ tt1 +SELECT * FROM tt1; +SELECT string(a) FROM tt1; +DROP VIEW tt1; + +-- Test view decompilation in the face of relation renaming conflicts + +CREATE TABLE tt1 (f1 int, f2 int, f3 string) using parquet; +CREATE TABLE tx1 (x1 int, x2 int, x3 string) using parquet; +CREATE TABLE temp_view_test.tt1 (y1 int, f2 int, f3 string) using parquet; + +CREATE VIEW aliased_view_1 AS + select * from tt1 + where exists (select 1 from tx1 where tt1.f1 = tx1.x1); +CREATE VIEW aliased_view_2 AS + select * from tt1 a1 + where exists (select 1 from tx1 where a1.f1 = tx1.x1); +CREATE VIEW aliased_view_3 AS + select * from tt1 + where exists (select 1 from tx1 a2 where tt1.f1 = a2.x1); +CREATE VIEW aliased_view_4 AS + select * from temp_view_test.tt1 + where exists (select 1 from tt1 where temp_view_test.tt1.y1 = tt1.f1); + +-- Replace the PostgreSQL meta command `\d` with `DESC` +-- \d+ aliased_view_1 +DESC TABLE aliased_view_1; +-- \d+ aliased_view_2 +DESC TABLE aliased_view_2; +-- \d+ aliased_view_3 +DESC TABLE aliased_view_3; +-- \d+ aliased_view_4 +DESC TABLE aliased_view_4; + +ALTER TABLE tx1 RENAME TO a1; + +-- Replace the PostgreSQL meta command `\d` with `DESC` +-- \d+ aliased_view_1 +DESC TABLE aliased_view_1; +-- \d+ aliased_view_2 +DESC TABLE aliased_view_2; +-- \d+ aliased_view_3 +DESC TABLE aliased_view_3; +-- \d+ aliased_view_4 +DESC TABLE aliased_view_4; + +ALTER TABLE tt1 RENAME TO a2; + +-- Replace the PostgreSQL meta command `\d` with `DESC` +-- \d+ aliased_view_1 +DESC TABLE aliased_view_1; +-- \d+ aliased_view_2 +DESC TABLE aliased_view_2; +-- \d+ aliased_view_3 +DESC TABLE aliased_view_3; +-- \d+ aliased_view_4 +DESC TABLE aliased_view_4; + +ALTER TABLE a1 RENAME TO tt1; + +-- Replace the PostgreSQL meta command `\d` with `DESC` +-- \d+ aliased_view_1 +DESC TABLE aliased_view_1; +-- \d+ aliased_view_2 +DESC TABLE aliased_view_2; +-- \d+ aliased_view_3 +DESC TABLE aliased_view_3; +-- \d+ aliased_view_4 +DESC TABLE aliased_view_4; + +ALTER TABLE a2 RENAME TO tx1; +-- [SPARK-29632] Support ALTER TABLE [relname] SET SCHEMA [dbname] +-- ALTER TABLE tx1 SET SCHEMA temp_view_test; + +-- \d+ aliased_view_1 +-- \d+ aliased_view_2 +-- \d+ aliased_view_3 +-- \d+ aliased_view_4 + +-- [SPARK-29632] Support ALTER TABLE [relname] SET SCHEMA [dbname] +-- ALTER TABLE temp_view_test.tt1 RENAME TO tmp1; +-- ALTER TABLE temp_view_test.tmp1 SET SCHEMA testviewschm2; +-- ALTER TABLE tmp1 RENAME TO tx1; + +-- Replace the PostgreSQL meta command `\d` with `DESC` +-- \d+ aliased_view_1 +-- \d+ aliased_view_2 +-- \d+ aliased_view_3 +-- \d+ aliased_view_4 + +-- Test aliasing of joins + +create view view_of_joins as +select * from + (select * from (tbl1 cross join tbl2) same) ss, + (tbl3 cross join tbl4) same; + +-- Replace the PostgreSQL meta command `\d` with `DESC` +-- \d+ view_of_joins + +-- Test view decompilation in the face of column addition/deletion/renaming + +create table tt2 (a int, b int, c int) using parquet; +create table tt3 (ax bigint, b short, c decimal) using parquet; +create table tt4 (ay int, b int, q int) using parquet; + +create view v1 as select * from tt2 natural join tt3; +create view v1a as select * from (tt2 natural join tt3) j; +create view v2 as select * from tt2 join tt3 using (b,c) join tt4 using (b); +create view v2a as select * from (tt2 join tt3 using (b,c) join tt4 using (b)) j; +create view v3 as select * from tt2 join tt3 using (b,c) full join tt4 using (b); + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('v1', true); +DESC TABLE v1; +-- select pg_get_viewdef('v1a', true); +DESC TABLE v1a; +-- select pg_get_viewdef('v2', true); +DESC TABLE v2; +-- select pg_get_viewdef('v2a', true); +DESC TABLE v2a; +-- select pg_get_viewdef('v3', true); +DESC TABLE v3; + +alter table tt2 add column d int; +alter table tt2 add column e int; + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('v1', true); +DESC TABLE v1; +-- select pg_get_viewdef('v1a', true); +DESC TABLE v1a; +-- select pg_get_viewdef('v2', true); +DESC TABLE v2; +-- select pg_get_viewdef('v2a', true); +DESC TABLE v2a; +-- select pg_get_viewdef('v3', true); +DESC TABLE v3; + +-- [SPARK-27764] Make COLUMN optional in ALTER TABLE +-- [SPARK-27589] Spark file source V2 (For supporting RENAME COLUMN in ALTER TABLE) +-- alter table tt3 rename c to d; +drop table tt3; +create table tt3 (ax bigint, b short, d decimal) using parquet; + +-- select pg_get_viewdef('v1', true); +-- select pg_get_viewdef('v1a', true); +-- select pg_get_viewdef('v2', true); +-- select pg_get_viewdef('v2a', true); +-- select pg_get_viewdef('v3', true); + +alter table tt3 add column c int; +alter table tt3 add column e int; + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('v1', true); +DESC TABLE v1; +-- select pg_get_viewdef('v1a', true); +DESC TABLE v1a; +-- select pg_get_viewdef('v2', true); +DESC TABLE v2; +-- select pg_get_viewdef('v2a', true); +DESC TABLE v2a; +-- select pg_get_viewdef('v3', true); +DESC TABLE v3; + +-- [SPARK-27589] Spark file source V2 (For supporting DROP COLUMN in ALTER TABLE) +-- alter table tt2 drop column d; + +-- select pg_get_viewdef('v1', true); +-- select pg_get_viewdef('v1a', true); +-- select pg_get_viewdef('v2', true); +-- select pg_get_viewdef('v2a', true); +-- select pg_get_viewdef('v3', true); + +create table tt5 (a int, b int) using parquet; +create table tt6 (c int, d int) using parquet; +create view vv1 as select * from (tt5 cross join tt6) j(aa,bb,cc,dd); +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv1', true); +DESC TABLE vv1; +alter table tt5 add column c int; +-- select pg_get_viewdef('vv1', true); +DESC TABLE vv1; +alter table tt5 add column cc int; +-- select pg_get_viewdef('vv1', true); +DESC TABLE vv1; +-- [SPARK-27589] Spark file source V2 (For supporting DROP COLUMN in ALTER TABLE) +-- alter table tt5 drop column c; +-- select pg_get_viewdef('vv1', true); + +-- Unnamed FULL JOIN USING is lots of fun too + +-- [SPARK-27589] Spark file source V2 (For supporting DROP COLUMN in ALTER TABLE) +create table tt7 (x int, /* xx int, */ y int) using parquet; +-- alter table tt7 drop column xx; +create table tt8 (x int, z int) using parquet; + +create view vv2 as +select * from (values(1,2,3,4,5)) v(a,b,c,d,e) +union all +select * from tt7 full join tt8 using (x), tt8 tt8x; + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv2', true); +DESC TABLE vv2; + +create view vv3 as +select * from (values(1,2,3,4,5,6)) v(a,b,c,x,e,f) +union all +select * from + tt7 full join tt8 using (x), + tt7 tt7x full join tt8 tt8x using (x); + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv3', true); +DESC TABLE vv3; + +create view vv4 as +select * from (values(1,2,3,4,5,6,7)) v(a,b,c,x,e,f,g) +union all +select * from + tt7 full join tt8 using (x), + tt7 tt7x full join tt8 tt8x using (x) full join tt8 tt8y using (x); + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv4', true); +DESC TABLE vv4; + +alter table tt7 add column zz int; +alter table tt7 add column z int; +-- [SPARK-27589] Spark file source V2 (For supporting DROP COLUMN in ALTER TABLE) +-- alter table tt7 drop column zz; +alter table tt8 add column z2 int; + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv2', true); +DESC TABLE vv2; +-- select pg_get_viewdef('vv3', true); +DESC TABLE vv3; +-- select pg_get_viewdef('vv4', true); +DESC TABLE vv4; + +-- Implicit coercions in a JOIN USING create issues similar to FULL JOIN + +-- [SPARK-27589] Spark file source V2 (For supporting DROP COLUMN in ALTER TABLE) +create table tt7a (x date, /* xx int, */ y int) using parquet; +-- alter table tt7a drop column xx; +create table tt8a (x timestamp, z int) using parquet; + +-- To pass the query, added exact column names in the select stmt +create view vv2a as +select * from (values(now(),2,3,now(),5)) v(a,b,c,d,e) +union all +select * from tt7a left join tt8a using (x), tt8a tt8ax; + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv4', true); +DESC TABLE vv4; +-- select pg_get_viewdef('vv2a', true); +DESC TABLE vv2a; + +-- +-- Also check dropping a column that existed when the view was made +-- + +create table tt9 (x int, xx int, y int) using parquet; +create table tt10 (x int, z int) using parquet; + +create view vv5 as select x,y,z from tt9 join tt10 using(x); + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv5', true); +DESC TABLE vv5; + +-- [SPARK-27589] Spark file source V2 (For supporting DROP COLUMN in ALTER TABLE) +-- alter table tt9 drop column xx; + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv5', true); +DESC TABLE vv5; + +-- +-- Another corner case is that we might add a column to a table below a +-- JOIN USING, and thereby make the USING column name ambiguous +-- + +create table tt11 (x int, y int) using parquet; +create table tt12 (x int, z int) using parquet; +create table tt13 (z int, q int) using parquet; + +create view vv6 as select x,y,z,q from + (tt11 join tt12 using(x)) join tt13 using(z); + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv6', true); +DESC TABLE vv6; + +alter table tt11 add column z int; + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('vv6', true); +DESC TABLE vv6; + +-- +-- Check cases involving dropped/altered columns in a function's rowtype result +-- + +-- Skip the tests below because Spark doesn't support PostgreSQL-specific UDFs/transactions +-- create table tt14t (f1 text, f2 text, f3 text, f4 text); +-- insert into tt14t values('foo', 'bar', 'baz', '42'); +-- +-- alter table tt14t drop column f2; +-- +-- create function tt14f() returns setof tt14t as +-- $$ +-- declare +-- rec1 record; +-- begin +-- for rec1 in select * from tt14t +-- loop +-- return next rec1; +-- end loop; +-- end; +-- $$ +-- language plpgsql; +-- +-- create view tt14v as select t.* from tt14f() t; +-- +-- select pg_get_viewdef('tt14v', true); +-- select * from tt14v; +-- +-- begin; +-- +-- -- this perhaps should be rejected, but it isn't: +-- alter table tt14t drop column f3; +-- +-- -- f3 is still in the view ... +-- select pg_get_viewdef('tt14v', true); +-- -- but will fail at execution +-- select f1, f4 from tt14v; +-- select * from tt14v; +-- +-- rollback; +-- +-- begin; +-- +-- -- this perhaps should be rejected, but it isn't: +-- alter table tt14t alter column f4 type integer using f4::integer; +-- +-- -- f4 is still in the view ... +-- select pg_get_viewdef('tt14v', true); +-- -- but will fail at execution +-- select f1, f3 from tt14v; +-- select * from tt14v; +-- +-- rollback; + +-- check display of whole-row variables in some corner cases + +-- Skip the tests below because we do not support creating types +-- create type nestedcomposite as (x int8_tbl); +-- create view tt15v as select row(i)::nestedcomposite from int8_tbl i; +-- select * from tt15v; +-- select pg_get_viewdef('tt15v', true); +-- select row(i.*::int8_tbl)::nestedcomposite from int8_tbl i; +-- +-- create view tt16v as select * from int8_tbl i, lateral(values(i)) ss; +-- select * from tt16v; +-- select pg_get_viewdef('tt16v', true); +-- select * from int8_tbl i, lateral(values(i.*::int8_tbl)) ss; +-- +-- create view tt17v as select * from int8_tbl i where i in (values(i)); +-- select * from tt17v; +-- select pg_get_viewdef('tt17v', true); +-- select * from int8_tbl i where i.* in (values(i.*::int8_tbl)); + +-- check unique-ification of overlength names + +CREATE TABLE int8_tbl (q1 int, q2 int) USING parquet; + +create view tt18v as + select * from int8_tbl xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxy + union all + select * from int8_tbl xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxz; +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('tt18v', true); +DESC TABLE tt18v; +-- explain (costs off) select * from tt18v; + +-- check display of ScalarArrayOp with a sub-select + +-- Skip the tests below because of PostgreSQL specific cases +-- select 'foo'::text = any(array['abc','def','foo']::text[]); +-- select 'foo'::text = any((select array['abc','def','foo']::text[])); -- fail +-- select 'foo'::text = any((select array['abc','def','foo']::text[])::text[]); +-- +-- create view tt19v as +-- select 'foo'::text = any(array['abc','def','foo']::text[]) c1, +-- 'foo'::text = any((select array['abc','def','foo']::text[])::text[]) c2; +-- select pg_get_viewdef('tt19v', true); + +-- check display of assorted RTE_FUNCTION expressions + +-- [SPARK-28682] ANSI SQL: Collation Support +-- create view tt20v as +-- select * from +-- coalesce(1,2) as c, +-- collation for ('x'::text) col, +-- current_date as d, +-- localtimestamp(3) as t, +-- cast(1+2 as int4) as i4, +-- cast(1+2 as int8) as i8; +-- select pg_get_viewdef('tt20v', true); + +-- corner cases with empty join conditions + +create view tt21v as +select * from tt5 natural inner join tt6; +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('tt21v', true); +DESC TABLE tt21v; + +create view tt22v as +select * from tt5 natural left join tt6; +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('tt22v', true); +DESC TABLE tt22v; + +-- check handling of views with immediately-renamed columns + +create view tt23v (col_a, col_b) as +select q1 as other_name1, q2 as other_name2 from int8_tbl +union +select 42, 43; + +-- Replace `pg_get_viewdef` with `DESC` +-- select pg_get_viewdef('tt23v', true); +DESC TABLE tt23v; +-- Skip the test below because of PostgreSQL specific cases +-- select pg_get_ruledef(oid, true) from pg_rewrite +-- where ev_class = 'tt23v'::regclass and ev_type = '1'; + +-- clean up all the random objects we made above +DROP SCHEMA temp_view_test CASCADE; +DROP SCHEMA testviewschm2 CASCADE; + +DROP VIEW temp_table; +DROP VIEW tt; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/date.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/date.sql new file mode 100644 index 000000000000..69851080847b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/date.sql @@ -0,0 +1,362 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- DATE +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/date.sql + +CREATE TABLE DATE_TBL (f1 date) USING parquet; + +-- PostgreSQL implicitly casts string literals to data with date types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO DATE_TBL VALUES (date('1957-04-09')); +INSERT INTO DATE_TBL VALUES (date('1957-06-13')); +INSERT INTO DATE_TBL VALUES (date('1996-02-28')); +INSERT INTO DATE_TBL VALUES (date('1996-02-29')); +INSERT INTO DATE_TBL VALUES (date('1996-03-01')); +INSERT INTO DATE_TBL VALUES (date('1996-03-02')); +INSERT INTO DATE_TBL VALUES (date('1997-02-28')); +-- [SPARK-27923] Skip invalid date: 1997-02-29 +-- INSERT INTO DATE_TBL VALUES ('1997-02-29')); +INSERT INTO DATE_TBL VALUES (date('1997-03-01')); +INSERT INTO DATE_TBL VALUES (date('1997-03-02')); +INSERT INTO DATE_TBL VALUES (date('2000-04-01')); +INSERT INTO DATE_TBL VALUES (date('2000-04-02')); +INSERT INTO DATE_TBL VALUES (date('2000-04-03')); +INSERT INTO DATE_TBL VALUES (date('2038-04-08')); +INSERT INTO DATE_TBL VALUES (date('2039-04-09')); +INSERT INTO DATE_TBL VALUES (date('2040-04-10')); + +SELECT f1 AS `Fifteen` FROM DATE_TBL; + +SELECT f1 AS `Nine` FROM DATE_TBL WHERE f1 < '2000-01-01'; + +SELECT f1 AS `Three` FROM DATE_TBL + WHERE f1 BETWEEN '2000-01-01' AND '2001-01-01'; + +-- Skip the formats that we do not supported. Please check [SPARK-8995] for all supported formats +-- +-- Check all the documented input formats +-- +-- [SPARK-28259] Date/Time Output Styles and Date Order Conventions +-- SET datestyle TO iso; -- display results in ISO + +-- SET datestyle TO ymd; + +-- SELECT date 'January 8, 1999'; +SELECT date '1999-01-08'; +SELECT date '1999-01-18'; +-- SELECT date '1/8/1999'; +-- SELECT date '1/18/1999'; +-- SELECT date '18/1/1999'; +-- SELECT date '01/02/03'; +-- SELECT date '19990108'; +-- SELECT date '990108'; +-- SELECT date '1999.008'; +-- SELECT date 'J2451187'; +-- SELECT date 'January 8, 99 BC'; + +-- SELECT date '99-Jan-08'; +-- SELECT date '1999-Jan-08'; +-- SELECT date '08-Jan-99'; +-- SELECT date '08-Jan-1999'; +-- SELECT date 'Jan-08-99'; +-- SELECT date 'Jan-08-1999'; +-- SELECT date '99-08-Jan'; +-- SELECT date '1999-08-Jan'; + +-- SELECT date '99 Jan 08'; +SELECT date '1999 Jan 08'; +-- SELECT date '08 Jan 99'; +-- SELECT date '08 Jan 1999'; +-- SELECT date 'Jan 08 99'; +-- SELECT date 'Jan 08 1999'; +-- SELECT date '99 08 Jan'; +SELECT date '1999 08 Jan'; + +-- SELECT date '99-01-08'; +SELECT date '1999-01-08'; +-- SELECT date '08-01-99'; +-- SELECT date '08-01-1999'; +-- SELECT date '01-08-99'; +-- SELECT date '01-08-1999'; +-- SELECT date '99-08-01'; +SELECT date '1999-08-01'; + +-- SELECT date '99 01 08'; +SELECT date '1999 01 08'; +-- SELECT date '08 01 99'; +-- SELECT date '08 01 1999'; +-- SELECT date '01 08 99'; +-- SELECT date '01 08 1999'; +-- SELECT date '99 08 01'; +SELECT date '1999 08 01'; + +-- SET datestyle TO dmy; + +-- SELECT date 'January 8, 1999'; +SELECT date '1999-01-08'; +-- SELECT date '1999-01-18'; +-- SELECT date '1/8/1999'; +-- SELECT date '1/18/1999'; +-- SELECT date '18/1/1999'; +-- SELECT date '01/02/03'; +-- SELECT date '19990108'; +-- SELECT date '990108'; +-- SELECT date '1999.008'; +-- SELECT date 'J2451187'; +-- SELECT date 'January 8, 99 BC'; + +-- SELECT date '99-Jan-08'; +-- SELECT date '1999-Jan-08'; +-- SELECT date '08-Jan-99'; +-- SELECT date '08-Jan-1999'; +-- SELECT date 'Jan-08-99'; +-- SELECT date 'Jan-08-1999'; +-- SELECT date '99-08-Jan'; +-- SELECT date '1999-08-Jan'; + +-- SELECT date '99 Jan 08'; +SELECT date '1999 Jan 08'; +-- SELECT date '08 Jan 99'; +-- SELECT date '08 Jan 1999'; +-- SELECT date 'Jan 08 99'; +-- SELECT date 'Jan 08 1999'; +-- SELECT date '99 08 Jan'; +SELECT date '1999 08 Jan'; + +-- SELECT date '99-01-08'; +SELECT date '1999-01-08'; +-- SELECT date '08-01-99'; +-- SELECT date '08-01-1999'; +-- SELECT date '01-08-99'; +-- SELECT date '01-08-1999'; +-- SELECT date '99-08-01'; +SELECT date '1999-08-01'; + +-- SELECT date '99 01 08'; +SELECT date '1999 01 08'; +-- SELECT date '08 01 99'; +-- SELECT date '08 01 1999'; +-- SELECT date '01 08 99'; +-- SELECT date '01 08 1999'; +-- SELECT date '99 08 01'; +SELECT date '1999 08 01'; + +-- SET datestyle TO mdy; + +-- SELECT date 'January 8, 1999'; +SELECT date '1999-01-08'; +SELECT date '1999-01-18'; +-- SELECT date '1/8/1999'; +-- SELECT date '1/18/1999'; +-- SELECT date '18/1/1999'; +-- SELECT date '01/02/03'; +-- SELECT date '19990108'; +-- SELECT date '990108'; +-- SELECT date '1999.008'; +-- SELECT date 'J2451187'; +-- SELECT date 'January 8, 99 BC'; + +-- SELECT date '99-Jan-08'; +-- SELECT date '1999-Jan-08'; +-- SELECT date '08-Jan-99'; +-- SELECT date '08-Jan-1999'; +-- SELECT date 'Jan-08-99'; +-- SELECT date 'Jan-08-1999'; +-- SELECT date '99-08-Jan'; +-- SELECT date '1999-08-Jan'; + +-- SELECT date '99 Jan 08'; +SELECT date '1999 Jan 08'; +-- SELECT date '08 Jan 99'; +-- SELECT date '08 Jan 1999'; +-- SELECT date 'Jan 08 99'; +-- SELECT date 'Jan 08 1999'; +-- SELECT date '99 08 Jan'; +SELECT date '1999 08 Jan'; + +-- SELECT date '99-01-08'; +SELECT date '1999-01-08'; +-- SELECT date '08-01-99'; +-- SELECT date '08-01-1999'; +-- SELECT date '01-08-99'; +-- SELECT date '01-08-1999'; +-- SELECT date '99-08-01'; +SELECT date '1999-08-01'; + +-- SELECT date '99 01 08'; +SELECT date '1999 01 08'; +-- SELECT date '08 01 99'; +-- SELECT date '08 01 1999'; +-- SELECT date '01 08 99'; +-- SELECT date '01 08 1999'; +-- SELECT date '99 08 01'; +SELECT date '1999 08 01'; + +-- [SPARK-28253] Date type have different low value and high value +-- Check upper and lower limits of date range +SELECT date '4714-11-24 BC'; +SELECT date '4714-11-23 BC'; -- out of range +SELECT date '5874897-12-31'; +SELECT date '5874898-01-01'; -- out of range + +-- RESET datestyle; + +-- +-- Simple math +-- Leave most of it for the horology tests +-- + +SELECT f1 - date '2000-01-01' AS `Days From 2K` FROM DATE_TBL; + +SELECT f1 - date 'epoch' AS `Days From Epoch` FROM DATE_TBL; + +SELECT date 'yesterday' - date 'today' AS `One day`; + +SELECT date 'today' - date 'tomorrow' AS `One day`; + +SELECT date 'yesterday' - date 'tomorrow' AS `Two days`; + +SELECT date 'tomorrow' - date 'today' AS `One day`; + +SELECT date 'today' - date 'yesterday' AS `One day`; + +SELECT date 'tomorrow' - date 'yesterday' AS `Two days`; + +-- [SPARK-28017] Enhance date EXTRACT +-- +-- test extract! +-- +-- epoch +-- +-- SELECT EXTRACT(EPOCH FROM DATE '1970-01-01'); -- 0 +-- SELECT EXTRACT(EPOCH FROM TIMESTAMP '1970-01-01'); -- 0 +-- SELECT EXTRACT(EPOCH FROM TIMESTAMPTZ '1970-01-01+00'); -- 0 +-- +-- century +-- +-- SELECT EXTRACT(CENTURY FROM TO_DATE('0101-12-31 BC', 'yyyy-MM-dd G')); -- -2 +-- SELECT EXTRACT(CENTURY FROM TO_DATE('0100-12-31 BC', 'yyyy-MM-dd G')); -- -1 +-- SELECT EXTRACT(CENTURY FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')); -- -1 +-- SELECT EXTRACT(CENTURY FROM DATE '0001-01-01'); -- 1 +-- SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD'); -- 1 +-- SELECT EXTRACT(CENTURY FROM DATE '1900-12-31'); -- 19 +-- SELECT EXTRACT(CENTURY FROM DATE '1901-01-01'); -- 20 +-- SELECT EXTRACT(CENTURY FROM DATE '2000-12-31'); -- 20 +-- SELECT EXTRACT(CENTURY FROM DATE '2001-01-01'); -- 21 +-- SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True; -- true +-- +-- millennium +-- +-- SELECT EXTRACT(MILLENNIUM FROM TO_DATE('0001-12-31 BC', 'yyyy-MM-dd G')); -- -1 +-- SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD'); -- 1 +-- SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31'); -- 1 +-- SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01'); -- 2 +-- SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31'); -- 2 +-- SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01'); -- 3 +-- next test to be fixed on the turn of the next millennium;-) +-- SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE); -- 3 +-- +-- decade +-- +-- SELECT EXTRACT(DECADE FROM DATE '1994-12-25'); -- 199 +-- SELECT EXTRACT(DECADE FROM DATE '0010-01-01'); -- 1 +-- SELECT EXTRACT(DECADE FROM DATE '0009-12-31'); -- 0 +-- SELECT EXTRACT(DECADE FROM TO_DATE('0001-01-01 BC', 'yyyy-MM-dd G')); -- 0 +-- SELECT EXTRACT(DECADE FROM TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')); -- -1 +-- SELECT EXTRACT(DECADE FROM TO_DATE('0011-01-01 BC', 'yyyy-MM-dd G')); -- -1 +-- SELECT EXTRACT(DECADE FROM TO_DATE('0012-12-31 BC', 'yyyy-MM-dd G')); -- -2 +-- +-- some other types: +-- +-- on a timestamp. +-- SELECT EXTRACT(CENTURY FROM NOW())>=21 AS True; -- true +-- SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000'); -- 20 +-- on an interval +-- SELECT EXTRACT(CENTURY FROM INTERVAL '100 y'); -- 1 +-- SELECT EXTRACT(CENTURY FROM INTERVAL '99 y'); -- 0 +-- SELECT EXTRACT(CENTURY FROM INTERVAL '-99 y'); -- 0 +-- SELECT EXTRACT(CENTURY FROM INTERVAL '-100 y'); -- -1 +-- +-- test trunc function! +-- SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000'); -- 1001 +-- SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20'); -- 1001-01-01 +-- SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000'); -- 1901 +-- SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20'); -- 1901 +-- SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10'); -- 2001-01-01 +-- SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04'); -- 0001-01-01 +-- SELECT DATE_TRUNC('CENTURY', TO_DATE('0055-08-10 BC', 'yyyy-MM-dd G')); -- 0100-01-01 BC +-- SELECT DATE_TRUNC('DECADE', DATE '1993-12-25'); -- 1990-01-01 +-- SELECT DATE_TRUNC('DECADE', DATE '0004-12-25'); -- 0001-01-01 BC +-- SELECT DATE_TRUNC('DECADE', TO_DATE('0002-12-31 BC', 'yyyy-MM-dd G')); -- 0011-01-01 BC + +-- [SPARK-29006] Support special date/timestamp values `infinity`/`-infinity` +-- +-- test infinity +-- +-- select 'infinity'::date, '-infinity'::date; +-- select 'infinity'::date > 'today'::date as t; +-- select '-infinity'::date < 'today'::date as t; +-- select isfinite('infinity'::date), isfinite('-infinity'::date), isfinite('today'::date); +-- +-- oscillating fields from non-finite date/timestamptz: +-- +-- SELECT EXTRACT(HOUR FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(HOUR FROM DATE '-infinity'); -- NULL +-- SELECT EXTRACT(HOUR FROM TIMESTAMP 'infinity'); -- NULL +-- SELECT EXTRACT(HOUR FROM TIMESTAMP '-infinity'); -- NULL +-- SELECT EXTRACT(HOUR FROM TIMESTAMPTZ 'infinity'); -- NULL +-- SELECT EXTRACT(HOUR FROM TIMESTAMPTZ '-infinity'); -- NULL +-- all possible fields +-- SELECT EXTRACT(MICROSECONDS FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(MILLISECONDS FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(SECOND FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(MINUTE FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(HOUR FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(DAY FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(MONTH FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(QUARTER FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(WEEK FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(DOW FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(ISODOW FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(DOY FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(TIMEZONE FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(TIMEZONE_M FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(TIMEZONE_H FROM DATE 'infinity'); -- NULL +-- +-- monotonic fields from non-finite date/timestamptz: +-- +-- SELECT EXTRACT(EPOCH FROM DATE 'infinity'); -- Infinity +-- SELECT EXTRACT(EPOCH FROM DATE '-infinity'); -- -Infinity +-- SELECT EXTRACT(EPOCH FROM TIMESTAMP 'infinity'); -- Infinity +-- SELECT EXTRACT(EPOCH FROM TIMESTAMP '-infinity'); -- -Infinity +-- SELECT EXTRACT(EPOCH FROM TIMESTAMPTZ 'infinity'); -- Infinity +-- SELECT EXTRACT(EPOCH FROM TIMESTAMPTZ '-infinity'); -- -Infinity +-- all possible fields +-- SELECT EXTRACT(YEAR FROM DATE 'infinity'); -- Infinity +-- SELECT EXTRACT(DECADE FROM DATE 'infinity'); -- Infinity +-- SELECT EXTRACT(CENTURY FROM DATE 'infinity'); -- Infinity +-- SELECT EXTRACT(MILLENNIUM FROM DATE 'infinity'); -- Infinity +-- SELECT EXTRACT(JULIAN FROM DATE 'infinity'); -- Infinity +-- SELECT EXTRACT(ISOYEAR FROM DATE 'infinity'); -- Infinity +-- SELECT EXTRACT(EPOCH FROM DATE 'infinity'); -- Infinity +-- +-- wrong fields from non-finite date: +-- +-- SELECT EXTRACT(MICROSEC FROM DATE 'infinity'); -- ERROR: timestamp units "microsec" not recognized +-- SELECT EXTRACT(UNDEFINED FROM DATE 'infinity'); -- ERROR: timestamp units "undefined" not supported + +-- test constructors +select make_date(2013, 7, 15); +-- [SPARK-28471] Formatting dates with negative years +select make_date(-44, 3, 15); +-- select make_time(8, 20, 0.0); +-- should fail +select make_date(2013, 2, 30); +select make_date(2013, 13, 1); +select make_date(2013, 11, -1); +-- select make_time(10, 55, 100.1); +-- select make_time(24, 0, 2.1); + +DROP TABLE DATE_TBL; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/float4.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/float4.sql new file mode 100644 index 000000000000..2989569e219f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/float4.sql @@ -0,0 +1,363 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- FLOAT4 +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/float4.sql + +CREATE TABLE FLOAT4_TBL (f1 float) USING parquet; + +-- PostgreSQL implicitly casts string literals to data with floating point types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO FLOAT4_TBL VALUES (float(' 0.0')); +INSERT INTO FLOAT4_TBL VALUES (float('1004.30 ')); +INSERT INTO FLOAT4_TBL VALUES (float(' -34.84 ')); +INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e+20')); +INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e-20')); + +-- [SPARK-28024] Incorrect numeric values when out of range +-- test for over and under flow +-- INSERT INTO FLOAT4_TBL VALUES ('10e70'); +-- INSERT INTO FLOAT4_TBL VALUES ('-10e70'); +-- INSERT INTO FLOAT4_TBL VALUES ('10e-70'); +-- INSERT INTO FLOAT4_TBL VALUES ('-10e-70'); + +-- INSERT INTO FLOAT4_TBL VALUES ('10e400'); +-- INSERT INTO FLOAT4_TBL VALUES ('-10e400'); +-- INSERT INTO FLOAT4_TBL VALUES ('10e-400'); +-- INSERT INTO FLOAT4_TBL VALUES ('-10e-400'); + +-- [SPARK-27923] Spark SQL insert there bad inputs to NULL +-- bad input +-- INSERT INTO FLOAT4_TBL VALUES (''); +-- INSERT INTO FLOAT4_TBL VALUES (' '); +-- INSERT INTO FLOAT4_TBL VALUES ('xyz'); +-- INSERT INTO FLOAT4_TBL VALUES ('5.0.0'); +-- INSERT INTO FLOAT4_TBL VALUES ('5 . 0'); +-- INSERT INTO FLOAT4_TBL VALUES ('5. 0'); +-- INSERT INTO FLOAT4_TBL VALUES (' - 3.0'); +-- INSERT INTO FLOAT4_TBL VALUES ('123 5'); + +-- special inputs +SELECT float('NaN'); +SELECT float('nan'); +SELECT float(' NAN '); +SELECT float('infinity'); +SELECT float(' -INFINiTY '); +-- [SPARK-27923] Spark SQL insert there bad special inputs to NULL +-- bad special inputs +SELECT float('N A N'); +SELECT float('NaN x'); +SELECT float(' INFINITY x'); + +SELECT float('Infinity') + 100.0; +SELECT float('Infinity') / float('Infinity'); +SELECT float('nan') / float('nan'); +SELECT float(decimal('nan')); + +SELECT '' AS five, * FROM FLOAT4_TBL; + +SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE f.f1 <> '1004.3'; + +SELECT '' AS one, f.* FROM FLOAT4_TBL f WHERE f.f1 = '1004.3'; + +SELECT '' AS three, f.* FROM FLOAT4_TBL f WHERE '1004.3' > f.f1; + +SELECT '' AS three, f.* FROM FLOAT4_TBL f WHERE f.f1 < '1004.3'; + +SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE '1004.3' >= f.f1; + +SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE f.f1 <= '1004.3'; + +SELECT '' AS three, f.f1, f.f1 * '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0'; + +SELECT '' AS three, f.f1, f.f1 + '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0'; + +SELECT '' AS three, f.f1, f.f1 / '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0'; + +SELECT '' AS three, f.f1, f.f1 - '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0'; + +-- [SPARK-27923] Spark SQL returns NULL +-- test divide by zero +-- SELECT '' AS bad, f.f1 / '0.0' from FLOAT4_TBL f; + +SELECT '' AS five, * FROM FLOAT4_TBL; + +-- [SPARK-28027] Spark SQL does not support prefix operator @ +-- test the unary float4abs operator +-- SELECT '' AS five, f.f1, @f.f1 AS abs_f1 FROM FLOAT4_TBL f; + +-- Spark SQL does not support update now. +-- UPDATE FLOAT4_TBL +-- SET f1 = FLOAT4_TBL.f1 * '-1' +-- WHERE FLOAT4_TBL.f1 > '0.0'; + +-- SELECT '' AS five, * FROM FLOAT4_TBL; + +-- [SPARK-28028] Cast numeric to integral type need round +-- [SPARK-28024] Incorrect numeric values when out of range +-- test edge-case coercions to integer +SELECT smallint(float('32767.4')); +SELECT smallint(float('32767.6')); +SELECT smallint(float('-32768.4')); +SELECT smallint(float('-32768.6')); +SELECT int(float('2147483520')); +SELECT int(float('2147483647')); +SELECT int(float('-2147483648.5')); +SELECT int(float('-2147483900')); +SELECT bigint(float('9223369837831520256')); +SELECT bigint(float('9223372036854775807')); +SELECT bigint(float('-9223372036854775808.5')); +SELECT bigint(float('-9223380000000000000')); + +-- [SPARK-28061] Support for converting float to binary format +-- Test for correct input rounding in edge cases. +-- These lists are from Paxson 1991, excluding subnormals and +-- inputs of over 9 sig. digits. + +-- SELECT float4send('5e-20'::float4); +-- SELECT float4send('67e14'::float4); +-- SELECT float4send('985e15'::float4); +-- SELECT float4send('55895e-16'::float4); +-- SELECT float4send('7038531e-32'::float4); +-- SELECT float4send('702990899e-20'::float4); + +-- SELECT float4send('3e-23'::float4); +-- SELECT float4send('57e18'::float4); +-- SELECT float4send('789e-35'::float4); +-- SELECT float4send('2539e-18'::float4); +-- SELECT float4send('76173e28'::float4); +-- SELECT float4send('887745e-11'::float4); +-- SELECT float4send('5382571e-37'::float4); +-- SELECT float4send('82381273e-35'::float4); +-- SELECT float4send('750486563e-38'::float4); + +-- Test that the smallest possible normalized input value inputs +-- correctly, either in 9-significant-digit or shortest-decimal +-- format. +-- +-- exact val is 1.1754943508... +-- shortest val is 1.1754944000 +-- midpoint to next val is 1.1754944208... + +-- SELECT float4send('1.17549435e-38'::float4); +-- SELECT float4send('1.1754944e-38'::float4); + +-- We do not support creating types, skip the test below +-- test output (and round-trip safety) of various values. +-- To ensure we're testing what we think we're testing, start with +-- float values specified by bit patterns (as a useful side effect, +-- this means we'll fail on non-IEEE platforms). + +-- create type xfloat4; +-- create function xfloat4in(cstring) returns xfloat4 immutable strict +-- language internal as 'int4in'; +-- create function xfloat4out(xfloat4) returns cstring immutable strict +-- language internal as 'int4out'; +-- create type xfloat4 (input = xfloat4in, output = xfloat4out, like = float4); +-- create cast (xfloat4 as float4) without function; +-- create cast (float4 as xfloat4) without function; +-- create cast (xfloat4 as integer) without function; +-- create cast (integer as xfloat4) without function; + +-- float4: seeeeeee emmmmmmm mmmmmmmm mmmmmmmm + +-- we don't care to assume the platform's strtod() handles subnormals +-- correctly; those are "use at your own risk". However we do test +-- subnormal outputs, since those are under our control. + +-- with testdata(bits) as (values +-- -- small subnormals +-- (x'00000001'), +-- (x'00000002'), (x'00000003'), +-- (x'00000010'), (x'00000011'), (x'00000100'), (x'00000101'), +-- (x'00004000'), (x'00004001'), (x'00080000'), (x'00080001'), +-- -- stress values +-- (x'0053c4f4'), -- 7693e-42 +-- (x'006c85c4'), -- 996622e-44 +-- (x'0041ca76'), -- 60419369e-46 +-- (x'004b7678'), -- 6930161142e-48 +-- -- taken from upstream testsuite +-- (x'00000007'), +-- (x'00424fe2'), +-- -- borderline between subnormal and normal +-- (x'007ffff0'), (x'007ffff1'), (x'007ffffe'), (x'007fffff')) +-- select float4send(flt) as ibits, +-- flt +-- from (select bits::integer::xfloat4::float4 as flt +-- from testdata +-- offset 0) s; + +-- with testdata(bits) as (values +-- (x'00000000'), +-- -- smallest normal values +-- (x'00800000'), (x'00800001'), (x'00800004'), (x'00800005'), +-- (x'00800006'), +-- -- small normal values chosen for short vs. long output +-- (x'008002f1'), (x'008002f2'), (x'008002f3'), +-- (x'00800e17'), (x'00800e18'), (x'00800e19'), +-- -- assorted values (random mantissae) +-- (x'01000001'), (x'01102843'), (x'01a52c98'), +-- (x'0219c229'), (x'02e4464d'), (x'037343c1'), (x'03a91b36'), +-- (x'047ada65'), (x'0496fe87'), (x'0550844f'), (x'05999da3'), +-- (x'060ea5e2'), (x'06e63c45'), (x'07f1e548'), (x'0fc5282b'), +-- (x'1f850283'), (x'2874a9d6'), +-- -- values around 5e-08 +-- (x'3356bf94'), (x'3356bf95'), (x'3356bf96'), +-- -- around 1e-07 +-- (x'33d6bf94'), (x'33d6bf95'), (x'33d6bf96'), +-- -- around 3e-07 .. 1e-04 +-- (x'34a10faf'), (x'34a10fb0'), (x'34a10fb1'), +-- (x'350637bc'), (x'350637bd'), (x'350637be'), +-- (x'35719786'), (x'35719787'), (x'35719788'), +-- (x'358637bc'), (x'358637bd'), (x'358637be'), +-- (x'36a7c5ab'), (x'36a7c5ac'), (x'36a7c5ad'), +-- (x'3727c5ab'), (x'3727c5ac'), (x'3727c5ad'), +-- -- format crossover at 1e-04 +-- (x'38d1b714'), (x'38d1b715'), (x'38d1b716'), +-- (x'38d1b717'), (x'38d1b718'), (x'38d1b719'), +-- (x'38d1b71a'), (x'38d1b71b'), (x'38d1b71c'), +-- (x'38d1b71d'), +-- -- +-- (x'38dffffe'), (x'38dfffff'), (x'38e00000'), +-- (x'38efffff'), (x'38f00000'), (x'38f00001'), +-- (x'3a83126e'), (x'3a83126f'), (x'3a831270'), +-- (x'3c23d709'), (x'3c23d70a'), (x'3c23d70b'), +-- (x'3dcccccc'), (x'3dcccccd'), (x'3dccccce'), +-- -- chosen to need 9 digits for 3dcccd70 +-- (x'3dcccd6f'), (x'3dcccd70'), (x'3dcccd71'), +-- -- +-- (x'3effffff'), (x'3f000000'), (x'3f000001'), +-- (x'3f333332'), (x'3f333333'), (x'3f333334'), +-- -- approach 1.0 with increasing numbers of 9s +-- (x'3f666665'), (x'3f666666'), (x'3f666667'), +-- (x'3f7d70a3'), (x'3f7d70a4'), (x'3f7d70a5'), +-- (x'3f7fbe76'), (x'3f7fbe77'), (x'3f7fbe78'), +-- (x'3f7ff971'), (x'3f7ff972'), (x'3f7ff973'), +-- (x'3f7fff57'), (x'3f7fff58'), (x'3f7fff59'), +-- (x'3f7fffee'), (x'3f7fffef'), +-- -- values very close to 1 +-- (x'3f7ffff0'), (x'3f7ffff1'), (x'3f7ffff2'), +-- (x'3f7ffff3'), (x'3f7ffff4'), (x'3f7ffff5'), +-- (x'3f7ffff6'), (x'3f7ffff7'), (x'3f7ffff8'), +-- (x'3f7ffff9'), (x'3f7ffffa'), (x'3f7ffffb'), +-- (x'3f7ffffc'), (x'3f7ffffd'), (x'3f7ffffe'), +-- (x'3f7fffff'), +-- (x'3f800000'), +-- (x'3f800001'), (x'3f800002'), (x'3f800003'), +-- (x'3f800004'), (x'3f800005'), (x'3f800006'), +-- (x'3f800007'), (x'3f800008'), (x'3f800009'), +-- -- values 1 to 1.1 +-- (x'3f80000f'), (x'3f800010'), (x'3f800011'), +-- (x'3f800012'), (x'3f800013'), (x'3f800014'), +-- (x'3f800017'), (x'3f800018'), (x'3f800019'), +-- (x'3f80001a'), (x'3f80001b'), (x'3f80001c'), +-- (x'3f800029'), (x'3f80002a'), (x'3f80002b'), +-- (x'3f800053'), (x'3f800054'), (x'3f800055'), +-- (x'3f800346'), (x'3f800347'), (x'3f800348'), +-- (x'3f8020c4'), (x'3f8020c5'), (x'3f8020c6'), +-- (x'3f8147ad'), (x'3f8147ae'), (x'3f8147af'), +-- (x'3f8ccccc'), (x'3f8ccccd'), (x'3f8cccce'), +-- -- +-- (x'3fc90fdb'), -- pi/2 +-- (x'402df854'), -- e +-- (x'40490fdb'), -- pi +-- -- +-- (x'409fffff'), (x'40a00000'), (x'40a00001'), +-- (x'40afffff'), (x'40b00000'), (x'40b00001'), +-- (x'411fffff'), (x'41200000'), (x'41200001'), +-- (x'42c7ffff'), (x'42c80000'), (x'42c80001'), +-- (x'4479ffff'), (x'447a0000'), (x'447a0001'), +-- (x'461c3fff'), (x'461c4000'), (x'461c4001'), +-- (x'47c34fff'), (x'47c35000'), (x'47c35001'), +-- (x'497423ff'), (x'49742400'), (x'49742401'), +-- (x'4b18967f'), (x'4b189680'), (x'4b189681'), +-- (x'4cbebc1f'), (x'4cbebc20'), (x'4cbebc21'), +-- (x'4e6e6b27'), (x'4e6e6b28'), (x'4e6e6b29'), +-- (x'501502f8'), (x'501502f9'), (x'501502fa'), +-- (x'51ba43b6'), (x'51ba43b7'), (x'51ba43b8'), +-- -- stress values +-- (x'1f6c1e4a'), -- 5e-20 +-- (x'59be6cea'), -- 67e14 +-- (x'5d5ab6c4'), -- 985e15 +-- (x'2cc4a9bd'), -- 55895e-16 +-- (x'15ae43fd'), -- 7038531e-32 +-- (x'2cf757ca'), -- 702990899e-20 +-- (x'665ba998'), -- 25933168707e13 +-- (x'743c3324'), -- 596428896559e20 +-- -- exercise fixed-point memmoves +-- (x'47f1205a'), +-- (x'4640e6ae'), +-- (x'449a5225'), +-- (x'42f6e9d5'), +-- (x'414587dd'), +-- (x'3f9e064b'), +-- -- these cases come from the upstream's testsuite +-- -- BoundaryRoundEven +-- (x'4c000004'), +-- (x'50061c46'), +-- (x'510006a8'), +-- -- ExactValueRoundEven +-- (x'48951f84'), +-- (x'45fd1840'), +-- -- LotsOfTrailingZeros +-- (x'39800000'), +-- (x'3b200000'), +-- (x'3b900000'), +-- (x'3bd00000'), +-- -- Regression +-- (x'63800000'), +-- (x'4b000000'), +-- (x'4b800000'), +-- (x'4c000001'), +-- (x'4c800b0d'), +-- (x'00d24584'), +-- (x'00d90b88'), +-- (x'45803f34'), +-- (x'4f9f24f7'), +-- (x'3a8722c3'), +-- (x'5c800041'), +-- (x'15ae43fd'), +-- (x'5d4cccfb'), +-- (x'4c800001'), +-- (x'57800ed8'), +-- (x'5f000000'), +-- (x'700000f0'), +-- (x'5f23e9ac'), +-- (x'5e9502f9'), +-- (x'5e8012b1'), +-- (x'3c000028'), +-- (x'60cde861'), +-- (x'03aa2a50'), +-- (x'43480000'), +-- (x'4c000000'), +-- -- LooksLikePow5 +-- (x'5D1502F9'), +-- (x'5D9502F9'), +-- (x'5E1502F9'), +-- -- OutputLength +-- (x'3f99999a'), +-- (x'3f9d70a4'), +-- (x'3f9df3b6'), +-- (x'3f9e0419'), +-- (x'3f9e0610'), +-- (x'3f9e064b'), +-- (x'3f9e0651'), +-- (x'03d20cfe') +-- ) +-- select float4send(flt) as ibits, +-- flt, +-- flt::text::float4 as r_flt, +-- float4send(flt::text::float4) as obits, +-- float4send(flt::text::float4) = float4send(flt) as correct +-- from (select bits::integer::xfloat4::float4 as flt +-- from testdata +-- offset 0) s; + +-- clean up, lest opr_sanity complain +-- drop type xfloat4 cascade; +DROP TABLE FLOAT4_TBL; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/float8.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/float8.sql new file mode 100644 index 000000000000..932cdb95fcf3 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/float8.sql @@ -0,0 +1,502 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- FLOAT8 +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/float8.sql + +CREATE TABLE FLOAT8_TBL(f1 double) USING parquet; + +-- PostgreSQL implicitly casts string literals to data with floating point types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO FLOAT8_TBL VALUES (double(' 0.0 ')); +INSERT INTO FLOAT8_TBL VALUES (double('1004.30 ')); +INSERT INTO FLOAT8_TBL VALUES (double(' -34.84')); +INSERT INTO FLOAT8_TBL VALUES (double('1.2345678901234e+200')); +INSERT INTO FLOAT8_TBL VALUES (double('1.2345678901234e-200')); + +-- [SPARK-28024] Incorrect numeric values when out of range +-- test for underflow and overflow handling +SELECT double('10e400'); +SELECT double('-10e400'); +SELECT double('10e-400'); +SELECT double('-10e-400'); + +-- [SPARK-28061] Support for converting float to binary format +-- test smallest normalized input +-- SELECT float8send('2.2250738585072014E-308'::float8); + +-- [SPARK-27923] Spark SQL insert there bad inputs to NULL +-- bad input +-- INSERT INTO FLOAT8_TBL VALUES (''); +-- INSERT INTO FLOAT8_TBL VALUES (' '); +-- INSERT INTO FLOAT8_TBL VALUES ('xyz'); +-- INSERT INTO FLOAT8_TBL VALUES ('5.0.0'); +-- INSERT INTO FLOAT8_TBL VALUES ('5 . 0'); +-- INSERT INTO FLOAT8_TBL VALUES ('5. 0'); +-- INSERT INTO FLOAT8_TBL VALUES (' - 3'); +-- INSERT INTO FLOAT8_TBL VALUES ('123 5'); + +-- special inputs +SELECT double('NaN'); +SELECT double('nan'); +SELECT double(' NAN '); +SELECT double('infinity'); +SELECT double(' -INFINiTY '); +-- [SPARK-27923] Spark SQL insert there bad special inputs to NULL +-- bad special inputs +SELECT double('N A N'); +SELECT double('NaN x'); +SELECT double(' INFINITY x'); + +SELECT double('Infinity') + 100.0; +SELECT double('Infinity') / double('Infinity'); +SELECT double('NaN') / double('NaN'); +-- [SPARK-28315] Decimal can not accept NaN as input +SELECT double(decimal('nan')); + +SELECT '' AS five, * FROM FLOAT8_TBL; + +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <> '1004.3'; + +SELECT '' AS one, f.* FROM FLOAT8_TBL f WHERE f.f1 = '1004.3'; + +SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE '1004.3' > f.f1; + +SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE f.f1 < '1004.3'; + +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE '1004.3' >= f.f1; + +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <= '1004.3'; + +SELECT '' AS three, f.f1, f.f1 * '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; + +SELECT '' AS three, f.f1, f.f1 + '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; + +SELECT '' AS three, f.f1, f.f1 / '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; + +SELECT '' AS three, f.f1, f.f1 - '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; +-- [SPARK-28007] Caret operator (^) means bitwise XOR in Spark/Hive and exponentiation in Postgres +-- SELECT '' AS one, f.f1 ^ '2.0' AS square_f1 +-- FROM FLOAT8_TBL f where f.f1 = '1004.3'; + +-- [SPARK-28027] Spark SQL does not support prefix operator @ +-- absolute value +-- SELECT '' AS five, f.f1, @f.f1 AS abs_f1 +-- FROM FLOAT8_TBL f; + +-- [SPARK-23906] Support Truncate number +-- truncate +-- SELECT '' AS five, f.f1, trunc(f.f1) AS trunc_f1 +-- FROM FLOAT8_TBL f; + +-- round +SELECT '' AS five, f.f1, round(f.f1) AS round_f1 + FROM FLOAT8_TBL f; + +-- [SPARK-28135] ceil/ceiling/floor returns incorrect values +-- ceil / ceiling +select ceil(f1) as ceil_f1 from float8_tbl f; +select ceiling(f1) as ceiling_f1 from float8_tbl f; + +-- floor +select floor(f1) as floor_f1 from float8_tbl f; + +-- sign +select sign(f1) as sign_f1 from float8_tbl f; + +-- avoid bit-exact output here because operations may not be bit-exact. +-- SET extra_float_digits = 0; + +-- square root +SELECT sqrt(double('64')) AS eight; + +-- [SPARK-28027] Spark SQL does not support prefix operator |/ +-- SELECT |/ float8 '64' AS eight; + +-- SELECT '' AS three, f.f1, |/f.f1 AS sqrt_f1 +-- FROM FLOAT8_TBL f +-- WHERE f.f1 > '0.0'; + +-- power +SELECT power(double('144'), double('0.5')); +SELECT power(double('NaN'), double('0.5')); +SELECT power(double('144'), double('NaN')); +SELECT power(double('NaN'), double('NaN')); +SELECT power(double('-1'), double('NaN')); +-- [SPARK-28135] power returns incorrect values +SELECT power(double('1'), double('NaN')); +SELECT power(double('NaN'), double('0')); + +-- take exp of ln(f.f1) +SELECT '' AS three, f.f1, exp(ln(f.f1)) AS exp_ln_f1 + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; + +-- [SPARK-28027] Spark SQL does not support prefix operator ||/ +-- cube root +-- SELECT ||/ float8 '27' AS three; + +-- SELECT '' AS five, f.f1, ||/f.f1 AS cbrt_f1 FROM FLOAT8_TBL f; + + +SELECT '' AS five, * FROM FLOAT8_TBL; + +-- UPDATE FLOAT8_TBL +-- SET f1 = FLOAT8_TBL.f1 * '-1' +-- WHERE FLOAT8_TBL.f1 > '0.0'; +-- Update the FLOAT8_TBL to UPDATED_FLOAT8_TBL +CREATE TEMPORARY VIEW UPDATED_FLOAT8_TBL as +SELECT + CASE WHEN FLOAT8_TBL.f1 > '0.0' THEN FLOAT8_TBL.f1 * '-1' ELSE FLOAT8_TBL.f1 END AS f1 +FROM FLOAT8_TBL; + +-- [SPARK-27923] Out of range, Spark SQL returns Infinity +SELECT '' AS bad, f.f1 * '1e200' from UPDATED_FLOAT8_TBL f; + +-- [SPARK-28007] Caret operator (^) means bitwise XOR in Spark/Hive and exponentiation in Postgres +-- SELECT '' AS bad, f.f1 ^ '1e200' from UPDATED_FLOAT8_TBL f; + +-- SELECT 0 ^ 0 + 0 ^ 1 + 0 ^ 0.0 + 0 ^ 0.5; + +-- [SPARK-27923] Cannot take logarithm of zero +-- SELECT '' AS bad, ln(f.f1) from UPDATED_FLOAT8_TBL f where f.f1 = '0.0' ; + +-- [SPARK-27923] Cannot take logarithm of a negative number +-- SELECT '' AS bad, ln(f.f1) from UPDATED_FLOAT8_TBL f where f.f1 < '0.0' ; + +-- [SPARK-28024] Incorrect numeric values when out of range +-- SELECT '' AS bad, exp(f.f1) from UPDATED_FLOAT8_TBL f; + +-- [SPARK-27923] Divide by zero, Spark SQL returns NULL +-- SELECT '' AS bad, f.f1 / '0.0' from UPDATED_FLOAT8_TBL f; + +SELECT '' AS five, * FROM UPDATED_FLOAT8_TBL; + +-- hyperbolic functions +-- we run these with extra_float_digits = 0 too, since different platforms +-- tend to produce results that vary in the last place. +SELECT sinh(double('1')); +SELECT cosh(double('1')); +SELECT tanh(double('1')); +SELECT asinh(double('1')); +SELECT acosh(double('2')); +SELECT atanh(double('0.5')); + +-- test Inf/NaN cases for hyperbolic functions +SELECT sinh(double('Infinity')); +SELECT sinh(double('-Infinity')); +SELECT sinh(double('NaN')); +SELECT cosh(double('Infinity')); +SELECT cosh(double('-Infinity')); +SELECT cosh(double('NaN')); +SELECT tanh(double('Infinity')); +SELECT tanh(double('-Infinity')); +SELECT tanh(double('NaN')); +SELECT asinh(double('Infinity')); +SELECT asinh(double('-Infinity')); +SELECT asinh(double('NaN')); +-- acosh(Inf) should be Inf, but some mingw versions produce NaN, so skip test +SELECT acosh(double('Infinity')); +SELECT acosh(double('-Infinity')); +SELECT acosh(double('NaN')); +SELECT atanh(double('Infinity')); +SELECT atanh(double('-Infinity')); +SELECT atanh(double('NaN')); + +-- RESET extra_float_digits; + +-- [SPARK-28024] Incorrect numeric values when out of range +-- test for over- and underflow +-- INSERT INTO FLOAT8_TBL VALUES ('10e400'); + +-- INSERT INTO FLOAT8_TBL VALUES ('-10e400'); + +-- INSERT INTO FLOAT8_TBL VALUES ('10e-400'); + +-- INSERT INTO FLOAT8_TBL VALUES ('-10e-400'); + +-- maintain external table consistency across platforms +-- delete all values and reinsert well-behaved ones + +TRUNCATE TABLE FLOAT8_TBL; + +-- PostgreSQL implicitly casts string literals to data with floating point types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO FLOAT8_TBL VALUES (double('0.0')); + +INSERT INTO FLOAT8_TBL VALUES (double('-34.84')); + +INSERT INTO FLOAT8_TBL VALUES (double('-1004.30')); + +INSERT INTO FLOAT8_TBL VALUES (double('-1.2345678901234e+200')); + +INSERT INTO FLOAT8_TBL VALUES (double('-1.2345678901234e-200')); + +SELECT '' AS five, * FROM FLOAT8_TBL; + +-- [SPARK-28028] Cast numeric to integral type need round +-- [SPARK-28024] Incorrect numeric values when out of range +-- test edge-case coercions to integer +SELECT smallint(double('32767.4')); +SELECT smallint(double('32767.6')); +SELECT smallint(double('-32768.4')); +SELECT smallint(double('-32768.6')); +SELECT int(double('2147483647.4')); +SELECT int(double('2147483647.6')); +SELECT int(double('-2147483648.4')); +SELECT int(double('-2147483648.6')); +SELECT bigint(double('9223372036854773760')); +SELECT bigint(double('9223372036854775807')); +SELECT bigint(double('-9223372036854775808.5')); +SELECT bigint(double('-9223372036854780000')); + +-- [SPARK-28134] Missing Trigonometric Functions +-- test exact cases for trigonometric functions in degrees + +-- SELECT x, +-- sind(x), +-- sind(x) IN (-1,-0.5,0,0.5,1) AS sind_exact +-- FROM (VALUES (0), (30), (90), (150), (180), +-- (210), (270), (330), (360)) AS t(x); + +-- SELECT x, +-- cosd(x), +-- cosd(x) IN (-1,-0.5,0,0.5,1) AS cosd_exact +-- FROM (VALUES (0), (60), (90), (120), (180), +-- (240), (270), (300), (360)) AS t(x); + +-- SELECT x, +-- tand(x), +-- tand(x) IN ('-Infinity'::float8,-1,0, +-- 1,'Infinity'::float8) AS tand_exact, +-- cotd(x), +-- cotd(x) IN ('-Infinity'::float8,-1,0, +-- 1,'Infinity'::float8) AS cotd_exact +-- FROM (VALUES (0), (45), (90), (135), (180), +-- (225), (270), (315), (360)) AS t(x); + +-- SELECT x, +-- asind(x), +-- asind(x) IN (-90,-30,0,30,90) AS asind_exact, +-- acosd(x), +-- acosd(x) IN (0,60,90,120,180) AS acosd_exact +-- FROM (VALUES (-1), (-0.5), (0), (0.5), (1)) AS t(x); + +-- SELECT x, +-- atand(x), +-- atand(x) IN (-90,-45,0,45,90) AS atand_exact +-- FROM (VALUES ('-Infinity'::float8), (-1), (0), (1), +-- ('Infinity'::float8)) AS t(x); + +-- SELECT x, y, +-- atan2d(y, x), +-- atan2d(y, x) IN (-90,0,90,180) AS atan2d_exact +-- FROM (SELECT 10*cosd(a), 10*sind(a) +-- FROM generate_series(0, 360, 90) AS t(a)) AS t(x,y); + +-- We do not support creating types, skip the test below +-- +-- test output (and round-trip safety) of various values. +-- To ensure we're testing what we think we're testing, start with +-- float values specified by bit patterns (as a useful side effect, +-- this means we'll fail on non-IEEE platforms). + +-- create type xfloat8; +-- create function xfloat8in(cstring) returns xfloat8 immutable strict +-- language internal as 'int8in'; +-- create function xfloat8out(xfloat8) returns cstring immutable strict +-- language internal as 'int8out'; +-- create type xfloat8 (input = xfloat8in, output = xfloat8out, like = float8); +-- create cast (xfloat8 as float8) without function; +-- create cast (float8 as xfloat8) without function; +-- create cast (xfloat8 as bigint) without function; +-- create cast (bigint as xfloat8) without function; + +-- float8: seeeeeee eeeeeeee eeeeeeee mmmmmmmm mmmmmmmm(x4) + +-- we don't care to assume the platform's strtod() handles subnormals +-- correctly; those are "use at your own risk". However we do test +-- subnormal outputs, since those are under our control. + +-- with testdata(bits) as (values +-- -- small subnormals +-- (x'0000000000000001'), +-- (x'0000000000000002'), (x'0000000000000003'), +-- (x'0000000000001000'), (x'0000000100000000'), +-- (x'0000010000000000'), (x'0000010100000000'), +-- (x'0000400000000000'), (x'0000400100000000'), +-- (x'0000800000000000'), (x'0000800000000001'), +-- -- these values taken from upstream testsuite +-- (x'00000000000f4240'), +-- (x'00000000016e3600'), +-- (x'0000008cdcdea440'), +-- -- borderline between subnormal and normal +-- (x'000ffffffffffff0'), (x'000ffffffffffff1'), +-- (x'000ffffffffffffe'), (x'000fffffffffffff')) +-- select float8send(flt) as ibits, +-- flt +-- from (select bits::bigint::xfloat8::float8 as flt +-- from testdata +-- offset 0) s; + +-- round-trip tests + +-- with testdata(bits) as (values +-- (x'0000000000000000'), +-- -- smallest normal values +-- (x'0010000000000000'), (x'0010000000000001'), +-- (x'0010000000000002'), (x'0018000000000000'), +-- -- +-- (x'3ddb7cdfd9d7bdba'), (x'3ddb7cdfd9d7bdbb'), (x'3ddb7cdfd9d7bdbc'), +-- (x'3e112e0be826d694'), (x'3e112e0be826d695'), (x'3e112e0be826d696'), +-- (x'3e45798ee2308c39'), (x'3e45798ee2308c3a'), (x'3e45798ee2308c3b'), +-- (x'3e7ad7f29abcaf47'), (x'3e7ad7f29abcaf48'), (x'3e7ad7f29abcaf49'), +-- (x'3eb0c6f7a0b5ed8c'), (x'3eb0c6f7a0b5ed8d'), (x'3eb0c6f7a0b5ed8e'), +-- (x'3ee4f8b588e368ef'), (x'3ee4f8b588e368f0'), (x'3ee4f8b588e368f1'), +-- (x'3f1a36e2eb1c432c'), (x'3f1a36e2eb1c432d'), (x'3f1a36e2eb1c432e'), +-- (x'3f50624dd2f1a9fb'), (x'3f50624dd2f1a9fc'), (x'3f50624dd2f1a9fd'), +-- (x'3f847ae147ae147a'), (x'3f847ae147ae147b'), (x'3f847ae147ae147c'), +-- (x'3fb9999999999999'), (x'3fb999999999999a'), (x'3fb999999999999b'), +-- -- values very close to 1 +-- (x'3feffffffffffff0'), (x'3feffffffffffff1'), (x'3feffffffffffff2'), +-- (x'3feffffffffffff3'), (x'3feffffffffffff4'), (x'3feffffffffffff5'), +-- (x'3feffffffffffff6'), (x'3feffffffffffff7'), (x'3feffffffffffff8'), +-- (x'3feffffffffffff9'), (x'3feffffffffffffa'), (x'3feffffffffffffb'), +-- (x'3feffffffffffffc'), (x'3feffffffffffffd'), (x'3feffffffffffffe'), +-- (x'3fefffffffffffff'), +-- (x'3ff0000000000000'), +-- (x'3ff0000000000001'), (x'3ff0000000000002'), (x'3ff0000000000003'), +-- (x'3ff0000000000004'), (x'3ff0000000000005'), (x'3ff0000000000006'), +-- (x'3ff0000000000007'), (x'3ff0000000000008'), (x'3ff0000000000009'), +-- -- +-- (x'3ff921fb54442d18'), +-- (x'4005bf0a8b14576a'), +-- (x'400921fb54442d18'), +-- -- +-- (x'4023ffffffffffff'), (x'4024000000000000'), (x'4024000000000001'), +-- (x'4058ffffffffffff'), (x'4059000000000000'), (x'4059000000000001'), +-- (x'408f3fffffffffff'), (x'408f400000000000'), (x'408f400000000001'), +-- (x'40c387ffffffffff'), (x'40c3880000000000'), (x'40c3880000000001'), +-- (x'40f869ffffffffff'), (x'40f86a0000000000'), (x'40f86a0000000001'), +-- (x'412e847fffffffff'), (x'412e848000000000'), (x'412e848000000001'), +-- (x'416312cfffffffff'), (x'416312d000000000'), (x'416312d000000001'), +-- (x'4197d783ffffffff'), (x'4197d78400000000'), (x'4197d78400000001'), +-- (x'41cdcd64ffffffff'), (x'41cdcd6500000000'), (x'41cdcd6500000001'), +-- (x'4202a05f1fffffff'), (x'4202a05f20000000'), (x'4202a05f20000001'), +-- (x'42374876e7ffffff'), (x'42374876e8000000'), (x'42374876e8000001'), +-- (x'426d1a94a1ffffff'), (x'426d1a94a2000000'), (x'426d1a94a2000001'), +-- (x'42a2309ce53fffff'), (x'42a2309ce5400000'), (x'42a2309ce5400001'), +-- (x'42d6bcc41e8fffff'), (x'42d6bcc41e900000'), (x'42d6bcc41e900001'), +-- (x'430c6bf52633ffff'), (x'430c6bf526340000'), (x'430c6bf526340001'), +-- (x'4341c37937e07fff'), (x'4341c37937e08000'), (x'4341c37937e08001'), +-- (x'4376345785d89fff'), (x'4376345785d8a000'), (x'4376345785d8a001'), +-- (x'43abc16d674ec7ff'), (x'43abc16d674ec800'), (x'43abc16d674ec801'), +-- (x'43e158e460913cff'), (x'43e158e460913d00'), (x'43e158e460913d01'), +-- (x'4415af1d78b58c3f'), (x'4415af1d78b58c40'), (x'4415af1d78b58c41'), +-- (x'444b1ae4d6e2ef4f'), (x'444b1ae4d6e2ef50'), (x'444b1ae4d6e2ef51'), +-- (x'4480f0cf064dd591'), (x'4480f0cf064dd592'), (x'4480f0cf064dd593'), +-- (x'44b52d02c7e14af5'), (x'44b52d02c7e14af6'), (x'44b52d02c7e14af7'), +-- (x'44ea784379d99db3'), (x'44ea784379d99db4'), (x'44ea784379d99db5'), +-- (x'45208b2a2c280290'), (x'45208b2a2c280291'), (x'45208b2a2c280292'), +-- -- +-- (x'7feffffffffffffe'), (x'7fefffffffffffff'), +-- -- round to even tests (+ve) +-- (x'4350000000000002'), +-- (x'4350000000002e06'), +-- (x'4352000000000003'), +-- (x'4352000000000004'), +-- (x'4358000000000003'), +-- (x'4358000000000004'), +-- (x'435f000000000020'), +-- -- round to even tests (-ve) +-- (x'c350000000000002'), +-- (x'c350000000002e06'), +-- (x'c352000000000003'), +-- (x'c352000000000004'), +-- (x'c358000000000003'), +-- (x'c358000000000004'), +-- (x'c35f000000000020'), +-- -- exercise fixed-point memmoves +-- (x'42dc12218377de66'), +-- (x'42a674e79c5fe51f'), +-- (x'4271f71fb04cb74c'), +-- (x'423cbe991a145879'), +-- (x'4206fee0e1a9e061'), +-- (x'41d26580b487e6b4'), +-- (x'419d6f34540ca453'), +-- (x'41678c29dcd6e9dc'), +-- (x'4132d687e3df217d'), +-- (x'40fe240c9fcb68c8'), +-- (x'40c81cd6e63c53d3'), +-- (x'40934a4584fd0fdc'), +-- (x'405edd3c07fb4c93'), +-- (x'4028b0fcd32f7076'), +-- (x'3ff3c0ca428c59f8'), +-- -- these cases come from the upstream's testsuite +-- -- LotsOfTrailingZeros) +-- (x'3e60000000000000'), +-- -- Regression +-- (x'c352bd2668e077c4'), +-- (x'434018601510c000'), +-- (x'43d055dc36f24000'), +-- (x'43e052961c6f8000'), +-- (x'3ff3c0ca2a5b1d5d'), +-- -- LooksLikePow5 +-- (x'4830f0cf064dd592'), +-- (x'4840f0cf064dd592'), +-- (x'4850f0cf064dd592'), +-- -- OutputLength +-- (x'3ff3333333333333'), +-- (x'3ff3ae147ae147ae'), +-- (x'3ff3be76c8b43958'), +-- (x'3ff3c083126e978d'), +-- (x'3ff3c0c1fc8f3238'), +-- (x'3ff3c0c9539b8887'), +-- (x'3ff3c0ca2a5b1d5d'), +-- (x'3ff3c0ca4283de1b'), +-- (x'3ff3c0ca43db770a'), +-- (x'3ff3c0ca428abd53'), +-- (x'3ff3c0ca428c1d2b'), +-- (x'3ff3c0ca428c51f2'), +-- (x'3ff3c0ca428c58fc'), +-- (x'3ff3c0ca428c59dd'), +-- (x'3ff3c0ca428c59f8'), +-- (x'3ff3c0ca428c59fb'), +-- -- 32-bit chunking +-- (x'40112e0be8047a7d'), +-- (x'40112e0be815a889'), +-- (x'40112e0be826d695'), +-- (x'40112e0be83804a1'), +-- (x'40112e0be84932ad'), +-- -- MinMaxShift +-- (x'0040000000000000'), +-- (x'007fffffffffffff'), +-- (x'0290000000000000'), +-- (x'029fffffffffffff'), +-- (x'4350000000000000'), +-- (x'435fffffffffffff'), +-- (x'1330000000000000'), +-- (x'133fffffffffffff'), +-- (x'3a6fa7161a4d6e0c') +-- ) +-- select float8send(flt) as ibits, +-- flt, +-- flt::text::float8 as r_flt, +-- float8send(flt::text::float8) as obits, +-- float8send(flt::text::float8) = float8send(flt) as correct +-- from (select bits::bigint::xfloat8::float8 as flt +-- from testdata +-- offset 0) s; + +-- clean up, lest opr_sanity complain +-- drop type xfloat8 cascade; +DROP TABLE FLOAT8_TBL; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/groupingsets.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/groupingsets.sql new file mode 100644 index 000000000000..f06b0276b008 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/groupingsets.sql @@ -0,0 +1,561 @@ +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- GROUPING SETS +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/groupingsets.sql + +-- test data sources + +create temp view gstest1(a,b,v) + as values (1,1,10),(1,1,11),(1,2,12),(1,2,13),(1,3,14), + (2,3,15), + (3,3,16),(3,4,17), + (4,1,18),(4,1,19); + +-- Since Spark doesn't support CREATE TEMPORARY TABLE, we used CREATE TABLE instead +-- create temp table gstest2 (a integer, b integer, c integer, d integer, +-- e integer, f integer, g integer, h integer); +create table gstest2 (a integer, b integer, c integer, d integer, + e integer, f integer, g integer, h integer) using parquet; +-- [SPARK-29386] Copy data between a file and a table +-- copy gstest2 from stdin; +-- 1 1 1 1 1 1 1 1 +-- 1 1 1 1 1 1 1 2 +-- 1 1 1 1 1 1 2 2 +-- 1 1 1 1 1 2 2 2 +-- 1 1 1 1 2 2 2 2 +-- 1 1 1 2 2 2 2 2 +-- 1 1 2 2 2 2 2 2 +-- 1 2 2 2 2 2 2 2 +-- 2 2 2 2 2 2 2 2 +-- \. +insert into gstest2 values + (1, 1, 1, 1, 1, 1, 1, 1), + (1, 1, 1, 1, 1, 1, 1, 2), + (1, 1, 1, 1, 1, 1, 2, 2), + (1, 1, 1, 1, 1, 2, 2, 2), + (1, 1, 1, 1, 2, 2, 2, 2), + (1, 1, 1, 2, 2, 2, 2, 2), + (1, 1, 2, 2, 2, 2, 2, 2), + (1, 2, 2, 2, 2, 2, 2, 2), + (2, 2, 2, 2, 2, 2, 2, 2); + +-- Since Spark doesn't support CREATE TEMPORARY TABLE, we used CREATE TABLE instead +-- create temp table gstest3 (a integer, b integer, c integer, d integer); +create table gstest3 (a integer, b integer, c integer, d integer) using parquet; +-- [SPARK-29386] Copy data between a file and a table +-- copy gstest3 from stdin; +-- 1 1 1 1 +-- 2 2 2 2 +-- \. +insert into gstest3 values + (1, 1, 1, 1), + (2, 2, 2, 2); +-- [SPARK-19842] Informational Referential Integrity Constraints Support in Spark +-- alter table gstest3 add primary key (a); + +-- Since Spark doesn't support CREATE TEMPORARY TABLE, we used CREATE TABLE instead +-- create temp table gstest4(id integer, v integer, +-- unhashable_col bit(4), unsortable_col xid); +-- [SPARK-29697] Support bit string types/literals +create table gstest4(id integer, v integer, + unhashable_col /* bit(4) */ byte, unsortable_col /* xid */ integer) using parquet; +insert into gstest4 +-- values (1,1,b'0000','1'), (2,2,b'0001','1'), +-- (3,4,b'0010','2'), (4,8,b'0011','2'), +-- (5,16,b'0000','2'), (6,32,b'0001','2'), +-- (7,64,b'0010','1'), (8,128,b'0011','1'); +values (1,1,tinyint('0'),1), (2,2,tinyint('1'),1), + (3,4,tinyint('2'),2), (4,8,tinyint('3'),2), + (5,16,tinyint('0'),2), (6,32,tinyint('1'),2), + (7,64,tinyint('2'),1), (8,128,tinyint('3'),1); + +-- Since Spark doesn't support CREATE TEMPORARY TABLE, we used CREATE TABLE instead +-- create temp table gstest_empty (a integer, b integer, v integer); +create table gstest_empty (a integer, b integer, v integer) using parquet; + +-- Spark doesn't handle UDFs in SQL +-- create function gstest_data(v integer, out a integer, out b integer) +-- returns setof record +-- as $f$ +-- begin +-- return query select v, i from generate_series(1,3) i; +-- end; +-- $f$ language plpgsql; + +-- basic functionality + +-- Ignore a PostgreSQL-specific option +-- set enable_hashagg = false; -- test hashing explicitly later + +-- simple rollup with multiple plain aggregates, with and without ordering +-- (and with ordering differing from grouping) + +-- [SPARK-29698] Support grouping function with multiple arguments +-- select a, b, grouping(a,b), sum(v), count(*), max(v) +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by rollup (a,b); +-- select a, b, grouping(a,b), sum(v), count(*), max(v) +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by rollup (a,b) order by a,b; +-- select a, b, grouping(a,b), sum(v), count(*), max(v) +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by rollup (a,b) order by b desc, a; +-- select a, b, grouping(a,b), sum(v), count(*), max(v) +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by rollup (a,b) order by coalesce(a,0)+coalesce(b,0), a; + +-- [SPARK-28664] ORDER BY in aggregate function +-- various types of ordered aggs +-- select a, b, grouping(a,b), +-- array_agg(v order by v), +-- string_agg(string(v:text, ':' order by v desc), +-- percentile_disc(0.5) within group (order by v), +-- rank(1,2,12) within group (order by a,b,v) +-- from gstest1 group by rollup (a,b) order by a,b; + +-- [SPARK-28664] ORDER BY in aggregate function +-- test usage of grouped columns in direct args of aggs +-- select grouping(a), a, array_agg(b), +-- rank(a) within group (order by b nulls first), +-- rank(a) within group (order by b nulls last) +-- from (values (1,1),(1,4),(1,5),(3,1),(3,2)) v(a,b) +-- group by rollup (a) order by a; + +-- nesting with window functions +-- [SPARK-29699] Different answers in nested aggregates with window functions +select a, b, sum(c), sum(sum(c)) over (order by a,b) as rsum + from gstest2 group by rollup (a,b) order by rsum, a, b; + +-- [SPARK-29700] Support nested grouping sets +-- nesting with grouping sets +-- select sum(c) from gstest2 +-- group by grouping sets((), grouping sets((), grouping sets(()))) +-- order by 1 desc; +-- select sum(c) from gstest2 +-- group by grouping sets((), grouping sets((), grouping sets(((a, b))))) +-- order by 1 desc; +-- select sum(c) from gstest2 +-- group by grouping sets(grouping sets(rollup(c), grouping sets(cube(c)))) +-- order by 1 desc; +-- select sum(c) from gstest2 +-- group by grouping sets(a, grouping sets(a, cube(b))) +-- order by 1 desc; +-- select sum(c) from gstest2 +-- group by grouping sets(grouping sets((a, (b)))) +-- order by 1 desc; +-- select sum(c) from gstest2 +-- group by grouping sets(grouping sets((a, b))) +-- order by 1 desc; +-- select sum(c) from gstest2 +-- group by grouping sets(grouping sets(a, grouping sets(a), a)) +-- order by 1 desc; +-- select sum(c) from gstest2 +-- group by grouping sets(grouping sets(a, grouping sets(a, grouping sets(a), ((a)), a, grouping sets(a), (a)), a)) +-- order by 1 desc; +-- select sum(c) from gstest2 +-- group by grouping sets((a,(a,b)), grouping sets((a,(a,b)),a)) +-- order by 1 desc; + +-- empty input: first is 0 rows, second 1, third 3 etc. +select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),a); +-- [SPARK-29701] Different answers when empty input given in GROUPING SETS +select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),()); +select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),(),(),()); +select sum(v), count(*) from gstest_empty group by grouping sets ((),(),()); + +-- empty input with joins tests some important code paths +-- [SPARK-29701] Different answers when empty input given in GROUPING SETS +select t1.a, t2.b, sum(t1.v), count(*) from gstest_empty t1, gstest_empty t2 + group by grouping sets ((t1.a,t2.b),()); + +-- simple joins, var resolution, GROUPING on join vars +-- [SPARK-29698] Support grouping function with multiple arguments +-- select t1.a, t2.b, grouping(t1.a, t2.b), sum(t1.v), max(t2.a) +select t1.a, t2.b, grouping(t1.a), grouping(t2.b), sum(t1.v), max(t2.a) + from gstest1 t1, gstest2 t2 + group by grouping sets ((t1.a, t2.b), ()); + +-- [SPARK-29698] Support grouping function with multiple arguments +-- select t1.a, t2.b, grouping(t1.a, t2.b), sum(t1.v), max(t2.a) +select t1.a, t2.b, grouping(t1.a), grouping(t2.b), sum(t1.v), max(t2.a) + from gstest1 t1 join gstest2 t2 on (t1.a=t2.a) + group by grouping sets ((t1.a, t2.b), ()); + +-- [SPARK-29698] Support grouping function with multiple arguments +-- select a, b, grouping(a, b), sum(t1.v), max(t2.c) +select a, b, grouping(a), grouping(b), sum(t1.v), max(t2.c) + from gstest1 t1 join gstest2 t2 using (a,b) + group by grouping sets ((a, b), ()); + +-- check that functionally dependent cols are not nulled +-- [SPARK-29698] Support grouping function with multiple arguments +-- [SPARK-19842] Informational Referential Integrity Constraints Support in Spark +-- [SPARK-29702] Resolve group-by columns with functional dependencies +-- select a, d, grouping(a,b,c) +-- from gstest3 +-- group by grouping sets ((a,b), (a,c)); + +-- check that distinct grouping columns are kept separate +-- even if they are equal() +-- explain (costs off) +-- select g as alias1, g as alias2 +-- from generate_series(1,3) g +-- group by alias1, rollup(alias2); + +-- [SPARK-27767] Built-in function: generate_series +-- [SPARK-29704] Support the combinations of grouping operations +-- select g as alias1, g as alias2 +-- from generate_series(1,3) g +-- group by alias1, rollup(alias2); + +-- check that pulled-up subquery outputs still go to null when appropriate +select four, x + from (select four, ten, 'foo' as x from tenk1) as t + group by grouping sets (four, x) + having x = 'foo'; + +select four, x || 'x' + from (select four, ten, 'foo' as x from tenk1) as t + group by grouping sets (four, x) + order by four; + +select (x+y)*1, sum(z) + from (select 1 as x, 2 as y, 3 as z) s + group by grouping sets (x+y, x); + +CREATE TEMP VIEW int8_tbl AS SELECT * FROM VALUES + (123L, 456L), + (123L, 4567890123456789L), + (4567890123456789L, 123L), + (4567890123456789L, 4567890123456789L), + (4567890123456789L, -4567890123456789L) as int8_tbl(q1, q2); + +select x, not x as not_x, q2 from + (select *, q1 = 1 as x from int8_tbl i1) as t + group by grouping sets(x, q2) + order by x, q2; + +DROP VIEW int8_tbl; + +-- simple rescan tests + +-- Spark doesn't handle UDFs in SQL +-- select a, b, sum(v.x) +-- from (values (1),(2)) v(x), gstest_data(v.x) +-- group by rollup (a,b); + +-- Spark doesn't handle UDFs in SQL +-- select * +-- from (values (1),(2)) v(x), +-- lateral (select a, b, sum(v.x) from gstest_data(v.x) group by rollup (a,b)) s; + +-- min max optimization should still work with GROUP BY () +-- explain (costs off) +-- select min(unique1) from tenk1 GROUP BY (); + +-- Views with GROUPING SET queries +-- [SPARK-29698] Support grouping function with multiple arguments +-- [SPARK-29705] Support more expressive forms in GroupingSets/Cube/Rollup +-- CREATE VIEW gstest_view AS select a, b, grouping(a,b), sum(c), count(*), max(c) +-- from gstest2 group by rollup ((a,b,c),(c,d)); + +-- select pg_get_viewdef('gstest_view'::regclass, true); + +-- Nested queries with 3 or more levels of nesting +-- [SPARK-29698] Support grouping function with multiple arguments +-- [SPARK-29703] grouping() can only be used with GroupingSets/Cube/Rollup +-- select(select (select grouping(a,b) from (values (1)) v2(c)) from (values (1,2)) v1(a,b) group by (a,b)) from (values(6,7)) v3(e,f) GROUP BY ROLLUP(e,f); +-- select(select (select grouping(e,f) from (values (1)) v2(c)) from (values (1,2)) v1(a,b) group by (a,b)) from (values(6,7)) v3(e,f) GROUP BY ROLLUP(e,f); +-- select(select (select grouping(c) from (values (1)) v2(c) GROUP BY c) from (values (1,2)) v1(a,b) group by (a,b)) from (values(6,7)) v3(e,f) GROUP BY ROLLUP(e,f); + +-- Combinations of operations +-- [SPARK-29704] Support the combinations of grouping operations +-- select a, b, c, d from gstest2 group by rollup(a,b),grouping sets(c,d); +-- select a, b from (values (1,2),(2,3)) v(a,b) group by a,b, grouping sets(a); + +-- Spark doesn't handle UDFs in SQL +-- Tests for chained aggregates +-- select a, b, grouping(a,b), sum(v), count(*), max(v) +-- from gstest1 group by grouping sets ((a,b),(a+1,b+1),(a+2,b+2)) order by 3,6; +-- select(select (select grouping(a,b) from (values (1)) v2(c)) from (values (1,2)) v1(a,b) group by (a,b)) from (values(6,7)) v3(e,f) GROUP BY ROLLUP((e+1),(f+1)); +-- select(select (select grouping(a,b) from (values (1)) v2(c)) from (values (1,2)) v1(a,b) group by (a,b)) from (values(6,7)) v3(e,f) GROUP BY CUBE((e+1),(f+1)) ORDER BY (e+1),(f+1); +-- select a, b, sum(c), sum(sum(c)) over (order by a,b) as rsum +-- from gstest2 group by cube (a,b) order by rsum, a, b; +-- select a, b, sum(c) from (values (1,1,10),(1,1,11),(1,2,12),(1,2,13),(1,3,14),(2,3,15),(3,3,16),(3,4,17),(4,1,18),(4,1,19)) v(a,b,c) group by rollup (a,b); +-- select a, b, sum(v.x) +-- from (values (1),(2)) v(x), gstest_data(v.x) +-- group by cube (a,b) order by a,b; + +-- Test reordering of grouping sets +-- explain (costs off) +-- select * from gstest1 group by grouping sets((a,b,v),(v)) order by v,b,a; + +-- [SPARK-29698] Support grouping function with multiple arguments +-- [SPARK-29703] grouping() can only be used with GroupingSets/Cube/Rollup +-- Agg level check. This query should error out. +-- select (select grouping(a), grouping(b) from gstest2) from gstest2 group by a,b; + +--Nested queries +-- [SPARK-29700] Support nested grouping sets +-- select a, b, sum(c), count(*) from gstest2 group by grouping sets (rollup(a,b),a); + +-- HAVING queries +select ten, sum(distinct four) from onek a +group by grouping sets((ten,four),(ten)) +having exists (select 1 from onek b where sum(distinct a.four) = b.four); + +-- Tests around pushdown of HAVING clauses, partially testing against previous bugs +select a,count(*) from gstest2 group by rollup(a) order by a; +select a,count(*) from gstest2 group by rollup(a) having a is distinct from 1 order by a; +-- explain (costs off) +-- select a,count(*) from gstest2 group by rollup(a) having a is distinct from 1 order by a; + +-- [SPARK-29706] Support an empty grouping expression +-- select v.c, (select count(*) from gstest2 group by () having v.c) +-- from (values (false),(true)) v(c) order by v.c; +-- explain (costs off) +-- select v.c, (select count(*) from gstest2 group by () having v.c) +-- from (values (false),(true)) v(c) order by v.c; + +-- HAVING with GROUPING queries +select ten, grouping(ten) from onek +group by grouping sets(ten) having grouping(ten) >= 0 +order by 2,1; +select ten, grouping(ten) from onek +group by grouping sets(ten, four) having grouping(ten) > 0 +order by 2,1; +select ten, grouping(ten) from onek +group by rollup(ten) having grouping(ten) > 0 +order by 2,1; +select ten, grouping(ten) from onek +group by cube(ten) having grouping(ten) > 0 +order by 2,1; +-- [SPARK-29703] grouping() can only be used with GroupingSets/Cube/Rollup +-- select ten, grouping(ten) from onek +-- group by (ten) having grouping(ten) >= 0 +-- order by 2,1; + +-- FILTER queries +select ten, sum(distinct four) filter (where string(four) like '123') from onek a +group by rollup(ten); + +-- More rescan tests +-- [SPARK-35554] Support outer references in Aggregate +-- select * from (values (1),(2)) v(a) left join lateral (select v.a, four, ten, count(*) from onek group by cube(four,ten)) s on true order by v.a,four,ten; +-- [SPARK-27878] Support ARRAY(sub-SELECT) expressions +-- select array(select row(v.a,s1.*) from (select two,four, count(*) from onek group by cube(two,four) order by two,four) s1) from (values (1),(2)) v(a); + +-- [SPARK-29704] Support the combinations of grouping operations +-- Grouping on text columns +-- select sum(ten) from onek group by two, rollup(string(four)) order by 1; +-- select sum(ten) from onek group by rollup(string(four)), two order by 1; + +-- hashing support + +-- Ignore a PostgreSQL-specific option +-- set enable_hashagg = true; + +-- failure cases + +-- Since this test is implementation specific for plans, it passes in Spark +select count(*) from gstest4 group by rollup(unhashable_col,unsortable_col); +-- [SPARK-27878] Support ARRAY(sub-SELECT) expressions +-- select array_agg(v order by v) from gstest4 group by grouping sets ((id,unsortable_col),(id)); + +-- simple cases + +-- [SPARK-29698] Support grouping function with multiple arguments +-- select a, b, grouping(a,b), sum(v), count(*), max(v) +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by grouping sets ((a),(b)) order by 3,4,1,2 /* 3,1,2 */; +-- explain (costs off) select a, b, grouping(a,b), sum(v), count(*), max(v) +-- from gstest1 group by grouping sets ((a),(b)) order by 3,1,2; + +-- [SPARK-29698] Support grouping function with multiple arguments +-- select a, b, grouping(a,b), sum(v), count(*), max(v) +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by cube(a,b) order by 3,4,1,2 /* 3,1,2 */; +-- explain (costs off) select a, b, grouping(a,b), sum(v), count(*), max(v) +-- from gstest1 group by cube(a,b) order by 3,1,2; + +-- shouldn't try and hash +-- explain (costs off) +-- select a, b, grouping(a,b), array_agg(v order by v) +-- from gstest1 group by cube(a,b); + +-- unsortable cases +select unsortable_col, count(*) + from gstest4 group by grouping sets ((unsortable_col),(unsortable_col)) + order by string(unsortable_col); + +-- mixed hashable/sortable cases +-- [SPARK-29698] Support grouping function with multiple arguments +select unhashable_col, unsortable_col, + -- grouping(unhashable_col, unsortable_col), + grouping(unhashable_col), grouping(unsortable_col), + count(*), sum(v) + from gstest4 group by grouping sets ((unhashable_col),(unsortable_col)) + order by 3, 4, 6 /* 3, 5 */; +-- explain (costs off) +-- select unhashable_col, unsortable_col, +-- grouping(unhashable_col, unsortable_col), +-- count(*), sum(v) +-- from gstest4 group by grouping sets ((unhashable_col),(unsortable_col)) +-- order by 3,5; + +-- [SPARK-29698] Support grouping function with multiple arguments +select unhashable_col, unsortable_col, + -- grouping(unhashable_col, unsortable_col), + grouping(unhashable_col), grouping(unsortable_col), + count(*), sum(v) + from gstest4 group by grouping sets ((v,unhashable_col),(v,unsortable_col)) + order by 3, 4, 6 /* 3,5 */; +-- explain (costs off) +-- select unhashable_col, unsortable_col, +-- grouping(unhashable_col, unsortable_col), +-- count(*), sum(v) +-- from gstest4 group by grouping sets ((v,unhashable_col),(v,unsortable_col)) +-- order by 3,5; + +-- empty input: first is 0 rows, second 1, third 3 etc. +select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),a); +-- explain (costs off) +-- select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),a); +-- [SPARK-29701] Different answers when empty input given in GROUPING SETS +select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),()); +select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),(),(),()); +-- explain (costs off) +-- select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),(),(),()); +-- [SPARK-29701] Different answers when empty input given in GROUPING SETS +select sum(v), count(*) from gstest_empty group by grouping sets ((),(),()); +-- explain (costs off) +-- select sum(v), count(*) from gstest_empty group by grouping sets ((),(),()); + +-- [SPARK-29698] Support grouping function with multiple arguments +-- [SPARK-19842] Informational Referential Integrity Constraints Support in Spark +-- [SPARK-29702] Resolve group-by columns with functional dependencies +-- check that functionally dependent cols are not nulled +-- select a, d, grouping(a,b,c) +-- from gstest3 +-- group by grouping sets ((a,b), (a,c)); +-- explain (costs off) +-- select a, d, grouping(a,b,c) +-- from gstest3 +-- group by grouping sets ((a,b), (a,c)); + +-- simple rescan tests + +-- select a, b, sum(v.x) +-- from (values (1),(2)) v(x), gstest_data(v.x) +-- group by grouping sets (a,b) +-- order by 1, 2, 3; +-- explain (costs off) +-- select a, b, sum(v.x) +-- from (values (1),(2)) v(x), gstest_data(v.x) +-- group by grouping sets (a,b) +-- order by 3, 1, 2; +-- select * +-- from (values (1),(2)) v(x), +-- lateral (select a, b, sum(v.x) from gstest_data(v.x) group by grouping sets (a,b)) s; +-- explain (costs off) +-- select * +-- from (values (1),(2)) v(x), +-- lateral (select a, b, sum(v.x) from gstest_data(v.x) group by grouping sets (a,b)) s; + +-- Tests for chained aggregates +-- [SPARK-29698] Support grouping function with multiple arguments +-- select a, b, grouping(a,b), sum(v), count(*), max(v) +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by grouping sets ((a,b),(a+1,b+1),(a+2,b+2)) order by 3,4,7 /* 3,6 */; +-- explain (costs off) +-- select a, b, grouping(a,b), sum(v), count(*), max(v) +-- from gstest1 group by grouping sets ((a,b),(a+1,b+1),(a+2,b+2)) order by 3,6; +-- [SPARK-29699] Different answers in nested aggregates with window functions +select a, b, sum(c), sum(sum(c)) over (order by a,b) as rsum + from gstest2 group by cube (a,b) order by rsum, a, b; +-- explain (costs off) +-- select a, b, sum(c), sum(sum(c)) over (order by a,b) as rsum +-- from gstest2 group by cube (a,b) order by rsum, a, b; +-- select a, b, sum(v.x) +-- from (values (1),(2)) v(x), gstest_data(v.x) +-- group by cube (a,b) order by a,b; +-- explain (costs off) +-- select a, b, sum(v.x) +-- from (values (1),(2)) v(x), gstest_data(v.x) +-- group by cube (a,b) order by a,b; + +-- Verify that we correctly handle the child node returning a +-- non-minimal slot, which happens if the input is pre-sorted, +-- e.g. due to an index scan. +-- BEGIN; +-- Ignore a PostgreSQL-specific option +-- SET LOCAL enable_hashagg = false; +-- EXPLAIN (COSTS OFF) SELECT a, b, count(*), max(a), max(b) FROM gstest3 GROUP BY GROUPING SETS(a, b,()) ORDER BY a, b; +SELECT a, b, count(*), max(a), max(b) FROM gstest3 GROUP BY GROUPING SETS(a, b,()) ORDER BY a, b; +-- Ignore a PostgreSQL-specific option +-- SET LOCAL enable_seqscan = false; +-- EXPLAIN (COSTS OFF) SELECT a, b, count(*), max(a), max(b) FROM gstest3 GROUP BY GROUPING SETS(a, b,()) ORDER BY a, b; +-- SELECT a, b, count(*), max(a), max(b) FROM gstest3 GROUP BY GROUPING SETS(a, b,()) ORDER BY a, b; +-- COMMIT; + +-- More rescan tests +-- [SPARK-35554] Support outer references in Aggregate +-- select * from (values (1),(2)) v(a) left join lateral (select v.a, four, ten, count(*) from onek group by cube(four,ten)) s on true order by v.a,four,ten; +-- [SPARK-27878] Support ARRAY(sub-SELECT) expressions +-- select array(select row(v.a,s1.*) from (select two,four, count(*) from onek group by cube(two,four) order by two,four) s1) from (values (1),(2)) v(a); + +-- Rescan logic changes when there are no empty grouping sets, so test +-- that too: +-- [SPARK-35554] Support outer references in Aggregate +-- select * from (values (1),(2)) v(a) left join lateral (select v.a, four, ten, count(*) from onek group by grouping sets(four,ten)) s on true order by v.a,four,ten; +-- [SPARK-27878] Support ARRAY(sub-SELECT) expressions +-- select array(select row(v.a,s1.*) from (select two,four, count(*) from onek group by grouping sets(two,four) order by two,four) s1) from (values (1),(2)) v(a); + +-- test the knapsack + +-- Ignore a PostgreSQL-specific option +-- set enable_indexscan = false; +-- set work_mem = '64kB'; +-- explain (costs off) +-- select unique1, +-- count(two), count(four), count(ten), +-- count(hundred), count(thousand), count(twothousand), +-- count(*) +-- from tenk1 group by grouping sets (unique1,twothousand,thousand,hundred,ten,four,two); +-- explain (costs off) +-- select unique1, +-- count(two), count(four), count(ten), +-- count(hundred), count(thousand), count(twothousand), +-- count(*) +-- from tenk1 group by grouping sets (unique1,hundred,ten,four,two); + +-- Ignore a PostgreSQL-specific option +-- set work_mem = '384kB'; +-- explain (costs off) +-- select unique1, +-- count(two), count(four), count(ten), +-- count(hundred), count(thousand), count(twothousand), +-- count(*) +-- from tenk1 group by grouping sets (unique1,twothousand,thousand,hundred,ten,four,two); + +-- check collation-sensitive matching between grouping expressions +-- (similar to a check for aggregates, but there are additional code +-- paths for GROUPING, so check again here) + +-- [SPARK-28382] Array Functions: unnest +select v||'a', case grouping(v||'a') when 1 then 1 else 0 end, count(*) + -- from unnest(array[1,1], array['a','b']) u(i,v) + from values (1, 'a'), (1, 'b') u(i,v) + group by rollup(i, v||'a') order by 1,3; +select v||'a', case when grouping(v||'a') = 1 then 1 else 0 end, count(*) + -- from unnest(array[1,1], array['a','b']) u(i,v) + from values (1, 'a'), (1, 'b') u(i,v) + group by rollup(i, v||'a') order by 1,3; + +-- end + +DROP VIEW gstest1; +DROP TABLE gstest2; +DROP TABLE gstest3; +DROP TABLE gstest4; +DROP TABLE gstest_empty; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/insert.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/insert.sql new file mode 100644 index 000000000000..6783dda9ff01 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/insert.sql @@ -0,0 +1,653 @@ +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- INSERT +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/insert.sql + +-- +-- insert with DEFAULT in the target_list +-- +-- [SPARK-19842] Informational Referential Integrity Constraints Support in Spark +-- [SPARK-29119] DEFAULT option is not supported in Spark +create table inserttest (col1 int, col2 int /* NOT NULL */, col3 string /* default 'testing' */) using parquet; +-- [SPARK-29119] DEFAULT option is not supported in Spark +-- [SPARK-20845] Support specification of column names in INSERT INTO +-- Skip a test below because the PK constraint is violated and the query fails in PostgreSQL +-- insert into inserttest (col1, col2, col3) values (DEFAULT, DEFAULT, DEFAULT); +-- insert into inserttest (col2, col3) values (3, DEFAULT); +insert into inserttest values (NULL, 3, 'testing'); +-- insert into inserttest (col1, col2, col3) values (DEFAULT, 5, DEFAULT); +insert into inserttest values (NULL, 5, 'testing'); +-- insert into inserttest values (DEFAULT, 5, 'test'); +insert into inserttest values (NULL, 5, 'test'); +-- insert into inserttest values (DEFAULT, 7); +insert into inserttest values (NULL, 7, 'testing'); + +select * from inserttest; + +-- +-- insert with similar expression / target_list values (all fail) +-- +-- [SPARK-20845] Support specification of column names in INSERT INTO +-- [SPARK-29119] DEFAULT option is not supported in Spark +-- insert into inserttest (col1, col2, col3) values (DEFAULT, DEFAULT); +-- insert into inserttest (col1, col2, col3) values (1, 2); +-- insert into inserttest (col1) values (1, 2); +-- insert into inserttest (col1) values (DEFAULT, DEFAULT); + +-- select * from inserttest; + +-- +-- VALUES test +-- +-- [SPARK-29119] DEFAULT option is not supported in Spark +-- [SPARK-29715] Support SELECT statements in VALUES of INSERT INTO +-- insert into inserttest values(10, 20, '40'), (-1, 2, DEFAULT), +-- ((select 2), (select i from (values(3)) as foo (i)), 'values are fun!'); + +-- select * from inserttest; + +-- +-- TOASTed value test +-- +insert into inserttest values(30, 50, repeat('x', 10000)); + +select col1, col2, char_length(col3) from inserttest; + +drop table inserttest; + +-- +-- check indirection (field/array assignment), cf bug #14265 +-- +-- these tests are aware that transformInsertStmt has 3 separate code paths +-- + +-- [SPARK-29716] Support [CREATE|DROP] TYPE +-- create type insert_test_type as (if1 int, if2 array); + +-- create table inserttest (f1 int, f2 int[], +-- f3 insert_test_type, f4 insert_test_type[]); +-- +-- insert into inserttest (f2[1], f2[2]) values (1,2); +-- insert into inserttest (f2[1], f2[2]) values (3,4), (5,6); +-- insert into inserttest (f2[1], f2[2]) select 7,8; +-- insert into inserttest (f2[1], f2[2]) values (1,default); -- not supported +-- +-- insert into inserttest (f3.if1, f3.if2) values (1,array['foo']); +-- insert into inserttest (f3.if1, f3.if2) values (1,'{foo}'), (2,'{bar}'); +-- insert into inserttest (f3.if1, f3.if2) select 3, '{baz,quux}'; +-- insert into inserttest (f3.if1, f3.if2) values (1,default); -- not supported +-- +-- insert into inserttest (f3.if2[1], f3.if2[2]) values ('foo', 'bar'); +-- insert into inserttest (f3.if2[1], f3.if2[2]) values ('foo', 'bar'), ('baz', 'quux'); +-- insert into inserttest (f3.if2[1], f3.if2[2]) select 'bear', 'beer'; +-- +-- insert into inserttest (f4[1].if2[1], f4[1].if2[2]) values ('foo', 'bar'); +-- insert into inserttest (f4[1].if2[1], f4[1].if2[2]) values ('foo', 'bar'), ('baz', 'quux'); +-- insert into inserttest (f4[1].if2[1], f4[1].if2[2]) select 'bear', 'beer'; +-- +-- select * from inserttest; + +-- also check reverse-listing +-- create table inserttest2 (f1 bigint, f2 string); +-- [SPARK-29717] Support [CREATE|DROP] RULE - define a new plan rewrite rule +-- create rule irule1 as on insert to inserttest2 do also +-- insert into inserttest (f3.if2[1], f3.if2[2]) +-- values (new.f1,new.f2); +-- create rule irule2 as on insert to inserttest2 do also +-- insert into inserttest (f4[1].if1, f4[1].if2[2]) +-- values (1,'fool'),(new.f1,new.f2); +-- create rule irule3 as on insert to inserttest2 do also +-- insert into inserttest (f4[1].if1, f4[1].if2[2]) +-- select new.f1, new.f2; +-- \d+ inserttest2 + +-- drop table inserttest2; +-- drop table inserttest; +-- [SPARK-29716] Support [CREATE|DROP] TYPE +-- drop type insert_test_type; + +-- direct partition inserts should check partition bound constraint +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table range_parted ( +-- a string, +-- b int +-- ) partition by range (a, (b+0)); + +-- no partitions, so fail +-- insert into range_parted values ('a', 11); + +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table part1 partition of range_parted for values from ('a', 1) to ('a', 10); +-- create table part2 partition of range_parted for values from ('a', 10) to ('a', 20); +-- create table part3 partition of range_parted for values from ('b', 1) to ('b', 10); +-- create table part4 partition of range_parted for values from ('b', 10) to ('b', 20); + +-- fail +-- insert into part1 values ('a', 11); +-- insert into part1 values ('b', 1); +-- ok +-- insert into part1 values ('a', 1); +-- fail +-- insert into part4 values ('b', 21); +-- insert into part4 values ('a', 10); +-- ok +-- insert into part4 values ('b', 10); + +-- fail (partition key a has a NOT NULL constraint) +-- insert into part1 values (null); +-- fail (expression key (b+0) cannot be null either) +-- insert into part1 values (1); + +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table list_parted ( +-- a text, +-- b int +-- ) partition by list (lower(a)); +-- create table part_aa_bb partition of list_parted FOR VALUES IN ('aa', 'bb'); +-- create table part_cc_dd partition of list_parted FOR VALUES IN ('cc', 'dd'); +-- create table part_null partition of list_parted FOR VALUES IN (null); + +-- fail +-- insert into part_aa_bb values ('cc', 1); +-- insert into part_aa_bb values ('AAa', 1); +-- insert into part_aa_bb values (null); +-- ok +-- insert into part_cc_dd values ('cC', 1); +-- insert into part_null values (null, 0); + +-- check in case of multi-level partitioned table +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table part_ee_ff partition of list_parted for values in ('ee', 'ff') partition by range (b); +-- create table part_ee_ff1 partition of part_ee_ff for values from (1) to (10); +-- create table part_ee_ff2 partition of part_ee_ff for values from (10) to (20); + +-- test default partition +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table part_default partition of list_parted default; +-- Negative test: a row, which would fit in other partition, does not fit +-- default partition, even when inserted directly +-- insert into part_default values ('aa', 2); +-- insert into part_default values (null, 2); +-- ok +-- insert into part_default values ('Zz', 2); +-- test if default partition works as expected for multi-level partitioned +-- table as well as when default partition itself is further partitioned +-- drop table part_default; +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table part_xx_yy partition of list_parted for values in ('xx', 'yy') partition by list (a); +-- create table part_xx_yy_p1 partition of part_xx_yy for values in ('xx'); +-- create table part_xx_yy_defpart partition of part_xx_yy default; +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table part_default partition of list_parted default partition by range(b); +-- create table part_default_p1 partition of part_default for values from (20) to (30); +-- create table part_default_p2 partition of part_default for values from (30) to (40); + +-- fail +-- insert into part_ee_ff1 values ('EE', 11); +-- insert into part_default_p2 values ('gg', 43); +-- fail (even the parent's, ie, part_ee_ff's partition constraint applies) +-- insert into part_ee_ff1 values ('cc', 1); +-- insert into part_default values ('gg', 43); +-- ok +-- insert into part_ee_ff1 values ('ff', 1); +-- insert into part_ee_ff2 values ('ff', 11); +-- insert into part_default_p1 values ('cd', 25); +-- insert into part_default_p2 values ('de', 35); +-- insert into list_parted values ('ab', 21); +-- insert into list_parted values ('xx', 1); +-- insert into list_parted values ('yy', 2); +-- select tableoid::regclass, * from list_parted; + +-- Check tuple routing for partitioned tables + +-- fail +-- insert into range_parted values ('a', 0); +-- ok +-- insert into range_parted values ('a', 1); +-- insert into range_parted values ('a', 10); +-- fail +-- insert into range_parted values ('a', 20); +-- ok +-- insert into range_parted values ('b', 1); +-- insert into range_parted values ('b', 10); +-- fail (partition key (b+0) is null) +-- insert into range_parted values ('a'); + +-- Check default partition +-- create table part_def partition of range_parted default; +-- fail +-- insert into part_def values ('b', 10); +-- ok +-- insert into part_def values ('c', 10); +-- insert into range_parted values (null, null); +-- insert into range_parted values ('a', null); +-- insert into range_parted values (null, 19); +-- insert into range_parted values ('b', 20); + +-- select tableoid::regclass, * from range_parted; +-- ok +-- insert into list_parted values (null, 1); +-- insert into list_parted (a) values ('aA'); +-- fail (partition of part_ee_ff not found in both cases) +-- insert into list_parted values ('EE', 0); +-- insert into part_ee_ff values ('EE', 0); +-- ok +-- insert into list_parted values ('EE', 1); +-- insert into part_ee_ff values ('EE', 10); +-- select tableoid::regclass, * from list_parted; + +-- some more tests to exercise tuple-routing with multi-level partitioning +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table part_gg partition of list_parted for values in ('gg') partition by range (b); +-- create table part_gg1 partition of part_gg for values from (minvalue) to (1); +-- create table part_gg2 partition of part_gg for values from (1) to (10) partition by range (b); +-- create table part_gg2_1 partition of part_gg2 for values from (1) to (5); +-- create table part_gg2_2 partition of part_gg2 for values from (5) to (10); + +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table part_ee_ff3 partition of part_ee_ff for values from (20) to (30) partition by range (b); +-- create table part_ee_ff3_1 partition of part_ee_ff3 for values from (20) to (25); +-- create table part_ee_ff3_2 partition of part_ee_ff3 for values from (25) to (30); + +-- truncate list_parted; +-- insert into list_parted values ('aa'), ('cc'); +-- [SPARK-27767] Built-in function: generate_series +-- insert into list_parted select 'Ff', s.a from generate_series(1, 29) s(a); +-- insert into list_parted select 'gg', s.a from generate_series(1, 9) s(a); +-- insert into list_parted (b) values (1); +-- select tableoid::regclass::text, a, min(b) as min_b, max(b) as max_b from list_parted group by 1, 2 order by 1; + +-- direct partition inserts should check hash partition bound constraint + +-- Use hand-rolled hash functions and operator classes to get predictable +-- result on different matchines. The hash function for int4 simply returns +-- the sum of the values passed to it and the one for text returns the length +-- of the non-empty string value passed to it or 0. + +-- create or replace function part_hashint4_noop(value int4, seed int8) +-- returns int8 as $$ +-- select value + seed; +-- $$ language sql immutable; + +-- create operator class part_test_int4_ops +-- for type int4 +-- using hash as +-- operator 1 =, +-- function 2 part_hashint4_noop(int4, int8); + +-- create or replace function part_hashtext_length(value text, seed int8) +-- RETURNS int8 AS $$ +-- select length(coalesce(value, ''))::int8 +-- $$ language sql immutable; + +-- create operator class part_test_text_ops +-- for type text +-- using hash as +-- operator 1 =, +-- function 2 part_hashtext_length(text, int8); + +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table hash_parted ( +-- a int +-- ) partition by hash (a part_test_int4_ops); +-- create table hpart0 partition of hash_parted for values with (modulus 4, remainder 0); +-- create table hpart1 partition of hash_parted for values with (modulus 4, remainder 1); +-- create table hpart2 partition of hash_parted for values with (modulus 4, remainder 2); +-- create table hpart3 partition of hash_parted for values with (modulus 4, remainder 3); + +-- [SPARK-27767] Built-in function: generate_series +-- insert into hash_parted values(generate_series(1,10)); + +-- direct insert of values divisible by 4 - ok; +-- insert into hpart0 values(12),(16); +-- fail; +-- insert into hpart0 values(11); +-- 11 % 4 -> 3 remainder i.e. valid data for hpart3 partition +-- insert into hpart3 values(11); + +-- view data +-- select tableoid::regclass as part, a, a%4 as "remainder = a % 4" +-- from hash_parted order by part; + +-- test \d+ output on a table which has both partitioned and unpartitioned +-- partitions +-- \d+ list_parted + +-- cleanup +-- drop table range_parted, list_parted; +-- drop table hash_parted; + +-- test that a default partition added as the first partition accepts any value +-- including null +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table list_parted (a int) partition by list (a); +-- create table part_default partition of list_parted default; +-- \d+ part_default +-- insert into part_default values (null); +-- insert into part_default values (1); +-- insert into part_default values (-1); +-- select tableoid::regclass, a from list_parted; +-- cleanup +-- drop table list_parted; + +-- more tests for certain multi-level partitioning scenarios +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table mlparted (a int, b int) partition by range (a, b); +-- create table mlparted1 (b int not null, a int not null) partition by range ((b+0)); +-- create table mlparted11 (like mlparted1); +-- alter table mlparted11 drop a; +-- alter table mlparted11 add a int; +-- alter table mlparted11 drop a; +-- alter table mlparted11 add a int not null; +-- attnum for key attribute 'a' is different in mlparted, mlparted1, and mlparted11 +-- select attrelid::regclass, attname, attnum +-- from pg_attribute +-- where attname = 'a' +-- and (attrelid = 'mlparted'::regclass +-- or attrelid = 'mlparted1'::regclass +-- or attrelid = 'mlparted11'::regclass) +-- order by attrelid::regclass::text; + +-- alter table mlparted1 attach partition mlparted11 for values from (2) to (5); +-- alter table mlparted attach partition mlparted1 for values from (1, 2) to (1, 10); + +-- check that "(1, 2)" is correctly routed to mlparted11. +-- insert into mlparted values (1, 2); +-- select tableoid::regclass, * from mlparted; + +-- check that proper message is shown after failure to route through mlparted1 +-- insert into mlparted (a, b) values (1, 5); + +-- truncate mlparted; +-- alter table mlparted add constraint check_b check (b = 3); + +-- have a BR trigger modify the row such that the check_b is violated +-- create function mlparted11_trig_fn() +-- returns trigger AS +-- $$ +-- begin +-- NEW.b := 4; +-- return NEW; +-- end; +-- $$ +-- language plpgsql; +-- create trigger mlparted11_trig before insert ON mlparted11 +-- for each row execute procedure mlparted11_trig_fn(); + +-- check that the correct row is shown when constraint check_b fails after +-- "(1, 2)" is routed to mlparted11 (actually "(1, 4)" would be shown due +-- to the BR trigger mlparted11_trig_fn) +-- insert into mlparted values (1, 2); +-- drop trigger mlparted11_trig on mlparted11; +-- drop function mlparted11_trig_fn(); + +-- check that inserting into an internal partition successfully results in +-- checking its partition constraint before inserting into the leaf partition +-- selected by tuple-routing +-- insert into mlparted1 (a, b) values (2, 3); + +-- check routing error through a list partitioned table when the key is null +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table lparted_nonullpart (a int, b char) partition by list (b); +-- create table lparted_nonullpart_a partition of lparted_nonullpart for values in ('a'); +-- insert into lparted_nonullpart values (1); +-- drop table lparted_nonullpart; + +-- check that RETURNING works correctly with tuple-routing +-- alter table mlparted drop constraint check_b; +-- create table mlparted12 partition of mlparted1 for values from (5) to (10); +-- create table mlparted2 (b int not null, a int not null); +-- alter table mlparted attach partition mlparted2 for values from (1, 10) to (1, 20); +-- create table mlparted3 partition of mlparted for values from (1, 20) to (1, 30); +-- create table mlparted4 (like mlparted); +-- alter table mlparted4 drop a; +-- alter table mlparted4 add a int not null; +-- alter table mlparted attach partition mlparted4 for values from (1, 30) to (1, 40); +-- [SPARK-27767] Built-in function: generate_series +-- with ins (a, b, c) as +-- (insert into mlparted (b, a) select s.a, 1 from generate_series(2, 39) s(a) returning tableoid::regclass, *) +-- select a, b, min(c), max(c) from ins group by a, b order by 1; + +-- alter table mlparted add c text; +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table mlparted5 (c text, a int not null, b int not null) partition by list (c); +-- create table mlparted5a (a int not null, c text, b int not null); +-- alter table mlparted5 attach partition mlparted5a for values in ('a'); +-- alter table mlparted attach partition mlparted5 for values from (1, 40) to (1, 50); +-- alter table mlparted add constraint check_b check (a = 1 and b < 45); +-- insert into mlparted values (1, 45, 'a'); +-- create function mlparted5abrtrig_func() returns trigger as $$ begin new.c = 'b'; return new; end; $$ language plpgsql; +-- create trigger mlparted5abrtrig before insert on mlparted5a for each row execute procedure mlparted5abrtrig_func(); +-- insert into mlparted5 (a, b, c) values (1, 40, 'a'); +-- drop table mlparted5; +-- alter table mlparted drop constraint check_b; + +-- Check multi-level default partition +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table mlparted_def partition of mlparted default partition by range(a); +-- create table mlparted_def1 partition of mlparted_def for values from (40) to (50); +-- create table mlparted_def2 partition of mlparted_def for values from (50) to (60); +-- insert into mlparted values (40, 100); +-- insert into mlparted_def1 values (42, 100); +-- insert into mlparted_def2 values (54, 50); +-- fail +-- insert into mlparted values (70, 100); +-- insert into mlparted_def1 values (52, 50); +-- insert into mlparted_def2 values (34, 50); +-- ok +-- create table mlparted_defd partition of mlparted_def default; +-- insert into mlparted values (70, 100); + +-- select tableoid::regclass, * from mlparted_def; + +-- Check multi-level tuple routing with attributes dropped from the +-- top-most parent. First remove the last attribute. +-- alter table mlparted add d int, add e int; +-- alter table mlparted drop e; +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table mlparted5 partition of mlparted +-- for values from (1, 40) to (1, 50) partition by range (c); +-- create table mlparted5_ab partition of mlparted5 +-- for values from ('a') to ('c') partition by list (c); +-- This partitioned table should remain with no partitions. +-- create table mlparted5_cd partition of mlparted5 +-- for values from ('c') to ('e') partition by list (c); +-- create table mlparted5_a partition of mlparted5_ab for values in ('a'); +-- create table mlparted5_b (d int, b int, c text, a int); +-- alter table mlparted5_ab attach partition mlparted5_b for values in ('b'); +-- truncate mlparted; +-- insert into mlparted values (1, 2, 'a', 1); +-- insert into mlparted values (1, 40, 'a', 1); -- goes to mlparted5_a +-- insert into mlparted values (1, 45, 'b', 1); -- goes to mlparted5_b +-- insert into mlparted values (1, 45, 'c', 1); -- goes to mlparted5_cd, fails +-- insert into mlparted values (1, 45, 'f', 1); -- goes to mlparted5, fails +-- select tableoid::regclass, * from mlparted order by a, b, c, d; +-- alter table mlparted drop d; +-- truncate mlparted; +-- Remove the before last attribute. +-- alter table mlparted add e int, add d int; +-- alter table mlparted drop e; +-- insert into mlparted values (1, 2, 'a', 1); +-- insert into mlparted values (1, 40, 'a', 1); -- goes to mlparted5_a +-- insert into mlparted values (1, 45, 'b', 1); -- goes to mlparted5_b +-- insert into mlparted values (1, 45, 'c', 1); -- goes to mlparted5_cd, fails +-- insert into mlparted values (1, 45, 'f', 1); -- goes to mlparted5, fails +-- select tableoid::regclass, * from mlparted order by a, b, c, d; +-- alter table mlparted drop d; +-- drop table mlparted5; + +-- check that message shown after failure to find a partition shows the +-- appropriate key description (or none) in various situations +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table key_desc (a int, b int) partition by list ((a+0)); +-- create table key_desc_1 partition of key_desc for values in (1) partition by range (b); + +-- create user regress_insert_other_user; +-- grant select (a) on key_desc_1 to regress_insert_other_user; +-- grant insert on key_desc to regress_insert_other_user; + +-- set role regress_insert_other_user; +-- no key description is shown +-- insert into key_desc values (1, 1); + +-- reset role; +-- grant select (b) on key_desc_1 to regress_insert_other_user; +-- set role regress_insert_other_user; +-- key description (b)=(1) is now shown +-- insert into key_desc values (1, 1); + +-- key description is not shown if key contains expression +-- insert into key_desc values (2, 1); +-- reset role; +-- revoke all on key_desc from regress_insert_other_user; +-- revoke all on key_desc_1 from regress_insert_other_user; +-- drop role regress_insert_other_user; +-- drop table key_desc, key_desc_1; + +-- test minvalue/maxvalue restrictions +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table mcrparted (a int, b int, c int) partition by range (a, abs(b), c); +-- create table mcrparted0 partition of mcrparted for values from (minvalue, 0, 0) to (1, maxvalue, maxvalue); +-- create table mcrparted2 partition of mcrparted for values from (10, 6, minvalue) to (10, maxvalue, minvalue); +-- create table mcrparted4 partition of mcrparted for values from (21, minvalue, 0) to (30, 20, minvalue); + +-- check multi-column range partitioning expression enforces the same +-- constraint as what tuple-routing would determine it to be +-- create table mcrparted0 partition of mcrparted for values from (minvalue, minvalue, minvalue) to (1, maxvalue, maxvalue); +-- create table mcrparted1 partition of mcrparted for values from (2, 1, minvalue) to (10, 5, 10); +-- create table mcrparted2 partition of mcrparted for values from (10, 6, minvalue) to (10, maxvalue, maxvalue); +-- create table mcrparted3 partition of mcrparted for values from (11, 1, 1) to (20, 10, 10); +-- create table mcrparted4 partition of mcrparted for values from (21, minvalue, minvalue) to (30, 20, maxvalue); +-- create table mcrparted5 partition of mcrparted for values from (30, 21, 20) to (maxvalue, maxvalue, maxvalue); + +-- null not allowed in range partition +-- insert into mcrparted values (null, null, null); + +-- routed to mcrparted0 +-- insert into mcrparted values (0, 1, 1); +-- insert into mcrparted0 values (0, 1, 1); + +-- routed to mcparted1 +-- insert into mcrparted values (9, 1000, 1); +-- insert into mcrparted1 values (9, 1000, 1); +-- insert into mcrparted values (10, 5, -1); +-- insert into mcrparted1 values (10, 5, -1); +-- insert into mcrparted values (2, 1, 0); +-- insert into mcrparted1 values (2, 1, 0); + +-- routed to mcparted2 +-- insert into mcrparted values (10, 6, 1000); +-- insert into mcrparted2 values (10, 6, 1000); +-- insert into mcrparted values (10, 1000, 1000); +-- insert into mcrparted2 values (10, 1000, 1000); + +-- no partition exists, nor does mcrparted3 accept it +-- insert into mcrparted values (11, 1, -1); +-- insert into mcrparted3 values (11, 1, -1); + +-- routed to mcrparted5 +-- insert into mcrparted values (30, 21, 20); +-- insert into mcrparted5 values (30, 21, 20); +-- insert into mcrparted4 values (30, 21, 20); -- error + +-- check rows +-- select tableoid::regclass::text, * from mcrparted order by 1; + +-- cleanup +-- drop table mcrparted; + +-- check that a BR constraint can't make partition contain violating rows +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table brtrigpartcon (a int, b text) partition by list (a); +-- create table brtrigpartcon1 partition of brtrigpartcon for values in (1); +-- create or replace function brtrigpartcon1trigf() returns trigger as $$begin new.a := 2; return new; end$$ language plpgsql; +-- create trigger brtrigpartcon1trig before insert on brtrigpartcon1 for each row execute procedure brtrigpartcon1trigf(); +-- insert into brtrigpartcon values (1, 'hi there'); +-- insert into brtrigpartcon1 values (1, 'hi there'); + +-- check that the message shows the appropriate column description in a +-- situation where the partitioned table is not the primary ModifyTable node +-- create table inserttest3 (f1 text default 'foo', f2 text default 'bar', f3 int); +-- create role regress_coldesc_role; +-- grant insert on inserttest3 to regress_coldesc_role; +-- grant insert on brtrigpartcon to regress_coldesc_role; +-- revoke select on brtrigpartcon from regress_coldesc_role; +-- set role regress_coldesc_role; +-- with result as (insert into brtrigpartcon values (1, 'hi there') returning 1) +-- insert into inserttest3 (f3) select * from result; +-- reset role; + +-- cleanup +-- revoke all on inserttest3 from regress_coldesc_role; +-- revoke all on brtrigpartcon from regress_coldesc_role; +-- drop role regress_coldesc_role; +-- drop table inserttest3; +-- drop table brtrigpartcon; +-- drop function brtrigpartcon1trigf(); + +-- check that "do nothing" BR triggers work with tuple-routing (this checks +-- that estate->es_result_relation_info is appropriately set/reset for each +-- routed tuple) +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table donothingbrtrig_test (a int, b text) partition by list (a); +-- create table donothingbrtrig_test1 (b text, a int); +-- create table donothingbrtrig_test2 (c text, b text, a int); +-- alter table donothingbrtrig_test2 drop column c; +-- create or replace function donothingbrtrig_func() returns trigger as $$begin raise notice 'b: %', new.b; return NULL; end$$ language plpgsql; +-- create trigger donothingbrtrig1 before insert on donothingbrtrig_test1 for each row execute procedure donothingbrtrig_func(); +-- create trigger donothingbrtrig2 before insert on donothingbrtrig_test2 for each row execute procedure donothingbrtrig_func(); +-- alter table donothingbrtrig_test attach partition donothingbrtrig_test1 for values in (1); +-- alter table donothingbrtrig_test attach partition donothingbrtrig_test2 for values in (2); +-- insert into donothingbrtrig_test values (1, 'foo'), (2, 'bar'); +-- [SPARK-29386] Copy data between a file and a table +-- copy donothingbrtrig_test from stdout; +-- 1 baz +-- 2 qux +-- \. +-- select tableoid::regclass, * from donothingbrtrig_test; + +-- cleanup +-- drop table donothingbrtrig_test; +-- drop function donothingbrtrig_func(); + +-- check multi-column range partitioning with minvalue/maxvalue constraints +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table mcrparted (a text, b int) partition by range(a, b); +-- create table mcrparted1_lt_b partition of mcrparted for values from (minvalue, minvalue) to ('b', minvalue); +-- create table mcrparted2_b partition of mcrparted for values from ('b', minvalue) to ('c', minvalue); +-- create table mcrparted3_c_to_common partition of mcrparted for values from ('c', minvalue) to ('common', minvalue); +-- create table mcrparted4_common_lt_0 partition of mcrparted for values from ('common', minvalue) to ('common', 0); +-- create table mcrparted5_common_0_to_10 partition of mcrparted for values from ('common', 0) to ('common', 10); +-- create table mcrparted6_common_ge_10 partition of mcrparted for values from ('common', 10) to ('common', maxvalue); +-- create table mcrparted7_gt_common_lt_d partition of mcrparted for values from ('common', maxvalue) to ('d', minvalue); +-- create table mcrparted8_ge_d partition of mcrparted for values from ('d', minvalue) to (maxvalue, maxvalue); + +-- \d+ mcrparted +-- \d+ mcrparted1_lt_b +-- \d+ mcrparted2_b +-- \d+ mcrparted3_c_to_common +-- \d+ mcrparted4_common_lt_0 +-- \d+ mcrparted5_common_0_to_10 +-- \d+ mcrparted6_common_ge_10 +-- \d+ mcrparted7_gt_common_lt_d +-- \d+ mcrparted8_ge_d + +-- insert into mcrparted values ('aaa', 0), ('b', 0), ('bz', 10), ('c', -10), +-- ('comm', -10), ('common', -10), ('common', 0), ('common', 10), +-- ('commons', 0), ('d', -10), ('e', 0); +-- select tableoid::regclass, * from mcrparted order by a, b; +-- drop table mcrparted; + +-- check that wholerow vars in the RETURNING list work with partitioned tables +-- [SPARK-29718] Support PARTITION BY [RANGE|LIST|HASH] and PARTITION OF in CREATE TABLE +-- create table returningwrtest (a int) partition by list (a); +-- create table returningwrtest1 partition of returningwrtest for values in (1); +-- insert into returningwrtest values (1) returning returningwrtest; + +-- check also that the wholerow vars in RETURNING list are converted as needed +-- alter table returningwrtest add b text; +-- create table returningwrtest2 (b text, c int, a int); +-- alter table returningwrtest2 drop c; +-- alter table returningwrtest attach partition returningwrtest2 for values in (2); +-- insert into returningwrtest values (2, 'foo') returning returningwrtest; +-- drop table returningwrtest; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/int2.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/int2.sql new file mode 100644 index 000000000000..07f5976ca6d2 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/int2.sql @@ -0,0 +1,131 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- INT2 +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/int2.sql + +CREATE TABLE INT2_TBL(f1 smallint) USING parquet; + +-- [SPARK-28023] Trim the string when cast string type to other types +-- PostgreSQL implicitly casts string literals to data with integral types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO INT2_TBL VALUES (smallint(trim('0 '))); + +INSERT INTO INT2_TBL VALUES (smallint(trim(' 1234 '))); + +INSERT INTO INT2_TBL VALUES (smallint(trim(' -1234'))); + +-- [SPARK-27923] Invalid input syntax for type short throws exception at PostgreSQL +-- INSERT INTO INT2_TBL VALUES ('34.5'); + +-- largest and smallest values +-- PostgreSQL implicitly casts string literals to data with integral types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO INT2_TBL VALUES (smallint('32767')); + +INSERT INTO INT2_TBL VALUES (smallint('-32767')); + +-- bad input values -- should give errors +-- INSERT INTO INT2_TBL VALUES ('100000'); +-- INSERT INTO INT2_TBL VALUES ('asdf'); +-- INSERT INTO INT2_TBL VALUES (' '); +-- INSERT INTO INT2_TBL VALUES ('- 1234'); +-- INSERT INTO INT2_TBL VALUES ('4 444'); +-- INSERT INTO INT2_TBL VALUES ('123 dt'); +-- INSERT INTO INT2_TBL VALUES (''); + + +SELECT '' AS five, * FROM INT2_TBL; + +SELECT '' AS four, i.* FROM INT2_TBL i WHERE i.f1 <> smallint('0'); + +SELECT '' AS four, i.* FROM INT2_TBL i WHERE i.f1 <> int('0'); + +SELECT '' AS one, i.* FROM INT2_TBL i WHERE i.f1 = smallint('0'); + +SELECT '' AS one, i.* FROM INT2_TBL i WHERE i.f1 = int('0'); + +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 < smallint('0'); + +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 < int('0'); + +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 <= smallint('0'); + +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 <= int('0'); + +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 > smallint('0'); + +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 > int('0'); + +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 >= smallint('0'); + +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 >= int('0'); + +-- positive odds +SELECT '' AS one, i.* FROM INT2_TBL i WHERE (i.f1 % smallint('2')) = smallint('1'); + +-- any evens +SELECT '' AS three, i.* FROM INT2_TBL i WHERE (i.f1 % int('2')) = smallint('0'); + +-- [SPARK-28024] Incorrect value when out of range +-- SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT2_TBL i; + +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT2_TBL i +WHERE abs(f1) < 16384; + +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT2_TBL i; + +-- [SPARK-28024] Incorrect value when out of range +-- SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT2_TBL i; + +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT2_TBL i +WHERE f1 < 32766; + +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT2_TBL i; + +-- [SPARK-28024] Incorrect value when out of range +-- SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT2_TBL i; + +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT2_TBL i +WHERE f1 > -32767; + +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT2_TBL i; + +SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT2_TBL i; + +SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT2_TBL i; + +-- corner cases +SELECT string(shiftleft(smallint(-1), 15)); +SELECT string(smallint(shiftleft(smallint(-1), 15))+1); + +-- check sane handling of INT16_MIN overflow cases +-- [SPARK-28024] Incorrect numeric values when out of range +-- SELECT smallint((-32768)) * smallint(-1); +-- SELECT smallint(-32768) / smallint(-1); +SELECT smallint(-32768) % smallint(-1); + +-- [SPARK-28028] Cast numeric to integral type need round +-- check rounding when casting from float +SELECT x, smallint(x) AS int2_value +FROM (VALUES float(-2.5), + float(-1.5), + float(-0.5), + float(0.0), + float(0.5), + float(1.5), + float(2.5)) t(x); + +-- [SPARK-28028] Cast numeric to integral type need round +-- check rounding when casting from numeric +SELECT x, smallint(x) AS int2_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x); + +DROP TABLE INT2_TBL; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/int4.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/int4.sql new file mode 100644 index 000000000000..3a409eea3483 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/int4.sql @@ -0,0 +1,181 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- INT4 +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/int4.sql +-- + +CREATE TABLE INT4_TBL(f1 int) USING parquet; + +-- [SPARK-28023] Trim the string when cast string type to other types +-- PostgreSQL implicitly casts string literals to data with integral types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO INT4_TBL VALUES (int(trim(' 0 '))); + +INSERT INTO INT4_TBL VALUES (int(trim('123456 '))); + +INSERT INTO INT4_TBL VALUES (int(trim(' -123456'))); + +-- [SPARK-27923] Invalid input syntax for integer: "34.5" at PostgreSQL +-- INSERT INTO INT4_TBL(f1) VALUES ('34.5'); + +-- largest and smallest values +-- PostgreSQL implicitly casts string literals to data with integral types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO INT4_TBL VALUES (int('2147483647')); + +INSERT INTO INT4_TBL VALUES (int('-2147483647')); + +-- [SPARK-27923] Spark SQL insert these bad inputs to NULL +-- bad input values +-- INSERT INTO INT4_TBL(f1) VALUES ('1000000000000'); +-- INSERT INTO INT4_TBL(f1) VALUES ('asdf'); +-- INSERT INTO INT4_TBL(f1) VALUES (' '); +-- INSERT INTO INT4_TBL(f1) VALUES (' asdf '); +-- INSERT INTO INT4_TBL(f1) VALUES ('- 1234'); +-- INSERT INTO INT4_TBL(f1) VALUES ('123 5'); +-- INSERT INTO INT4_TBL(f1) VALUES (''); + +SELECT '' AS five, * FROM INT4_TBL; + +SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> smallint('0'); + +SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> int('0'); + +SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = smallint('0'); + +SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = int('0'); + +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < smallint('0'); + +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < int('0'); + +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= smallint('0'); + +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= int('0'); + +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > smallint('0'); + +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > int('0'); + +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= smallint('0'); + +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= int('0'); + +-- positive odds +SELECT '' AS one, i.* FROM INT4_TBL i WHERE (i.f1 % smallint('2')) = smallint('1'); + +-- any evens +SELECT '' AS three, i.* FROM INT4_TBL i WHERE (i.f1 % int('2')) = smallint('0'); + +-- [SPARK-28024] Incorrect value when out of range +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i; + +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i +WHERE abs(f1) < 1073741824; + +-- [SPARK-28024] Incorrect value when out of range +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i; + +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i +WHERE abs(f1) < 1073741824; + +-- [SPARK-28024] Incorrect value when out of range +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i; + +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i +WHERE f1 < 2147483646; + +-- [SPARK-28024] Incorrect value when out of range +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i; + +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i +WHERE f1 < 2147483646; + +-- [SPARK-28024] Incorrect value when out of range +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i; + +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i +WHERE f1 > -2147483647; + +-- [SPARK-28024] Incorrect value when out of range +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i; + +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i +WHERE f1 > -2147483647; + +SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT4_TBL i; + +SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT4_TBL i; + +-- +-- more complex expressions +-- + +-- variations on unary minus parsing +SELECT -2+3 AS one; + +SELECT 4-2 AS two; + +SELECT 2- -1 AS three; + +SELECT 2 - -2 AS four; + +SELECT smallint('2') * smallint('2') = smallint('16') / smallint('4') AS true; + +SELECT int('2') * smallint('2') = smallint('16') / int('4') AS true; + +SELECT smallint('2') * int('2') = int('16') / smallint('4') AS true; + +-- [SPARK-28349] We do not need to follow PostgreSQL to support reserved words in column alias +SELECT int('1000') < int('999') AS `false`; + +-- [SPARK-28027] Our ! and !! has different meanings +-- SELECT 4! AS twenty_four; + +-- SELECT !!3 AS six; + +SELECT 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 AS ten; + +SELECT 2 + 2 / 2 AS three; + +SELECT (2 + 2) / 2 AS two; + +-- [SPARK-28027] Add bitwise shift left/right operators +-- corner case +SELECT string(shiftleft(int(-1), 31)); +SELECT string(int(shiftleft(int(-1), 31))+1); + +-- [SPARK-28024] Incorrect numeric values when out of range +-- check sane handling of INT_MIN overflow cases +-- SELECT (-2147483648)::int4 * (-1)::int4; +-- SELECT (-2147483648)::int4 / (-1)::int4; +SELECT int(-2147483648) % int(-1); +-- SELECT (-2147483648)::int4 * (-1)::int2; +-- SELECT (-2147483648)::int4 / (-1)::int2; +SELECT int(-2147483648) % smallint(-1); + +-- [SPARK-28028] Cast numeric to integral type need round +-- check rounding when casting from float +SELECT x, int(x) AS int4_value +FROM (VALUES double(-2.5), + double(-1.5), + double(-0.5), + double(0.0), + double(0.5), + double(1.5), + double(2.5)) t(x); + +-- [SPARK-28028] Cast numeric to integral type need round +-- check rounding when casting from numeric +SELECT x, int(x) AS int4_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x); + +DROP TABLE INT4_TBL; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/int8.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/int8.sql new file mode 100644 index 000000000000..fac23b4a26f3 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/int8.sql @@ -0,0 +1,240 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- INT8 +-- Test int8 64-bit integers. +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/int8.sql +-- +CREATE TABLE INT8_TBL(q1 bigint, q2 bigint) USING parquet; + +-- PostgreSQL implicitly casts string literals to data with integral types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO INT8_TBL VALUES(bigint(trim(' 123 ')),bigint(trim(' 456'))); +INSERT INTO INT8_TBL VALUES(bigint(trim('123 ')),bigint('4567890123456789')); +INSERT INTO INT8_TBL VALUES(bigint('4567890123456789'),bigint('123')); +INSERT INTO INT8_TBL VALUES(+4567890123456789,bigint('4567890123456789')); +INSERT INTO INT8_TBL VALUES(bigint('+4567890123456789'),bigint('-4567890123456789')); + +-- [SPARK-27923] Spark SQL insert there bad inputs to NULL +-- bad inputs +-- INSERT INTO INT8_TBL(q1) VALUES (' '); +-- INSERT INTO INT8_TBL(q1) VALUES ('xxx'); +-- INSERT INTO INT8_TBL(q1) VALUES ('3908203590239580293850293850329485'); +-- INSERT INTO INT8_TBL(q1) VALUES ('-1204982019841029840928340329840934'); +-- INSERT INTO INT8_TBL(q1) VALUES ('- 123'); +-- INSERT INTO INT8_TBL(q1) VALUES (' 345 5'); +-- INSERT INTO INT8_TBL(q1) VALUES (''); + +SELECT * FROM INT8_TBL; + +-- int8/int8 cmp +SELECT * FROM INT8_TBL WHERE q2 = 4567890123456789; +SELECT * FROM INT8_TBL WHERE q2 <> 4567890123456789; +SELECT * FROM INT8_TBL WHERE q2 < 4567890123456789; +SELECT * FROM INT8_TBL WHERE q2 > 4567890123456789; +SELECT * FROM INT8_TBL WHERE q2 <= 4567890123456789; +SELECT * FROM INT8_TBL WHERE q2 >= 4567890123456789; + +-- int8/int4 cmp +SELECT * FROM INT8_TBL WHERE q2 = 456; +SELECT * FROM INT8_TBL WHERE q2 <> 456; +SELECT * FROM INT8_TBL WHERE q2 < 456; +SELECT * FROM INT8_TBL WHERE q2 > 456; +SELECT * FROM INT8_TBL WHERE q2 <= 456; +SELECT * FROM INT8_TBL WHERE q2 >= 456; + +-- int4/int8 cmp +SELECT * FROM INT8_TBL WHERE 123 = q1; +SELECT * FROM INT8_TBL WHERE 123 <> q1; +SELECT * FROM INT8_TBL WHERE 123 < q1; +SELECT * FROM INT8_TBL WHERE 123 > q1; +SELECT * FROM INT8_TBL WHERE 123 <= q1; +SELECT * FROM INT8_TBL WHERE 123 >= q1; + +-- int8/int2 cmp +SELECT * FROM INT8_TBL WHERE q2 = smallint('456'); +SELECT * FROM INT8_TBL WHERE q2 <> smallint('456'); +SELECT * FROM INT8_TBL WHERE q2 < smallint('456'); +SELECT * FROM INT8_TBL WHERE q2 > smallint('456'); +SELECT * FROM INT8_TBL WHERE q2 <= smallint('456'); +SELECT * FROM INT8_TBL WHERE q2 >= smallint('456'); + +-- int2/int8 cmp +SELECT * FROM INT8_TBL WHERE smallint('123') = q1; +SELECT * FROM INT8_TBL WHERE smallint('123') <> q1; +SELECT * FROM INT8_TBL WHERE smallint('123') < q1; +SELECT * FROM INT8_TBL WHERE smallint('123') > q1; +SELECT * FROM INT8_TBL WHERE smallint('123') <= q1; +SELECT * FROM INT8_TBL WHERE smallint('123') >= q1; + + +-- [SPARK-28349] We do not need to follow PostgreSQL to support reserved words in column alias +SELECT '' AS five, q1 AS plus, -q1 AS `minus` FROM INT8_TBL; + +SELECT '' AS five, q1, q2, q1 + q2 AS plus FROM INT8_TBL; +SELECT '' AS five, q1, q2, q1 - q2 AS `minus` FROM INT8_TBL; +SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL; +SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL + WHERE q1 < 1000 or (q2 > 0 and q2 < 1000); +SELECT '' AS five, q1, q2, q1 / q2 AS divide, q1 % q2 AS mod FROM INT8_TBL; + +SELECT '' AS five, q1, double(q1) FROM INT8_TBL; +SELECT '' AS five, q2, double(q2) FROM INT8_TBL; + +SELECT 37 + q1 AS plus4 FROM INT8_TBL; +SELECT 37 - q1 AS minus4 FROM INT8_TBL; +SELECT '' AS five, 2 * q1 AS `twice int4` FROM INT8_TBL; +SELECT '' AS five, q1 * 2 AS `twice int4` FROM INT8_TBL; + +-- int8 op int4 +SELECT q1 + int(42) AS `8plus4`, q1 - int(42) AS `8minus4`, q1 * int(42) AS `8mul4`, q1 / int(42) AS `8div4` FROM INT8_TBL; +-- int4 op int8 +SELECT int(246) + q1 AS `4plus8`, int(246) - q1 AS `4minus8`, int(246) * q1 AS `4mul8`, int(246) / q1 AS `4div8` FROM INT8_TBL; + +-- int8 op int2 +SELECT q1 + smallint(42) AS `8plus2`, q1 - smallint(42) AS `8minus2`, q1 * smallint(42) AS `8mul2`, q1 / smallint(42) AS `8div2` FROM INT8_TBL; +-- int2 op int8 +SELECT smallint(246) + q1 AS `2plus8`, smallint(246) - q1 AS `2minus8`, smallint(246) * q1 AS `2mul8`, smallint(246) / q1 AS `2div8` FROM INT8_TBL; + +SELECT q2, abs(q2) FROM INT8_TBL; +SELECT min(q1), min(q2) FROM INT8_TBL; +SELECT max(q1), max(q2) FROM INT8_TBL; + +-- TO_CHAR() +-- some queries are commented out as the format string is not supported by Spark +SELECT '' AS to_char_1, to_char(q1, '9G999G999G999G999G999'), to_char(q2, '9,999,999,999,999,999') +FROM INT8_TBL; + +-- SELECT '' AS to_char_2, to_char(q1, '9G999G999G999G999G999D999G999'), to_char(q2, '9,999,999,999,999,999.999,999') +-- FROM INT8_TBL; + +SELECT '' AS to_char_3, to_char( (q1 * -1), '9999999999999999PR'), to_char( (q2 * -1), '9999999999999999.999PR') +FROM INT8_TBL; + +SELECT '' AS to_char_4, to_char( (q1 * -1), '9999999999999999S'), to_char( (q2 * -1), 'S9999999999999999') +FROM INT8_TBL; + +SELECT '' AS to_char_5, to_char(q2, 'MI9999999999999999') FROM INT8_TBL; +-- SELECT '' AS to_char_6, to_char(q2, 'FMS9999999999999999') FROM INT8_TBL; +-- SELECT '' AS to_char_7, to_char(q2, 'FM9999999999999999THPR') FROM INT8_TBL; +-- SELECT '' AS to_char_8, to_char(q2, 'SG9999999999999999th') FROM INT8_TBL; +SELECT '' AS to_char_9, to_char(q2, '0999999999999999') FROM INT8_TBL; +SELECT '' AS to_char_10, to_char(q2, 'S0999999999999999') FROM INT8_TBL; +-- SELECT '' AS to_char_11, to_char(q2, 'FM0999999999999999') FROM INT8_TBL; +-- SELECT '' AS to_char_12, to_char(q2, 'FM9999999999999999.000') FROM INT8_TBL; +-- SELECT '' AS to_char_13, to_char(q2, 'L9999999999999999.000') FROM INT8_TBL; +-- SELECT '' AS to_char_14, to_char(q2, 'FM9999999999999999.999') FROM INT8_TBL; +-- SELECT '' AS to_char_15, to_char(q2, 'S 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 9 9') FROM INT8_TBL; +-- SELECT '' AS to_char_16, to_char(q2, E'99999 "text" 9999 "9999" 999 "\\"text between quote marks\\"" 9999') FROM INT8_TBL; +-- SELECT '' AS to_char_17, to_char(q2, '999999SG9999999999') FROM INT8_TBL; + +-- [SPARK-26218] Throw exception on overflow for integers +-- check min/max values and overflow behavior + +-- select bigint('-9223372036854775808'); +-- select bigint('-9223372036854775809'); +-- select bigint('9223372036854775807'); +-- select bigint('9223372036854775808'); + +-- select bigint('9223372036854775808'); + +-- select -(bigint('-9223372036854775807')); +-- select -(bigint('-9223372036854775808')); + +-- select bigint('9223372036854775800') + bigint('9223372036854775800'); +-- select bigint('-9223372036854775800') + bigint('-9223372036854775800'); + +-- select bigint('9223372036854775800') - bigint('-9223372036854775800'); +-- select bigint('-9223372036854775800') - bigint('9223372036854775800'); + +-- select bigint('9223372036854775800') * bigint('9223372036854775800'); + +select bigint('9223372036854775800') / bigint('0'); +-- select bigint('9223372036854775800') % bigint('0'); + +-- select abs(bigint('-9223372036854775808')); + +-- select bigint('9223372036854775800') + int('100'); +-- select bigint('-9223372036854775800') - int('100'); +-- select bigint('9223372036854775800') * int('100'); + +-- select int('100') + bigint('9223372036854775800'); +-- select int('-100') - bigint('9223372036854775800'); +-- select int('100') * bigint('9223372036854775800'); + +-- select bigint('9223372036854775800') + smallint('100'); +-- select bigint('-9223372036854775800') - smallint('100'); +-- select bigint('9223372036854775800') * smallint('100'); +select bigint('-9223372036854775808') / smallint('0'); + +-- select smallint('100') + bigint('9223372036854775800'); +-- select smallint('-100') - bigint('9223372036854775800'); +-- select smallint('100') * bigint('9223372036854775800'); +select smallint('100') / bigint('0'); + +SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 = 456; +SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 <> 456; + +SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 = 456; +SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 <> 456; + +SELECT CAST(smallint('42') AS bigint), CAST(smallint('-37') AS bigint); + +SELECT CAST(q1 AS float), CAST(q2 AS double) FROM INT8_TBL; +SELECT CAST(float('36854775807.0') AS bigint); +SELECT CAST(double('922337203685477580700.0') AS bigint); + + +-- [SPARK-28027] Missing some mathematical operators +-- bit operations + +-- SELECT q1, q2, q1 & q2 AS `and`, q1 | q2 AS `or`, q1 # q2 AS `xor`, ~q1 AS `not` FROM INT8_TBL; +SELECT q1, q2, q1 & q2 AS `and`, q1 | q2 AS `or`, ~q1 AS `not` FROM INT8_TBL; +-- SELECT q1, q1 << 2 AS `shl`, q1 >> 3 AS `shr` FROM INT8_TBL; + + +-- generate_series + +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799')); +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799'), 0); +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799'), 2); + +-- corner case +SELECT string(shiftleft(bigint(-1), 63)); +SELECT string(int(shiftleft(bigint(-1), 63))+1); + +-- [SPARK-26218] Throw exception on overflow for integers +-- check sane handling of INT64_MIN overflow cases +SELECT bigint((-9223372036854775808)) * bigint((-1)); +SELECT bigint((-9223372036854775808)) / bigint((-1)); +SELECT bigint((-9223372036854775808)) % bigint((-1)); +SELECT bigint((-9223372036854775808)) * int((-1)); +SELECT bigint((-9223372036854775808)) / int((-1)); +SELECT bigint((-9223372036854775808)) % int((-1)); +SELECT bigint((-9223372036854775808)) * smallint((-1)); +SELECT bigint((-9223372036854775808)) / smallint((-1)); +SELECT bigint((-9223372036854775808)) % smallint((-1)); + +-- [SPARK-28028] Cast numeric to integral type need round +-- check rounding when casting from float +SELECT x, bigint(x) AS int8_value +FROM (VALUES (double(-2.5)), + (double(-1.5)), + (double(-0.5)), + (double(0.0)), + (double(0.5)), + (double(1.5)), + (double(2.5))) t(x); + +-- check rounding when casting from numeric +SELECT x, bigint(x) AS int8_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x); + +DROP TABLE INT8_TBL; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/interval.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/interval.sql new file mode 100644 index 000000000000..eb8cc3441951 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/interval.sql @@ -0,0 +1,344 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- INTERVAL +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/interval.sql + +-- [SPARK-28259] Date/Time Output Styles and Date Order Conventions +-- SET DATESTYLE = 'ISO'; +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle to postgres; + +-- check acceptance of "time zone style" +-- [SPARK-29369] Accept strings without `interval` prefix in casting to intervals +-- [SPARK-29370] Interval strings without explicit unit markings +-- SELECT INTERVAL '01:00' AS `One hour`; +-- SELECT INTERVAL '+02:00' AS `Two hours`; +-- SELECT INTERVAL '-08:00' AS `Eight hours`; +-- SELECT INTERVAL '-1 +02:03' AS `22 hours ago...`; +-- SELECT INTERVAL '-1 days +02:03' AS `22 hours ago...`; +-- [SPARK-29371] Support interval field values with fractional parts +-- SELECT INTERVAL '1.5 weeks' AS `Ten days twelve hours`; +-- SELECT INTERVAL '1.5 months' AS `One month 15 days`; +-- SELECT INTERVAL '10 years -11 month -12 days +13:14' AS `9 years...`; + +-- [SPARK-29382] Support writing `INTERVAL` type to datasource table +-- CREATE TABLE INTERVAL_TBL (f1 interval); + +-- [SPARK-29383] Support the optional prefix `@` in interval strings +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 1 minute'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 5 hour'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 10 day'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 34 year'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 3 months'); +-- [SPARK-29384] Support `ago` in interval strings +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 14 seconds ago'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('1 day 2 hours 3 minutes 4 seconds'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('6 years'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('5 months'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('5 months 12 hours'); + +-- badly formatted interval +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('badly formatted interval'); +-- INSERT INTO INTERVAL_TBL (f1) VALUES ('@ 30 eons ago'); + +-- test interval operators + +-- SELECT '' AS ten, * FROM INTERVAL_TBL; +-- [SPARK-29385] Make `INTERVAL` values comparable +-- SELECT '' AS nine, * FROM INTERVAL_TBL +-- WHERE INTERVAL_TBL.f1 <> interval '@ 10 days'; + +-- SELECT '' AS three, * FROM INTERVAL_TBL +-- WHERE INTERVAL_TBL.f1 <= interval '@ 5 hours'; + +-- SELECT '' AS three, * FROM INTERVAL_TBL +-- WHERE INTERVAL_TBL.f1 < interval '@ 1 day'; + +-- SELECT '' AS one, * FROM INTERVAL_TBL +-- WHERE INTERVAL_TBL.f1 = interval '@ 34 years'; + +-- SELECT '' AS five, * FROM INTERVAL_TBL +-- WHERE INTERVAL_TBL.f1 >= interval '@ 1 month'; + +-- SELECT '' AS nine, * FROM INTERVAL_TBL +-- WHERE INTERVAL_TBL.f1 > interval '@ 3 seconds ago'; + +-- SELECT '' AS fortyfive, r1.*, r2.* +-- FROM INTERVAL_TBL r1, INTERVAL_TBL r2 +-- WHERE r1.f1 > r2.f1 +-- ORDER BY r1.f1, r2.f1; + +-- Test intervals that are large enough to overflow 64 bits in comparisons +-- [SPARK-29369] Accept strings without `interval` prefix in casting to intervals +-- CREATE TEMP TABLE INTERVAL_TBL_OF (f1 interval); +-- INSERT INTO INTERVAL_TBL_OF (f1) VALUES +-- ('2147483647 days 2147483647 months'), +-- ('2147483647 days -2147483648 months'), +-- ('1 year'), +-- ('-2147483648 days 2147483647 months'), +-- ('-2147483648 days -2147483648 months'); +-- these should fail as out-of-range +-- INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('2147483648 days'); +-- INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('-2147483649 days'); +-- INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('2147483647 years'); +-- INSERT INTO INTERVAL_TBL_OF (f1) VALUES ('-2147483648 years'); + +-- SELECT r1.*, r2.* +-- FROM INTERVAL_TBL_OF r1, INTERVAL_TBL_OF r2 +-- WHERE r1.f1 > r2.f1 +-- ORDER BY r1.f1, r2.f1; + +-- CREATE INDEX ON INTERVAL_TBL_OF USING btree (f1); +-- SET enable_seqscan TO false; +-- EXPLAIN (COSTS OFF) +-- SELECT f1 FROM INTERVAL_TBL_OF r1 ORDER BY f1; +-- SELECT f1 FROM INTERVAL_TBL_OF r1 ORDER BY f1; +-- RESET enable_seqscan; + +-- DROP TABLE INTERVAL_TBL_OF; + +-- Test multiplication and division with intervals. +-- Floating point arithmetic rounding errors can lead to unexpected results, +-- though the code attempts to do the right thing and round up to days and +-- minutes to avoid results such as '3 days 24:00 hours' or '14:20:60'. +-- Note that it is expected for some day components to be greater than 29 and +-- some time components be greater than 23:59:59 due to how intervals are +-- stored internally. +-- [SPARK-29386] Copy data between a file and a table +-- CREATE TABLE INTERVAL_MULDIV_TBL (span interval); +-- COPY INTERVAL_MULDIV_TBL FROM STDIN; +-- 41 mon 12 days 360:00 +-- -41 mon -12 days +360:00 +-- -12 days +-- 9 mon -27 days 12:34:56 +-- -3 years 482 days 76:54:32.189 +-- 4 mon +-- 14 mon +-- 999 mon 999 days +-- \. +-- [SPARK-29387] Support `*` and `\` operators for intervals +-- SELECT span * 0.3 AS product +-- FROM INTERVAL_MULDIV_TBL; + +-- SELECT span * 8.2 AS product +-- FROM INTERVAL_MULDIV_TBL; + +-- SELECT span / 10 AS quotient +-- FROM INTERVAL_MULDIV_TBL; + +-- SELECT span / 100 AS quotient +-- FROM INTERVAL_MULDIV_TBL; + +-- DROP TABLE INTERVAL_MULDIV_TBL; +-- [SPARK-28259] Date/Time Output Styles and Date Order Conventions +-- SET DATESTYLE = 'postgres'; +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle to postgres_verbose; + +-- SELECT '' AS ten, * FROM INTERVAL_TBL; + +-- test avg(interval), which is somewhat fragile since people have been +-- known to change the allowed input syntax for type interval without +-- updating pg_aggregate.agginitval + +-- select avg(f1) from interval_tbl; + +-- test long interval input +-- [SPARK-29388] Construct intervals from the `millenniums`, `centuries` or `decades` units +-- select '4 millenniums 5 centuries 4 decades 1 year 4 months 4 days 17 minutes 31 seconds'::interval; + +-- test long interval output +-- Note: the actual maximum length of the interval output is longer, +-- but we need the test to work for both integer and floating-point +-- timestamps. +-- [SPARK-29389] Support synonyms for interval units +-- select '100000000y 10mon -1000000000d -100000h -10min -10.000001s ago'::interval; + +-- test justify_hours() and justify_days() +-- [SPARK-29390] Add the justify_days(), justify_hours() and justify_interval() functions +-- SELECT justify_hours(interval '6 months 3 days 52 hours 3 minutes 2 seconds') as `6 mons 5 days 4 hours 3 mins 2 seconds`; +-- SELECT justify_days(interval '6 months 36 days 5 hours 4 minutes 3 seconds') as `7 mons 6 days 5 hours 4 mins 3 seconds`; + +-- test justify_interval() + +-- SELECT justify_interval(interval '1 month -1 hour') as `1 month -1 hour`; + +-- test fractional second input, and detection of duplicate units +-- [SPARK-28259] Date/Time Output Styles and Date Order Conventions +-- SET DATESTYLE = 'ISO'; +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle TO postgres; +-- [SPARK-29369] Accept strings without `interval` prefix in casting to intervals +-- SELECT '1 millisecond'::interval, '1 microsecond'::interval, +-- '500 seconds 99 milliseconds 51 microseconds'::interval; +-- SELECT '3 days 5 milliseconds'::interval; + +-- SELECT '1 second 2 seconds'::interval; -- error +-- SELECT '10 milliseconds 20 milliseconds'::interval; -- error +-- SELECT '5.5 seconds 3 milliseconds'::interval; -- error +-- SELECT '1:20:05 5 microseconds'::interval; -- error +-- SELECT '1 day 1 day'::interval; -- error +-- [SPARK-29391] Default year-month units +-- SELECT interval '1-2'; -- SQL year-month literal +SELECT interval '999' second; -- oversize leading field is ok +SELECT interval '999' minute; +SELECT interval '999' hour; +SELECT interval '999' day; +SELECT interval '999' month; + +-- test SQL-spec syntaxes for restricted field sets +SELECT interval '1' year; +SELECT interval '2' month; +SELECT interval '3' day; +SELECT interval '4' hour; +SELECT interval '5' minute; +SELECT interval '6' second; +-- [SPARK-29391] Default year-month units +-- SELECT interval '1' year to month; +SELECT interval '1-2' year to month; +-- [SPARK-29391] Default year-month units +-- SELECT interval '1 2' day to hour; +SELECT interval '1 2:03' day to hour; +SELECT interval '1 2:03:04' day to hour; +-- SELECT interval '1 2' day to minute; +SELECT interval '1 2:03' day to minute; +SELECT interval '1 2:03:04' day to minute; +-- SELECT interval '1 2' day to second; +SELECT interval '1 2:03' day to second; +SELECT interval '1 2:03:04' day to second; +-- SELECT interval '1 2' hour to minute; +SELECT interval '1 2:03' hour to minute; +SELECT interval '1 2:03:04' hour to minute; +-- SELECT interval '1 2' hour to second; +SELECT interval '1 2:03' hour to second; +SELECT interval '1 2:03:04' hour to second; +-- SELECT interval '1 2' minute to second; +SELECT interval '1 2:03' minute to second; +SELECT interval '1 2:03:04' minute to second; +-- [SPARK-29370] Interval strings without explicit unit markings +-- SELECT interval '1 +2:03' minute to second; +-- SELECT interval '1 +2:03:04' minute to second; +-- SELECT interval '1 -2:03' minute to second; +-- SELECT interval '1 -2:03:04' minute to second; +-- SELECT interval '123 11' day to hour; -- ok +-- SELECT interval '123 11' day; -- not ok +-- SELECT interval '123 11'; -- not ok, too ambiguous +-- SELECT interval '123 2:03 -2:04'; -- not ok, redundant hh:mm fields + +-- test syntaxes for restricted precision +-- [SPARK-29395] Precision of the interval type +-- SELECT interval(0) '1 day 01:23:45.6789'; +-- SELECT interval(2) '1 day 01:23:45.6789'; +-- SELECT interval '12:34.5678' minute to second(2); -- per SQL spec +-- SELECT interval '1.234' second; +-- SELECT interval '1.234' second(2); +-- SELECT interval '1 2.345' day to second(2); +-- SELECT interval '1 2:03' day to second(2); +-- SELECT interval '1 2:03.4567' day to second(2); +-- SELECT interval '1 2:03:04.5678' day to second(2); +-- SELECT interval '1 2.345' hour to second(2); +-- SELECT interval '1 2:03.45678' hour to second(2); +-- SELECT interval '1 2:03:04.5678' hour to second(2); +-- SELECT interval '1 2.3456' minute to second(2); +-- SELECT interval '1 2:03.5678' minute to second(2); +-- SELECT interval '1 2:03:04.5678' minute to second(2); + +-- test casting to restricted precision (bug #14479) +-- SELECT f1, f1::INTERVAL DAY TO MINUTE AS `minutes`, +-- (f1 + INTERVAL '1 month')::INTERVAL MONTH::INTERVAL YEAR AS `years` +-- FROM interval_tbl; + +-- test inputting and outputting SQL standard interval literals +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle TO sql_standard; +-- [SPARK-29407] Support syntax for zero interval +-- SELECT interval '0' AS zero, +-- interval '1-2' year to month AS `year-month`, +-- interval '1 2:03:04' day to second AS `day-time`, +-- [SPARK-29408] Support interval literal with negative sign `-` +-- - interval '1-2' AS `negative year-month`, +-- - interval '1 2:03:04' AS `negative day-time`; + +-- test input of some not-quite-standard interval values in the sql style +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle TO postgres; +-- SELECT interval '+1 -1:00:00', +-- interval '-1 +1:00:00', +-- interval '+1-2 -3 +4:05:06.789', +-- interval '-1-2 +3 -4:05:06.789'; + +-- test output of couple non-standard interval values in the sql style +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle TO sql_standard; +-- SELECT interval '1 day -1 hours', +-- interval '-1 days +1 hours', +-- interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds', +-- - interval '1 years 2 months -3 days 4 hours 5 minutes 6.789 seconds'; + +-- test outputting iso8601 intervals +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle to iso_8601; +-- select interval '0' AS zero, +-- interval '1-2' AS `a year 2 months`, +-- interval '1 2:03:04' AS `a bit over a day`, +-- interval '2:03:04.45679' AS `a bit over 2 hours`, +-- (interval '1-2' + interval '3 4:05:06.7') AS `all fields`, +-- (interval '1-2' - interval '3 4:05:06.7') AS `mixed sign`, +-- (- interval '1-2' + interval '3 4:05:06.7') AS negative; + +-- test inputting ISO 8601 4.4.2.1 "Format With Time Unit Designators" +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle to sql_standard; +-- [SPARK-29394] Support ISO 8601 format for intervals +-- select interval 'P0Y' AS zero, +-- interval 'P1Y2M' AS `a year 2 months`, +-- interval 'P1W' AS `a week`, +-- interval 'P1DT2H3M4S' AS `a bit over a day`, +-- interval 'P1Y2M3DT4H5M6.7S' AS `all fields`, +-- interval 'P-1Y-2M-3DT-4H-5M-6.7S' AS negative, +-- interval 'PT-0.1S' AS `fractional second`; + +-- test inputting ISO 8601 4.4.2.2 "Alternative Format" +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle to postgres; +-- select interval 'P00021015T103020' AS `ISO8601 Basic Format`, +-- interval 'P0002-10-15T10:30:20' AS `ISO8601 Extended Format`; + +-- Make sure optional ISO8601 alternative format fields are optional. +-- select interval 'P0002' AS `year only`, +-- interval 'P0002-10' AS `year month`, +-- interval 'P0002-10-15' AS `year month day`, +-- interval 'P0002T1S' AS `year only plus time`, +-- interval 'P0002-10T1S' AS `year month plus time`, +-- interval 'P0002-10-15T1S' AS `year month day plus time`, +-- interval 'PT10' AS `hour only`, +-- interval 'PT10:30' AS `hour minute`; + +-- test a couple rounding cases that changed since 8.3 w/ HAVE_INT64_TIMESTAMP. +-- [SPARK-29406] Interval output styles +-- SET IntervalStyle to postgres_verbose; +-- select interval '-10 mons -3 days +03:55:06.70'; +-- select interval '1 year 2 mons 3 days 04:05:06.699999'; +-- select interval '0:0:0.7', interval '@ 0.70 secs', interval '0.7 seconds'; + +-- check that '30 days' equals '1 month' according to the hash function +-- [SPARK-29385] Make `INTERVAL` values comparable +-- select '30 days'::interval = '1 month'::interval as t; +-- select interval_hash('30 days'::interval) = interval_hash('1 month'::interval) as t; + +-- numeric constructor +-- [SPARK-29393] Add the make_interval() function +-- select make_interval(years := 2); +-- select make_interval(years := 1, months := 6); +-- select make_interval(years := 1, months := -1, weeks := 5, days := -7, hours := 25, mins := -180); + +-- select make_interval() = make_interval(years := 0, months := 0, weeks := 0, days := 0, mins := 0, secs := 0.0); +-- select make_interval(hours := -2, mins := -10, secs := -25.3); + +-- select make_interval(years := 'inf'::float::int); +-- select make_interval(months := 'NaN'::float::int); +-- select make_interval(secs := 'inf'); +-- select make_interval(secs := 'NaN'); +-- select make_interval(secs := 7e12); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/join.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/join.sql new file mode 100644 index 000000000000..adc2af6aca7f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/join.sql @@ -0,0 +1,2106 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- JOIN +-- Test JOIN clauses +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/join.sql +-- + +-- There are 2 dimensions we want to test +-- 1. run with broadcast hash join, sort merge join or shuffle hash join. +-- 2. run with whole-stage-codegen, operator codegen or no codegen. + +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=10485760 +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.forceApplyShuffledHashJoin=true + +--CONFIG_DIM2 spark.sql.codegen.wholeStage=true +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +CREATE OR REPLACE TEMPORARY VIEW INT2_TBL(f1) AS VALUES + (smallint(trim('0 '))), + (smallint(trim(' 1234 '))), + (smallint(trim(' -1234'))), + (smallint('32767')), + (smallint('-32767')); +CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM + (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) + AS v(f1); +CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM + (VALUES + (123, 456), + (123, 4567890123456789), + (4567890123456789, 123), + (4567890123456789, 4567890123456789), + (4567890123456789, -4567890123456789)) + AS v(q1, q2); +CREATE OR REPLACE TEMPORARY VIEW FLOAT8_TBL AS SELECT * FROM + (VALUES (0.0), (1004.30), (-34.84), + (cast('1.2345678901234e+200' as double)), (cast('1.2345678901234e-200' as double))) + AS v(f1); +CREATE OR REPLACE TEMPORARY VIEW TEXT_TBL AS SELECT * FROM + (VALUES ('doh!'), ('hi de ho neighbor')) + AS v(f1); +CREATE OR REPLACE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1; + +CREATE TABLE J1_TBL ( + i integer, + j integer, + t string +) USING parquet; + +CREATE TABLE J2_TBL ( + i integer, + k integer +) USING parquet; + + +INSERT INTO J1_TBL VALUES (1, 4, 'one'); +INSERT INTO J1_TBL VALUES (2, 3, 'two'); +INSERT INTO J1_TBL VALUES (3, 2, 'three'); +INSERT INTO J1_TBL VALUES (4, 1, 'four'); +INSERT INTO J1_TBL VALUES (5, 0, 'five'); +INSERT INTO J1_TBL VALUES (6, 6, 'six'); +INSERT INTO J1_TBL VALUES (7, 7, 'seven'); +INSERT INTO J1_TBL VALUES (8, 8, 'eight'); +INSERT INTO J1_TBL VALUES (0, NULL, 'zero'); +INSERT INTO J1_TBL VALUES (NULL, NULL, 'null'); +INSERT INTO J1_TBL VALUES (NULL, 0, 'zero'); + +INSERT INTO J2_TBL VALUES (1, -1); +INSERT INTO J2_TBL VALUES (2, 2); +INSERT INTO J2_TBL VALUES (3, -3); +INSERT INTO J2_TBL VALUES (2, 4); +INSERT INTO J2_TBL VALUES (5, -5); +INSERT INTO J2_TBL VALUES (5, -5); +INSERT INTO J2_TBL VALUES (0, NULL); +INSERT INTO J2_TBL VALUES (NULL, NULL); +INSERT INTO J2_TBL VALUES (NULL, 0); + +-- [SPARK-20856] Do not need onerow because it only used for test statement using nested joins +-- useful in some tests below +-- create temp table onerow(); +-- insert into onerow default values; +-- analyze onerow; + + +-- +-- CORRELATION NAMES +-- Make sure that table/column aliases are supported +-- before diving into more complex join syntax. +-- + +SELECT '' AS `xxx`, * + FROM J1_TBL AS tx; + +SELECT '' AS `xxx`, * + FROM J1_TBL tx; + +SELECT '' AS `xxx`, * + FROM J1_TBL AS t1 (a, b, c); + +SELECT '' AS `xxx`, * + FROM J1_TBL t1 (a, b, c); + +SELECT '' AS `xxx`, * + FROM J1_TBL t1 (a, b, c), J2_TBL t2 (d, e); + +-- [SPARK-28377] Fully support correlation names in the FROM clause +-- SELECT '' AS "xxx", t1.a, t2.e +-- FROM J1_TBL t1 (a, b, c), J2_TBL t2 (d, e) +-- WHERE t1.a = t2.d; + + +-- +-- CROSS JOIN +-- Qualifications are not allowed on cross joins, +-- which degenerate into a standard unqualified inner join. +-- + +SELECT '' AS `xxx`, * + FROM J1_TBL CROSS JOIN J2_TBL; + +-- ambiguous column +SELECT '' AS `xxx`, i, k, t + FROM J1_TBL CROSS JOIN J2_TBL; + +-- resolve previous ambiguity by specifying the table name +SELECT '' AS `xxx`, t1.i, k, t + FROM J1_TBL t1 CROSS JOIN J2_TBL t2; + +SELECT '' AS `xxx`, ii, tt, kk + FROM (J1_TBL CROSS JOIN J2_TBL) + AS tx (ii, jj, tt, ii2, kk); + +-- [SPARK-28377] Fully support correlation names in the FROM clause +-- SELECT '' AS `xxx`, tx.ii, tx.jj, tx.kk +-- FROM (J1_TBL t1 (a, b, c) CROSS JOIN J2_TBL t2 (d, e)) +-- AS tx (ii, jj, tt, ii2, kk); + +SELECT '' AS `xxx`, * + FROM J1_TBL CROSS JOIN J2_TBL a CROSS JOIN J2_TBL b; + + +-- +-- +-- Inner joins (equi-joins) +-- +-- + +-- +-- Inner joins (equi-joins) with USING clause +-- The USING syntax changes the shape of the resulting table +-- by including a column in the USING clause only once in the result. +-- + +-- Inner equi-join on specified column +SELECT '' AS `xxx`, * + FROM J1_TBL INNER JOIN J2_TBL USING (i); + +-- Same as above, slightly different syntax +SELECT '' AS `xxx`, * + FROM J1_TBL JOIN J2_TBL USING (i); + +SELECT '' AS `xxx`, * + FROM J1_TBL t1 (a, b, c) JOIN J2_TBL t2 (a, d) USING (a) + ORDER BY a, d; + +-- [SPARK-28377] Fully support correlation names in the FROM clause +-- SELECT '' AS `xxx`, * +-- FROM J1_TBL t1 (a, b, c) JOIN J2_TBL t2 (a, b) USING (b) +-- ORDER BY b, t1.a; + + +-- +-- NATURAL JOIN +-- Inner equi-join on all columns with the same name +-- + +SELECT '' AS `xxx`, * + FROM J1_TBL NATURAL JOIN J2_TBL; + +SELECT '' AS `xxx`, * + FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (a, d); + +SELECT '' AS `xxx`, * + FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (d, a); + +-- [SPARK-28377] Fully support correlation names in the FROM clause +-- mismatch number of columns +-- currently, Postgres will fill in with underlying names +-- SELECT '' AS `xxx`, * +-- FROM J1_TBL t1 (a, b) NATURAL JOIN J2_TBL t2 (a); + + +-- +-- Inner joins (equi-joins) +-- + +SELECT '' AS `xxx`, * + FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = J2_TBL.i); + +SELECT '' AS `xxx`, * + FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = J2_TBL.k); + + +-- +-- Non-equi-joins +-- + +SELECT '' AS `xxx`, * + FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i <= J2_TBL.k); + + +-- +-- Outer joins +-- Note that OUTER is a noise word +-- + +SELECT '' AS `xxx`, * + FROM J1_TBL LEFT OUTER JOIN J2_TBL USING (i) + ORDER BY i, k, t; + +SELECT '' AS `xxx`, * + FROM J1_TBL LEFT JOIN J2_TBL USING (i) + ORDER BY i, k, t; + +SELECT '' AS `xxx`, * + FROM J1_TBL RIGHT OUTER JOIN J2_TBL USING (i); + +SELECT '' AS `xxx`, * + FROM J1_TBL RIGHT JOIN J2_TBL USING (i); + +SELECT '' AS `xxx`, * + FROM J1_TBL FULL OUTER JOIN J2_TBL USING (i) + ORDER BY i, k, t; + +SELECT '' AS `xxx`, * + FROM J1_TBL FULL JOIN J2_TBL USING (i) + ORDER BY i, k, t; + +SELECT '' AS `xxx`, * + FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (k = 1); + +SELECT '' AS `xxx`, * + FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (i = 1); + +-- +-- semijoin selectivity for <> +-- +-- explain (costs off) +-- select * from int4_tbl i4, tenk1 a +-- where exists(select * from tenk1 b +-- where a.twothousand = b.twothousand and a.fivethous <> b.fivethous) +-- and i4.f1 = a.tenthous; + + +-- +-- More complicated constructs +-- + +-- +-- Multiway full join +-- + +CREATE TABLE t1 (name STRING, n INTEGER) USING parquet; +CREATE TABLE t2 (name STRING, n INTEGER) USING parquet; +CREATE TABLE t3 (name STRING, n INTEGER) USING parquet; + +INSERT INTO t1 VALUES ( 'bb', 11 ); +INSERT INTO t2 VALUES ( 'bb', 12 ); +INSERT INTO t2 VALUES ( 'cc', 22 ); +INSERT INTO t2 VALUES ( 'ee', 42 ); +INSERT INTO t3 VALUES ( 'bb', 13 ); +INSERT INTO t3 VALUES ( 'cc', 23 ); +INSERT INTO t3 VALUES ( 'dd', 33 ); + +SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name); + +-- +-- Test interactions of join syntax and subqueries +-- + +-- Basic cases (we expect planner to pull up the subquery here) +SELECT * FROM +(SELECT * FROM t2) as s2 +INNER JOIN +(SELECT * FROM t3) s3 +USING (name); + +SELECT * FROM +(SELECT * FROM t2) as s2 +LEFT JOIN +(SELECT * FROM t3) s3 +USING (name); + +SELECT * FROM +(SELECT * FROM t2) as s2 +FULL JOIN +(SELECT * FROM t3) s3 +USING (name); + +-- Cases with non-nullable expressions in subquery results; +-- make sure these go to null as expected +SELECT * FROM +(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL INNER JOIN +(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3; + +SELECT * FROM +(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL LEFT JOIN +(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3; + +SELECT * FROM +(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL FULL JOIN +(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3; + +SELECT * FROM +(SELECT name, n as s1_n, 1 as s1_1 FROM t1) as s1 +NATURAL INNER JOIN +(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL INNER JOIN +(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3; + +SELECT * FROM +(SELECT name, n as s1_n, 1 as s1_1 FROM t1) as s1 +NATURAL FULL JOIN +(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL FULL JOIN +(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3; + +SELECT * FROM +(SELECT name, n as s1_n FROM t1) as s1 +NATURAL FULL JOIN + (SELECT * FROM + (SELECT name, n as s2_n FROM t2) as s2 + NATURAL FULL JOIN + (SELECT name, n as s3_n FROM t3) as s3 + ) ss2; + +SELECT * FROM +(SELECT name, n as s1_n FROM t1) as s1 +NATURAL FULL JOIN + (SELECT * FROM + (SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 + NATURAL FULL JOIN + (SELECT name, n as s3_n FROM t3) as s3 + ) ss2; + +-- Constants as join keys can also be problematic +SELECT * FROM + (SELECT name, n as s1_n FROM t1) as s1 +FULL JOIN + (SELECT name, 2 as s2_n FROM t2) as s2 +ON (s1_n = s2_n); + + +-- Test for propagation of nullability constraints into sub-joins + +create or replace temporary view x as select * from + (values (1,11), (2,22), (3,null), (4,44), (5,null)) + as v(x1, x2); + +create or replace temporary view y as select * from + (values (1,111), (2,222), (3,333), (4,null)) + as v(y1, y2); + +select * from x; +select * from y; + +select * from x left join y on (x1 = y1 and x2 is not null); +select * from x left join y on (x1 = y1 and y2 is not null); + +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1); +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1 and x2 is not null); +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1 and y2 is not null); +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1 and xx2 is not null); +-- these should NOT give the same answers as above +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1) where (x2 is not null); +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1) where (y2 is not null); +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1) where (xx2 is not null); + +-- +-- regression test: check for bug with propagation of implied equality +-- to outside an IN +-- +select count(*) from tenk1 a where unique1 in + (select unique1 from tenk1 b join tenk1 c using (unique1) + where b.unique2 = 42); + +-- +-- regression test: check for failure to generate a plan with multiple +-- degenerate IN clauses +-- +select count(*) from tenk1 x where + x.unique1 in (select a.f1 from int4_tbl a,float8_tbl b where a.f1=b.f1) and + x.unique1 = 0 and + x.unique1 in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=bb.f1); + +-- try that with GEQO too +-- begin; +-- set geqo = on; +-- set geqo_threshold = 2; +select count(*) from tenk1 x where + x.unique1 in (select a.f1 from int4_tbl a,float8_tbl b where a.f1=b.f1) and + x.unique1 = 0 and + x.unique1 in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=bb.f1); +-- rollback; + +-- Skip this test because table b inherits from table a and we do not support this feature, see inherits.sql +-- +-- regression test: be sure we cope with proven-dummy append rels +-- +-- explain (costs off) +-- select aa, bb, unique1, unique1 +-- from tenk1 right join b on aa = unique1 +-- where bb < bb and bb is null; + +-- select aa, bb, unique1, unique1 +-- from tenk1 right join b on aa = unique1 +-- where bb < bb and bb is null; + +-- +-- regression test: check handling of empty-FROM subquery underneath outer join +-- +-- explain (costs off) +-- select * from int8_tbl i1 left join (int8_tbl i2 join +-- (select 123 as x) ss on i2.q1 = x) on i1.q2 = i2.q2 +-- order by 1, 2; + +select * from int8_tbl i1 left join (int8_tbl i2 join + (select 123 as x) ss on i2.q1 = x) on i1.q2 = i2.q2 +order by 1, 2; + +-- +-- regression test: check a case where join_clause_is_movable_into() gives +-- an imprecise result, causing an assertion failure +-- +select count(*) +from + (select t3.tenthous as x1, coalesce(t1.stringu1, t2.stringu1) as x2 + from tenk1 t1 + left join tenk1 t2 on t1.unique1 = t2.unique1 + join tenk1 t3 on t1.unique2 = t3.unique2) ss, + tenk1 t4, + tenk1 t5 +where t4.thousand = t5.unique1 and ss.x1 = t4.tenthous and ss.x2 = t5.stringu1; + +-- +-- regression test: check a case where we formerly missed including an EC +-- enforcement clause because it was expected to be handled at scan level +-- +-- explain (costs off) +-- select a.f1, b.f1, t.thousand, t.tenthous from +-- tenk1 t, +-- (select sum(f1)+1 as f1 from int4_tbl i4a) a, +-- (select sum(f1) as f1 from int4_tbl i4b) b +-- where b.f1 = t.thousand and a.f1 = b.f1 and (a.f1+b.f1+999) = t.tenthous; + +select a.f1, b.f1, t.thousand, t.tenthous from + tenk1 t, + (select sum(f1)+1 as f1 from int4_tbl i4a) a, + (select sum(f1) as f1 from int4_tbl i4b) b +where b.f1 = t.thousand and a.f1 = b.f1 and (a.f1+b.f1+999) = t.tenthous; + +-- +-- check a case where we formerly got confused by conflicting sort orders +-- in redundant merge join path keys +-- +-- explain (costs off) +-- select * from +-- j1_tbl full join +-- (select * from j2_tbl order by j2_tbl.i desc, j2_tbl.k asc) j2_tbl +-- on j1_tbl.i = j2_tbl.i and j1_tbl.i = j2_tbl.k; + +select * from + j1_tbl full join + (select * from j2_tbl order by j2_tbl.i desc, j2_tbl.k asc) j2_tbl + on j1_tbl.i = j2_tbl.i and j1_tbl.i = j2_tbl.k; + +-- +-- a different check for handling of redundant sort keys in merge joins +-- +-- explain (costs off) +-- select count(*) from +-- (select * from tenk1 x order by x.thousand, x.twothousand, x.fivethous) x +-- left join +-- (select * from tenk1 y order by y.unique2) y +-- on x.thousand = y.unique2 and x.twothousand = y.hundred and x.fivethous = y.unique2; + +select count(*) from + (select * from tenk1 x order by x.thousand, x.twothousand, x.fivethous) x + left join + (select * from tenk1 y order by y.unique2) y + on x.thousand = y.unique2 and x.twothousand = y.hundred and x.fivethous = y.unique2; + + +-- +-- Clean up +-- + +DROP TABLE t1; +DROP TABLE t2; +DROP TABLE t3; + +DROP TABLE J1_TBL; +DROP TABLE J2_TBL; + +-- Both DELETE and UPDATE allow the specification of additional tables +-- to "join" against to determine which rows should be modified. + +-- CREATE TEMP TABLE t1 (a int, b int); +-- CREATE TEMP TABLE t2 (a int, b int); +-- CREATE TEMP TABLE t3 (x int, y int); + +-- INSERT INTO t1 VALUES (5, 10); +-- INSERT INTO t1 VALUES (15, 20); +-- INSERT INTO t1 VALUES (100, 100); +-- INSERT INTO t1 VALUES (200, 1000); +-- INSERT INTO t2 VALUES (200, 2000); +-- INSERT INTO t3 VALUES (5, 20); +-- INSERT INTO t3 VALUES (6, 7); +-- INSERT INTO t3 VALUES (7, 8); +-- INSERT INTO t3 VALUES (500, 100); + +-- DELETE FROM t3 USING t1 table1 WHERE t3.x = table1.a; +-- SELECT * FROM t3; +-- DELETE FROM t3 USING t1 JOIN t2 USING (a) WHERE t3.x > t1.a; +-- SELECT * FROM t3; +-- DELETE FROM t3 USING t3 t3_other WHERE t3.x = t3_other.x AND t3.y = t3_other.y; +-- SELECT * FROM t3; + +-- Test join against inheritance tree + +-- create temp table t2a () inherits (t2); + +-- insert into t2a values (200, 2001); + +-- select * from t1 left join t2 on (t1.a = t2.a); + +-- Test matching of column name with wrong alias + +-- select t1.x from t1 join t3 on (t1.a = t3.x); + +-- +-- regression test for 8.1 merge right join bug +-- + +create or replace temporary view tt1 as select * from + (values (1, 11), (2, NULL)) + as v(tt1_id, joincol); + +create or replace temporary view tt2 as select * from + (values (21, 11), (22, 11)) + as v(tt2_id, joincol); + +-- set enable_hashjoin to off; +-- set enable_nestloop to off; + +-- these should give the same results + +select tt1.*, tt2.* from tt1 left join tt2 on tt1.joincol = tt2.joincol; + +select tt1.*, tt2.* from tt2 right join tt1 on tt1.joincol = tt2.joincol; + +-- reset enable_hashjoin; +-- reset enable_nestloop; + +-- +-- regression test for bug #13908 (hash join with skew tuples & nbatch increase) +-- + +-- set work_mem to '64kB'; +-- set enable_mergejoin to off; + +-- explain (costs off) +-- select count(*) from tenk1 a, tenk1 b +-- where a.hundred = b.thousand and (b.fivethous % 10) < 10; +select count(*) from tenk1 a, tenk1 b + where a.hundred = b.thousand and (b.fivethous % 10) < 10; + +-- reset work_mem; +-- reset enable_mergejoin; + +-- +-- regression test for 8.2 bug with improper re-ordering of left joins +-- + +create or replace temporary view tt3 as select * from + (SELECT cast(x.id as int), repeat('xyzzy', 100) FROM range(1,10001) x) + as v(f1, f2); +-- create index tt3i on tt3(f1); +-- analyze tt3; + +create or replace temporary view tt4 as select * from + (values (0), (1), (9999)) + as v(f1); +-- analyze tt4; + +SELECT a.f1 +FROM tt4 a +LEFT JOIN ( + SELECT b.f1 + FROM tt3 b LEFT JOIN tt3 c ON (b.f1 = c.f1) + WHERE c.f1 IS NULL +) AS d ON (a.f1 = d.f1) +WHERE d.f1 IS NULL; + +-- +-- regression test for proper handling of outer joins within antijoins +-- + +-- create temp table tt4x(c1 int, c2 int, c3 int); + +-- explain (costs off) +-- select * from tt4x t1 +-- where not exists ( +-- select 1 from tt4x t2 +-- left join tt4x t3 on t2.c3 = t3.c1 +-- left join ( select t5.c1 as c1 +-- from tt4x t4 left join tt4x t5 on t4.c2 = t5.c1 +-- ) a1 on t3.c2 = a1.c1 +-- where t1.c1 = t2.c2 +-- ); + +-- +-- regression test for problems of the sort depicted in bug #3494 +-- + +create or replace temporary view tt5 as select * from + (values (1, 10), (1, 11)) + as v(f1, f2); +create or replace temporary view tt6 as select * from + (values (1, 9), (1, 2), (2, 9)) + as v(f1, f2); + +select * from tt5,tt6 where tt5.f1 = tt6.f1 and tt5.f1 = tt5.f2 - tt6.f2; + +-- +-- regression test for problems of the sort depicted in bug #3588 +-- + +create or replace temporary view xx as select * from + (values (1), (2), (3)) + as v(pkxx); +create or replace temporary view yy as select * from + (values (101, 1), (201, 2), (301, NULL)) + as v(pkyy, pkxx); + +select yy.pkyy as yy_pkyy, yy.pkxx as yy_pkxx, yya.pkyy as yya_pkyy, + xxa.pkxx as xxa_pkxx, xxb.pkxx as xxb_pkxx +from yy + left join (SELECT * FROM yy where pkyy = 101) as yya ON yy.pkyy = yya.pkyy + left join xx xxa on yya.pkxx = xxa.pkxx + left join xx xxb on coalesce (xxa.pkxx, 1) = xxb.pkxx; + +-- +-- regression test for improper pushing of constants across outer-join clauses +-- (as seen in early 8.2.x releases) +-- + +create or replace temporary view zt1 as select * from + (values (53)) + as v(f1); +create or replace temporary view zt2 as select * from + (values (53)) + as v(f2); +create or replace temporary view zt3(f3 int) using parquet; + +select * from + zt2 left join zt3 on (f2 = f3) + left join zt1 on (f3 = f1) +where f2 = 53; + +create temp view zv1 as select *,'dummy' AS junk from zt1; + +select * from + zt2 left join zt3 on (f2 = f3) + left join zv1 on (f3 = f1) +where f2 = 53; + +-- +-- regression test for improper extraction of OR indexqual conditions +-- (as seen in early 8.3.x releases) +-- + +select a.unique2, a.ten, b.tenthous, b.unique2, b.hundred +from tenk1 a left join tenk1 b on a.unique2 = b.tenthous +where a.unique1 = 42 and + ((b.unique2 is null and a.ten = 2) or b.hundred = 3); + +-- +-- test proper positioning of one-time quals in EXISTS (8.4devel bug) +-- +-- prepare foo(bool) as +-- select count(*) from tenk1 a left join tenk1 b +-- on (a.unique2 = b.unique1 and exists +-- (select 1 from tenk1 c where c.thousand = b.unique2 and $1)); +-- execute foo(true); +-- execute foo(false); + +-- +-- test for sane behavior with noncanonical merge clauses, per bug #4926 +-- + +-- begin; + +-- set enable_mergejoin = 1; +-- set enable_hashjoin = 0; +-- set enable_nestloop = 0; + +create or replace temporary view a (i integer) using parquet; +create or replace temporary view b (x integer, y integer) using parquet; + +select * from a left join b on i = x and i = y and x = i; + +-- rollback; + +-- +-- test handling of merge clauses using record_ops +-- +-- begin; + +-- create type mycomptype as (id int, v bigint); + +-- create temp table tidv (idv mycomptype); +-- create index on tidv (idv); + +-- explain (costs off) +-- select a.idv, b.idv from tidv a, tidv b where a.idv = b.idv; + +-- set enable_mergejoin = 0; + +-- explain (costs off) +-- select a.idv, b.idv from tidv a, tidv b where a.idv = b.idv; + +-- rollback; + +-- +-- test NULL behavior of whole-row Vars, per bug #5025 +-- +--- [SPARK-34199] changed the `count(t2.*)` to `count(t2.q1, t2.q2)` since we have +--- blocked `count(tblName.*)`. Besides this, in pgsql, `count(t2.*)` of outter join +--- means how many matching rows produced by t2 while Spark SQL doesn't have this semantic. +--- So here we use `count(t2.q1, t2.q2)` instead of `count(1)` to keep the query output +--- unchanged. +select t1.q2, count(t2.q1, t2.q2) +from int8_tbl t1 left join int8_tbl t2 on (t1.q2 = t2.q1) +group by t1.q2 order by 1; + +select t1.q2, count(t2.q1, t2.q2) +from int8_tbl t1 left join (select * from int8_tbl) t2 on (t1.q2 = t2.q1) +group by t1.q2 order by 1; + +-- [SPARK-28330] Enhance query limit +-- select t1.q2, count(t2.q1, t2.q2) +-- from int8_tbl t1 left join (select * from int8_tbl offset 0) t2 on (t1.q2 = t2.q1) +-- group by t1.q2 order by 1; + +select t1.q2, count(t2.q1, t2.q2) +from int8_tbl t1 left join + (select q1, case when q2=1 then 1 else q2 end as q2 from int8_tbl) t2 + on (t1.q2 = t2.q1) +group by t1.q2 order by 1; + +-- +-- test incorrect failure to NULL pulled-up subexpressions +-- +-- begin; +create or replace temporary view a as select * from + (values ('p'), ('q')) + as v(code); +create or replace temporary view b as select * from + (values ('p', 1), ('p', 2)) + as v(a, num); +create or replace temporary view c as select * from + (values ('A', 'p'), ('B', 'q'), ('C', null)) + as v(name, a); + +select c.name, ss.code, ss.b_cnt, ss.const +from c left join + (select a.code, coalesce(b_grp.cnt, 0) as b_cnt, -1 as const + from a left join + (select count(1) as cnt, b.a from b group by b.a) as b_grp + on a.code = b_grp.a + ) as ss + on (c.a = ss.code) +order by c.name; + +-- rollback; + +-- +-- test incorrect handling of placeholders that only appear in targetlists, +-- per bug #6154 +-- +SELECT * FROM +( SELECT 1 as key1 ) sub1 +LEFT JOIN +( SELECT sub3.key3, sub4.value2, COALESCE(sub4.value2, 66) as value3 FROM + ( SELECT 1 as key3 ) sub3 + LEFT JOIN + ( SELECT sub5.key5, COALESCE(sub6.value1, 1) as value2 FROM + ( SELECT 1 as key5 ) sub5 + LEFT JOIN + ( SELECT 2 as key6, 42 as value1 ) sub6 + ON sub5.key5 = sub6.key6 + ) sub4 + ON sub4.key5 = sub3.key3 +) sub2 +ON sub1.key1 = sub2.key3; + +-- test the path using join aliases, too +SELECT * FROM +( SELECT 1 as key1 ) sub1 +LEFT JOIN +( SELECT sub3.key3, value2, COALESCE(value2, 66) as value3 FROM + ( SELECT 1 as key3 ) sub3 + LEFT JOIN + ( SELECT sub5.key5, COALESCE(sub6.value1, 1) as value2 FROM + ( SELECT 1 as key5 ) sub5 + LEFT JOIN + ( SELECT 2 as key6, 42 as value1 ) sub6 + ON sub5.key5 = sub6.key6 + ) sub4 + ON sub4.key5 = sub3.key3 +) sub2 +ON sub1.key1 = sub2.key3; + +-- +-- test case where a PlaceHolderVar is used as a nestloop parameter +-- + +-- EXPLAIN (COSTS OFF) +-- SELECT qq, unique1 +-- FROM +-- ( SELECT COALESCE(q1, 0) AS qq FROM int8_tbl a ) AS ss1 +-- FULL OUTER JOIN +-- ( SELECT COALESCE(q2, -1) AS qq FROM int8_tbl b ) AS ss2 +-- USING (qq) +-- INNER JOIN tenk1 c ON qq = unique2; + +SELECT qq, unique1 + FROM + ( SELECT COALESCE(q1, 0) AS qq FROM int8_tbl a ) AS ss1 + FULL OUTER JOIN + ( SELECT COALESCE(q2, -1) AS qq FROM int8_tbl b ) AS ss2 + USING (qq) + INNER JOIN tenk1 c ON qq = unique2; + +-- +-- nested nestloops can require nested PlaceHolderVars +-- + +create or replace temporary view nt1 as select * from + (values(1,true,true), (2,true,false), (3,false,false)) + as v(id, a1, a2); +create or replace temporary view nt2 as select * from + (values(1,1,true,true), (2,2,true,false), (3,3,false,false)) + as v(id, nt1_id, b1, b2); +create or replace temporary view nt3 as select * from + (values(1,1,true), (2,2,false), (3,3,true)) + as v(id, nt2_id, c1); +-- explain (costs off) +-- select nt3.id +-- from nt3 as nt3 +-- left join +-- (select nt2.*, (nt2.b1 and ss1.a3) AS b3 +-- from nt2 as nt2 +-- left join +-- (select nt1.*, (nt1.id is not null) as a3 from nt1) as ss1 +-- on ss1.id = nt2.nt1_id +-- ) as ss2 +-- on ss2.id = nt3.nt2_id +-- where nt3.id = 1 and ss2.b3; + +select nt3.id +from nt3 as nt3 + left join + (select nt2.*, (nt2.b1 and ss1.a3) AS b3 + from nt2 as nt2 + left join + (select nt1.*, (nt1.id is not null) as a3 from nt1) as ss1 + on ss1.id = nt2.nt1_id + ) as ss2 + on ss2.id = nt3.nt2_id +where nt3.id = 1 and ss2.b3; + +-- Accessing outer query column is not allowed in LocalLimit +-- +-- test case where a PlaceHolderVar is propagated into a subquery +-- + +-- explain (costs off) +-- select * from +-- int8_tbl t1 left join +-- (select q1 as x, 42 as y from int8_tbl t2) ss +-- on t1.q2 = ss.x +-- where +-- 1 = (select 1 from int8_tbl t3 where ss.y is not null limit 1) +-- order by 1,2; + +-- select * from +-- int8_tbl t1 left join +-- (select q1 as x, 42 as y from int8_tbl t2) ss +-- on t1.q2 = ss.x +-- where +-- 1 = (select 1 from int8_tbl t3 where ss.y is not null limit 1) +-- order by 1,2; + +-- +-- test the corner cases FULL JOIN ON TRUE and FULL JOIN ON FALSE +-- +select * from int4_tbl a full join int4_tbl b on true; +select * from int4_tbl a full join int4_tbl b on false; + +-- +-- test for ability to use a cartesian join when necessary +-- + +-- explain (costs off) +-- select * from +-- tenk1 join int4_tbl on f1 = twothousand, +-- int4(sin(1)) q1, +-- int4(sin(0)) q2 +-- where q1 = thousand or q2 = thousand; + +-- explain (costs off) +-- select * from +-- tenk1 join int4_tbl on f1 = twothousand, +-- int4(sin(1)) q1, +-- int4(sin(0)) q2 +-- where thousand = (q1 + q2); + +-- +-- test ability to generate a suitable plan for a star-schema query +-- + +-- explain (costs off) +-- select * from +-- tenk1, int8_tbl a, int8_tbl b +-- where thousand = a.q1 and tenthous = b.q1 and a.q2 = 1 and b.q2 = 2; + +-- +-- test a corner case in which we shouldn't apply the star-schema optimization +-- + +-- explain (costs off) +-- select t1.unique2, t1.stringu1, t2.unique1, t2.stringu2 from +-- tenk1 t1 +-- inner join int4_tbl i1 +-- left join (select v1.x2, v2.y1, 11 AS d1 +-- from (select 1,0 from onerow) v1(x1,x2) +-- left join (select 3,1 from onerow) v2(y1,y2) +-- on v1.x1 = v2.y2) subq1 +-- on (i1.f1 = subq1.x2) +-- on (t1.unique2 = subq1.d1) +-- left join tenk1 t2 +-- on (subq1.y1 = t2.unique1) +-- where t1.unique2 < 42 and t1.stringu1 > t2.stringu2; + +-- [SPARK-20856] support statement using nested joins +-- select t1.unique2, t1.stringu1, t2.unique1, t2.stringu2 from +-- tenk1 t1 +-- inner join int4_tbl i1 +-- left join (select v1.x2, v2.y1, 11 AS d1 +-- from (select 1,0 from onerow) v1(x1,x2) +-- left join (select 3,1 from onerow) v2(y1,y2) +-- on v1.x1 = v2.y2) subq1 +-- on (i1.f1 = subq1.x2) +-- on (t1.unique2 = subq1.d1) +-- left join tenk1 t2 +-- on (subq1.y1 = t2.unique1) +-- where t1.unique2 < 42 and t1.stringu1 > t2.stringu2; + +-- variant that isn't quite a star-schema case + +-- [SPARK-16452] basic INFORMATION_SCHEMA support +-- select ss1.d1 from +-- tenk1 as t1 +-- inner join tenk1 as t2 +-- on t1.tenthous = t2.ten +-- inner join +-- int8_tbl as i8 +-- left join int4_tbl as i4 +-- inner join (select 64::information_schema.cardinal_number as d1 +-- from tenk1 t3, +-- lateral (select abs(t3.unique1) + random()) ss0(x) +-- where t3.fivethous < 0) as ss1 +-- on i4.f1 = ss1.d1 +-- on i8.q1 = i4.f1 +-- on t1.tenthous = ss1.d1 +-- where t1.unique1 < i4.f1; + +-- this variant is foldable by the remove-useless-RESULT-RTEs code + +-- explain (costs off) +-- select t1.unique2, t1.stringu1, t2.unique1, t2.stringu2 from +-- tenk1 t1 +-- inner join int4_tbl i1 +-- left join (select v1.x2, v2.y1, 11 AS d1 +-- from (values(1,0)) v1(x1,x2) +-- left join (values(3,1)) v2(y1,y2) +-- on v1.x1 = v2.y2) subq1 +-- on (i1.f1 = subq1.x2) +-- on (t1.unique2 = subq1.d1) +-- left join tenk1 t2 +-- on (subq1.y1 = t2.unique1) +-- where t1.unique2 < 42 and t1.stringu1 > t2.stringu2; + +-- [SPARK-20856] support statement using nested joins +-- select t1.unique2, t1.stringu1, t2.unique1, t2.stringu2 from +-- tenk1 t1 +-- inner join int4_tbl i1 +-- left join (select v1.x2, v2.y1, 11 AS d1 +-- from (values(1,0)) v1(x1,x2) +-- left join (values(3,1)) v2(y1,y2) +-- on v1.x1 = v2.y2) subq1 +-- on (i1.f1 = subq1.x2) +-- on (t1.unique2 = subq1.d1) +-- left join tenk1 t2 +-- on (subq1.y1 = t2.unique1) +-- where t1.unique2 < 42 and t1.stringu1 > t2.stringu2; + +-- +-- test extraction of restriction OR clauses from join OR clause +-- (we used to only do this for indexable clauses) +-- + +-- explain (costs off) +-- select * from tenk1 a join tenk1 b on +-- (a.unique1 = 1 and b.unique1 = 2) or (a.unique2 = 3 and b.hundred = 4); +-- explain (costs off) +-- select * from tenk1 a join tenk1 b on +-- (a.unique1 = 1 and b.unique1 = 2) or (a.unique2 = 3 and b.ten = 4); +-- explain (costs off) +-- select * from tenk1 a join tenk1 b on +-- (a.unique1 = 1 and b.unique1 = 2) or +-- ((a.unique2 = 3 or a.unique2 = 7) and b.hundred = 4); + +-- +-- test placement of movable quals in a parameterized join tree +-- + +-- explain (costs off) +-- select * from tenk1 t1 left join +-- (tenk1 t2 join tenk1 t3 on t2.thousand = t3.unique2) +-- on t1.hundred = t2.hundred and t1.ten = t3.ten +-- where t1.unique1 = 1; + +-- explain (costs off) +-- select * from tenk1 t1 left join +-- (tenk1 t2 join tenk1 t3 on t2.thousand = t3.unique2) +-- on t1.hundred = t2.hundred and t1.ten + t2.ten = t3.ten +-- where t1.unique1 = 1; + +-- explain (costs off) +-- select count(*) from +-- tenk1 a join tenk1 b on a.unique1 = b.unique2 +-- left join tenk1 c on a.unique2 = b.unique1 and c.thousand = a.thousand +-- join int4_tbl on b.thousand = f1; + +select count(*) from + tenk1 a join tenk1 b on a.unique1 = b.unique2 + left join tenk1 c on a.unique2 = b.unique1 and c.thousand = a.thousand + join int4_tbl on b.thousand = f1; + +-- explain (costs off) +-- select b.unique1 from +-- tenk1 a join tenk1 b on a.unique1 = b.unique2 +-- left join tenk1 c on b.unique1 = 42 and c.thousand = a.thousand +-- join int4_tbl i1 on b.thousand = f1 +-- right join int4_tbl i2 on i2.f1 = b.tenthous +-- order by 1; + +select b.unique1 from + tenk1 a join tenk1 b on a.unique1 = b.unique2 + left join tenk1 c on b.unique1 = 42 and c.thousand = a.thousand + join int4_tbl i1 on b.thousand = f1 + right join int4_tbl i2 on i2.f1 = b.tenthous + order by 1; + +-- explain (costs off) +-- select * from +-- ( +-- select unique1, q1, coalesce(unique1, -1) + q1 as fault +-- from int8_tbl left join tenk1 on (q2 = unique2) +-- ) ss +-- where fault = 122 +-- order by fault; + +select * from +( + select unique1, q1, coalesce(unique1, -1) + q1 as fault + from int8_tbl left join tenk1 on (q2 = unique2) +) ss +where fault = 122 +order by fault; + +-- explain (costs off) +-- select * from +-- (values (1, array[10,20]), (2, array[20,30])) as v1(v1x,v1ys) +-- left join (values (1, 10), (2, 20)) as v2(v2x,v2y) on v2x = v1x +-- left join unnest(v1ys) as u1(u1y) on u1y = v2y; + +-- [SPARK-28382] Array Functions: unnest +-- select * from +-- (values (1, array(10,20)), (2, array(20,30))) as v1(v1x,v1ys) +-- left join (values (1, 10), (2, 20)) as v2(v2x,v2y) on v2x = v1x +-- left join unnest(v1ys) as u1(u1y) on u1y = v2y; + +-- +-- test handling of potential equivalence clauses above outer joins +-- + +-- explain (costs off) +-- select q1, unique2, thousand, hundred +-- from int8_tbl a left join tenk1 b on q1 = unique2 +-- where coalesce(thousand,123) = q1 and q1 = coalesce(hundred,123); + +select q1, unique2, thousand, hundred + from int8_tbl a left join tenk1 b on q1 = unique2 + where coalesce(thousand,123) = q1 and q1 = coalesce(hundred,123); + +-- explain (costs off) +-- select f1, unique2, case when unique2 is null then f1 else 0 end +-- from int4_tbl a left join tenk1 b on f1 = unique2 +-- where (case when unique2 is null then f1 else 0 end) = 0; + +select f1, unique2, case when unique2 is null then f1 else 0 end + from int4_tbl a left join tenk1 b on f1 = unique2 + where (case when unique2 is null then f1 else 0 end) = 0; + +-- +-- another case with equivalence clauses above outer joins (bug #8591) +-- + +-- explain (costs off) +-- select a.unique1, b.unique1, c.unique1, coalesce(b.twothousand, a.twothousand) +-- from tenk1 a left join tenk1 b on b.thousand = a.unique1 left join tenk1 c on c.unique2 = coalesce(b.twothousand, a.twothousand) +-- where a.unique2 < 10 and coalesce(b.twothousand, a.twothousand) = 44; + +select a.unique1, b.unique1, c.unique1, coalesce(b.twothousand, a.twothousand) + from tenk1 a left join tenk1 b on b.thousand = a.unique1 left join tenk1 c on c.unique2 = coalesce(b.twothousand, a.twothousand) + where a.unique2 < 10 and coalesce(b.twothousand, a.twothousand) = 44; + +-- +-- check handling of join aliases when flattening multiple levels of subquery +-- + +-- explain (verbose, costs off) +-- select foo1.join_key as foo1_id, foo3.join_key AS foo3_id, bug_field from +-- (values (0),(1)) foo1(join_key) +-- left join +-- (select join_key, bug_field from +-- (select ss1.join_key, ss1.bug_field from +-- (select f1 as join_key, 666 as bug_field from int4_tbl i1) ss1 +-- ) foo2 +-- left join +-- (select unique2 as join_key from tenk1 i2) ss2 +-- using (join_key) +-- ) foo3 +-- using (join_key); + + +-- [SPARK-28377] Fully support correlation names in the FROM clause +-- select foo1.join_key as foo1_id, foo3.join_key AS foo3_id, bug_field from +-- (values (0),(1)) foo1(join_key) +-- left join +-- (select join_key, bug_field from +-- (select ss1.join_key, ss1.bug_field from +-- (select f1 as join_key, 666 as bug_field from int4_tbl i1) ss1 +-- ) foo2 +-- left join +-- (select unique2 as join_key from tenk1 i2) ss2 +-- using (join_key) +-- ) foo3 +-- using (join_key); + +-- [SPARK-20856] Support statement using nested joins +-- +-- test successful handling of nested outer joins with degenerate join quals +-- + +-- explain (verbose, costs off) +-- select t1.* from +-- text_tbl t1 +-- left join (select *, '***'::text as d1 from int8_tbl i8b1) b1 +-- left join int8_tbl i8 +-- left join (select *, null::int as d2 from int8_tbl i8b2) b2 +-- on (i8.q1 = b2.q1) +-- on (b2.d2 = b1.q2) +-- on (t1.f1 = b1.d1) +-- left join int4_tbl i4 +-- on (i8.q2 = i4.f1); + +-- select t1.* from +-- text_tbl t1 +-- left join (select *, string('***') as d1 from int8_tbl i8b1) b1 +-- left join int8_tbl i8 +-- left join (select *, int(null) as d2 from int8_tbl i8b2) b2 +-- on (i8.q1 = b2.q1) +-- on (b2.d2 = b1.q2) +-- on (t1.f1 = b1.d1) +-- left join int4_tbl i4 +-- on (i8.q2 = i4.f1); + +-- explain (verbose, costs off) +-- select t1.* from +-- text_tbl t1 +-- left join (select *, '***'::text as d1 from int8_tbl i8b1) b1 +-- left join int8_tbl i8 +-- left join (select *, null::int as d2 from int8_tbl i8b2, int4_tbl i4b2) b2 +-- on (i8.q1 = b2.q1) +-- on (b2.d2 = b1.q2) +-- on (t1.f1 = b1.d1) +-- left join int4_tbl i4 +-- on (i8.q2 = i4.f1); + +-- select t1.* from +-- text_tbl t1 +-- left join (select *, string('***') as d1 from int8_tbl i8b1) b1 +-- left join int8_tbl i8 +-- left join (select *, int(null) as d2 from int8_tbl i8b2, int4_tbl i4b2) b2 +-- on (i8.q1 = b2.q1) +-- on (b2.d2 = b1.q2) +-- on (t1.f1 = b1.d1) +-- left join int4_tbl i4 +-- on (i8.q2 = i4.f1); + +-- explain (verbose, costs off) +-- select t1.* from +-- text_tbl t1 +-- left join (select *, '***'::text as d1 from int8_tbl i8b1) b1 +-- left join int8_tbl i8 +-- left join (select *, null::int as d2 from int8_tbl i8b2, int4_tbl i4b2 +-- where q1 = f1) b2 +-- on (i8.q1 = b2.q1) +-- on (b2.d2 = b1.q2) +-- on (t1.f1 = b1.d1) +-- left join int4_tbl i4 +-- on (i8.q2 = i4.f1); + +-- select t1.* from +-- text_tbl t1 +-- left join (select *, string('***') as d1 from int8_tbl i8b1) b1 +-- left join int8_tbl i8 +-- left join (select *, int(null) as d2 from int8_tbl i8b2, int4_tbl i4b2 +-- where q1 = f1) b2 +-- on (i8.q1 = b2.q1) +-- on (b2.d2 = b1.q2) +-- on (t1.f1 = b1.d1) +-- left join int4_tbl i4 +-- on (i8.q2 = i4.f1); + +-- explain (verbose, costs off) +-- select * from +-- text_tbl t1 +-- inner join int8_tbl i8 +-- on i8.q2 = 456 +-- right join text_tbl t2 +-- on t1.f1 = 'doh!' +-- left join int4_tbl i4 +-- on i8.q1 = i4.f1; + +select * from + text_tbl t1 + inner join int8_tbl i8 + on i8.q2 = 456 + right join text_tbl t2 + on t1.f1 = 'doh!' + left join int4_tbl i4 + on i8.q1 = i4.f1; + +-- [SPARK-27877] ANSI SQL: LATERAL derived table(T491) +-- +-- test for appropriate join order in the presence of lateral references +-- + +-- explain (verbose, costs off) +select * from + text_tbl t1 + left join int8_tbl i8 + on i8.q2 = 123, + lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss +where t1.f1 = ss.f1; + +select * from + text_tbl t1 + left join int8_tbl i8 + on i8.q2 = 123, + lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss +where t1.f1 = ss.f1; + +-- explain (verbose, costs off) +select * from + text_tbl t1 + left join int8_tbl i8 + on i8.q2 = 123, + lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss1, + lateral (select ss1.* from text_tbl t3 limit 1) as ss2 +where t1.f1 = ss2.f1; + +select * from + text_tbl t1 + left join int8_tbl i8 + on i8.q2 = 123, + lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss1, + lateral (select ss1.* from text_tbl t3 limit 1) as ss2 +where t1.f1 = ss2.f1; + +-- explain (verbose, costs off) +select 1 from + text_tbl as tt1 + inner join text_tbl as tt2 on (tt1.f1 = 'foo') + left join text_tbl as tt3 on (tt3.f1 = 'foo') + left join text_tbl as tt4 on (tt3.f1 = tt4.f1), + lateral (select tt4.f1 as c0 from text_tbl as tt5 limit 1) as ss1 +where tt1.f1 = ss1.c0; + +select 1 from + text_tbl as tt1 + inner join text_tbl as tt2 on (tt1.f1 = 'foo') + left join text_tbl as tt3 on (tt3.f1 = 'foo') + left join text_tbl as tt4 on (tt3.f1 = tt4.f1), + lateral (select tt4.f1 as c0 from text_tbl as tt5 limit 1) as ss1 +where tt1.f1 = ss1.c0; + +-- +-- check a case in which a PlaceHolderVar forces join order +-- + +-- explain (verbose, costs off) +-- select ss2.* from +-- int4_tbl i41 +-- left join int8_tbl i8 +-- join (select i42.f1 as c1, i43.f1 as c2, 42 as c3 +-- from int4_tbl i42, int4_tbl i43) ss1 +-- on i8.q1 = ss1.c2 +-- on i41.f1 = ss1.c1, +-- lateral (select i41.*, i8.*, ss1.* from text_tbl limit 1) ss2 +-- where ss1.c2 = 0; + +-- select ss2.* from +-- int4_tbl i41 +-- left join int8_tbl i8 +-- join (select i42.f1 as c1, i43.f1 as c2, 42 as c3 +-- from int4_tbl i42, int4_tbl i43) ss1 +-- on i8.q1 = ss1.c2 +-- on i41.f1 = ss1.c1, +-- lateral (select i41.*, i8.*, ss1.* from text_tbl limit 1) ss2 +-- where ss1.c2 = 0; + +-- +-- test successful handling of full join underneath left join (bug #14105) +-- + +-- explain (costs off) +-- select * from +-- (select 1 as id) as xx +-- left join +-- (tenk1 as a1 full join (select 1 as id) as yy on (a1.unique1 = yy.id)) +-- on (xx.id = coalesce(yy.id)); + +select * from + (select 1 as id) as xx + left join + (tenk1 as a1 full join (select 1 as id) as yy on (a1.unique1 = yy.id)) + on (xx.id = coalesce(yy.id)); + +-- +-- test ability to push constants through outer join clauses +-- + +-- explain (costs off) +-- select * from int4_tbl a left join tenk1 b on f1 = unique2 where f1 = 0; + +-- explain (costs off) +-- select * from tenk1 a full join tenk1 b using(unique2) where unique2 = 42; + +-- +-- test that quals attached to an outer join have correct semantics, +-- specifically that they don't re-use expressions computed below the join; +-- we force a mergejoin so that coalesce(b.q1, 1) appears as a join input +-- + +-- set enable_hashjoin to off; +-- set enable_nestloop to off; + +-- explain (verbose, costs off) +-- select a.q2, b.q1 +-- from int8_tbl a left join int8_tbl b on a.q2 = coalesce(b.q1, 1) +-- where coalesce(b.q1, 1) > 0; +select a.q2, b.q1 + from int8_tbl a left join int8_tbl b on a.q2 = coalesce(b.q1, 1) + where coalesce(b.q1, 1) > 0; + +-- reset enable_hashjoin; +-- reset enable_nestloop; + +-- Skip these test because it only test explain +-- +-- test join removal +-- + +-- begin; + +-- CREATE TEMP TABLE a (id int PRIMARY KEY, b_id int); +-- CREATE TEMP TABLE b (id int PRIMARY KEY, c_id int); +-- CREATE TEMP TABLE c (id int PRIMARY KEY); +-- CREATE TEMP TABLE d (a int, b int); +-- INSERT INTO a VALUES (0, 0), (1, NULL); +-- INSERT INTO b VALUES (0, 0), (1, NULL); +-- INSERT INTO c VALUES (0), (1); +-- INSERT INTO d VALUES (1,3), (2,2), (3,1); + +-- all three cases should be optimizable into a simple seqscan +-- explain (costs off) SELECT a.* FROM a LEFT JOIN b ON a.b_id = b.id; +-- explain (costs off) SELECT b.* FROM b LEFT JOIN c ON b.c_id = c.id; +-- explain (costs off) +-- SELECT a.* FROM a LEFT JOIN (b left join c on b.c_id = c.id) +-- ON (a.b_id = b.id); + +-- check optimization of outer join within another special join +-- explain (costs off) +-- select id from a where id in ( +-- select b.id from b left join c on b.id = c.id +-- ); + +-- check that join removal works for a left join when joining a subquery +-- that is guaranteed to be unique by its GROUP BY clause +-- explain (costs off) +-- select d.* from d left join (select * from b group by b.id, b.c_id) s +-- on d.a = s.id and d.b = s.c_id; + +-- similarly, but keying off a DISTINCT clause +-- explain (costs off) +-- select d.* from d left join (select distinct * from b) s +-- on d.a = s.id and d.b = s.c_id; + +-- join removal is not possible when the GROUP BY contains a column that is +-- not in the join condition. (Note: as of 9.6, we notice that b.id is a +-- primary key and so drop b.c_id from the GROUP BY of the resulting plan; +-- but this happens too late for join removal in the outer plan level.) +-- explain (costs off) +-- select d.* from d left join (select * from b group by b.id, b.c_id) s +-- on d.a = s.id; + +-- similarly, but keying off a DISTINCT clause +-- explain (costs off) +-- select d.* from d left join (select distinct * from b) s +-- on d.a = s.id; + +-- check join removal works when uniqueness of the join condition is enforced +-- by a UNION +-- explain (costs off) +-- select d.* from d left join (select id from a union select id from b) s +-- on d.a = s.id; + +-- check join removal with a cross-type comparison operator +-- explain (costs off) +-- select i8.* from int8_tbl i8 left join (select f1 from int4_tbl group by f1) i4 +-- on i8.q1 = i4.f1; + +-- check join removal with lateral references +-- explain (costs off) +-- select 1 from (select a.id FROM a left join b on a.b_id = b.id) q, +-- lateral generate_series(1, q.id) gs(i) where q.id = gs.i; + +-- rollback; + +create or replace temporary view parent as select * from + (values (1, 10), (2, 20), (3, 30)) + as v(k, pd); +create or replace temporary view child as select * from + (values (1, 100), (4, 400)) + as v(k, cd); + +-- this case is optimizable +select p.* from parent p left join child c on (p.k = c.k); +-- explain (costs off) +-- select p.* from parent p left join child c on (p.k = c.k); + +-- this case is not +select p.*, linked from parent p + left join (select c.*, true as linked from child c) as ss + on (p.k = ss.k); +-- explain (costs off) +-- select p.*, linked from parent p +-- left join (select c.*, true as linked from child c) as ss +-- on (p.k = ss.k); + +-- check for a 9.0rc1 bug: join removal breaks pseudoconstant qual handling +select p.* from + parent p left join child c on (p.k = c.k) + where p.k = 1 and p.k = 2; +-- explain (costs off) +-- select p.* from +-- parent p left join child c on (p.k = c.k) +-- where p.k = 1 and p.k = 2; + +select p.* from + (parent p left join child c on (p.k = c.k)) join parent x on p.k = x.k + where p.k = 1 and p.k = 2; +-- explain (costs off) +-- select p.* from +-- (parent p left join child c on (p.k = c.k)) join parent x on p.k = x.k +-- where p.k = 1 and p.k = 2; + +-- bug 5255: this is not optimizable by join removal +-- begin; + +create or replace temporary view a as select * from + (values (0), (1)) + as v(id); +create or replace temporary view b as select * from + (values (0, 0), (1, NULL)) + as v(id, a_id); + +SELECT * FROM b LEFT JOIN a ON (b.a_id = a.id) WHERE (a.id IS NULL OR a.id > 0); +SELECT b.* FROM b LEFT JOIN a ON (b.a_id = a.id) WHERE (a.id IS NULL OR a.id > 0); + +-- rollback; + +-- another join removal bug: this is not optimizable, either +-- begin; + +create or replace temporary view innertab as select * from + (values (123L, 42L)) + as v(id, dat1); + +SELECT * FROM + (SELECT 1 AS x) ss1 + LEFT JOIN + (SELECT q1, q2, COALESCE(dat1, q1) AS y + FROM int8_tbl LEFT JOIN innertab ON q2 = id) ss2 + ON true; + +-- rollback; + +-- another join removal bug: we must clean up correctly when removing a PHV +-- begin; + +-- create temp table uniquetbl (f1 text unique); + +-- explain (costs off) +-- select t1.* from +-- uniquetbl as t1 +-- left join (select *, '***'::text as d1 from uniquetbl) t2 +-- on t1.f1 = t2.f1 +-- left join uniquetbl t3 +-- on t2.d1 = t3.f1; + +-- explain (costs off) +-- select t0.* +-- from +-- text_tbl t0 +-- left join +-- (select case t1.ten when 0 then 'doh!'::text else null::text end as case1, +-- t1.stringu2 +-- from tenk1 t1 +-- join int4_tbl i4 ON i4.f1 = t1.unique2 +-- left join uniquetbl u1 ON u1.f1 = t1.string4) ss +-- on t0.f1 = ss.case1 +-- where ss.stringu2 !~* ss.case1; + +-- [SPARK-27987] Support POSIX Regular Expressions +-- select t0.* +-- from +-- text_tbl t0 +-- left join +-- (select case t1.ten when 0 then 'doh!'::text else null::text end as case1, +-- t1.stringu2 +-- from tenk1 t1 +-- join int4_tbl i4 ON i4.f1 = t1.unique2 +-- left join uniquetbl u1 ON u1.f1 = t1.string4) ss +-- on t0.f1 = ss.case1 +-- where ss.stringu2 !~* ss.case1; + +-- rollback; + +-- bug #8444: we've historically allowed duplicate aliases within aliased JOINs + +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = f1; -- error +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = y.f1; -- error +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on q1 = f1; -- ok + +-- +-- Test hints given on incorrect column references are useful +-- + +select t1.uunique1 from + tenk1 t1 join tenk2 t2 on t1.two = t2.two; -- error, prefer "t1" suggestion +select t2.uunique1 from + tenk1 t1 join tenk2 t2 on t1.two = t2.two; -- error, prefer "t2" suggestion +select uunique1 from + tenk1 t1 join tenk2 t2 on t1.two = t2.two; -- error, suggest both at once + +-- Skip this test because it is a PostgreSQL specific case +-- +-- Take care to reference the correct RTE +-- + +-- -- select atts.relid::regclass, s.* from pg_stats s join +-- -- pg_attribute a on s.attname = a.attname and s.tablename = +-- -- a.attrelid::regclass::text join (select unnest(indkey) attnum, +-- -- indexrelid from pg_index i) atts on atts.attnum = a.attnum where +-- schemaname != 'pg_catalog'; + +-- [SPARK-27877] ANSI SQL: LATERAL derived table(T491) +-- +-- Test LATERAL +-- +select unique2, x.* +from tenk1 a, lateral (select * from int4_tbl b where f1 = a.unique1) x; +-- explain (costs off) +-- select unique2, x.* +-- from tenk1 a, lateral (select * from int4_tbl b where f1 = a.unique1) x; +select unique2, x.* +from int4_tbl x, lateral (select unique2 from tenk1 where f1 = unique1) ss; +-- explain (costs off) +-- select unique2, x.* +-- from int4_tbl x, lateral (select unique2 from tenk1 where f1 = unique1) ss; +-- explain (costs off) +-- select unique2, x.* +-- from int4_tbl x cross join lateral (select unique2 from tenk1 where f1 = unique1) ss; +select unique2, x.* +from int4_tbl x left join lateral (select unique1, unique2 from tenk1 where f1 = unique1) ss on true; +-- explain (costs off) +-- select unique2, x.* +-- from int4_tbl x left join lateral (select unique1, unique2 from tenk1 where f1 = unique1) ss on true; + +-- [SPARK-27877] ANSI SQL: LATERAL derived table(T491) +-- check scoping of lateral versus parent references +-- the first of these should return int8_tbl.q2, the second int8_tbl.q1 +-- Expressions referencing the outer query are not supported outside of WHERE/HAVING clauses +-- select *, (select r from (select q1 as q2) x, (select q2 as r) y) from int8_tbl; +-- select *, (select r from (select q1 as q2) x, lateral (select q2 as r) y) from int8_tbl; + +-- lateral with function in FROM +-- select count(*) from tenk1 a, lateral generate_series(1,two) g; +-- explain (costs off) +-- select count(*) from tenk1 a, lateral generate_series(1,two) g; +-- explain (costs off) +-- select count(*) from tenk1 a cross join lateral generate_series(1,two) g; +-- don't need the explicit LATERAL keyword for functions +-- explain (costs off) +-- select count(*) from tenk1 a, generate_series(1,two) g; + +-- lateral with UNION ALL subselect +-- explain (costs off) +-- select * from generate_series(100,200) g, +-- lateral (select * from int8_tbl a where g = q1 union all +-- select * from int8_tbl b where g = q2) ss; +-- select * from generate_series(100,200) g, +-- lateral (select * from int8_tbl a where g = q1 union all +-- select * from int8_tbl b where g = q2) ss; + +-- lateral with VALUES +-- explain (costs off) +-- select count(*) from tenk1 a, +-- tenk1 b join lateral (values(a.unique1)) ss(x) on b.unique2 = ss.x; +-- select count(*) from tenk1 a, +-- tenk1 b join lateral (values(a.unique1)) ss(x) on b.unique2 = ss.x; + +-- lateral with VALUES, no flattening possible +-- explain (costs off) +-- select count(*) from tenk1 a, +-- tenk1 b join lateral (values(a.unique1),(-1)) ss(x) on b.unique2 = ss.x; +-- select count(*) from tenk1 a, +-- tenk1 b join lateral (values(a.unique1),(-1)) ss(x) on b.unique2 = ss.x; + +-- lateral injecting a strange outer join condition +-- explain (costs off) +-- select * from int8_tbl a, +-- int8_tbl x left join lateral (select a.q1 from int4_tbl y) ss(z) +-- on x.q2 = ss.z +-- order by a.q1, a.q2, x.q1, x.q2, ss.z; +-- select * from int8_tbl a, +-- int8_tbl x left join lateral (select a.q1 from int4_tbl y) ss(z) +-- on x.q2 = ss.z +-- order by a.q1, a.q2, x.q1, x.q2, ss.z; + +-- lateral reference to a join alias variable + select * from (select f1/2 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1, + lateral (select x) ss2(y); +-- select * from (select f1 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1, +-- lateral (values(x)) ss2(y); + select * from ((select f1/2 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1) j, + lateral (select x) ss2(y); + +-- lateral references requiring pullup +-- select * from (values(1)) x(lb), +-- lateral generate_series(lb,4) x4; +-- select * from (select f1/1000000000 from int4_tbl) x(lb), +-- lateral generate_series(lb,4) x4; +-- select * from (values(1)) x(lb), +-- lateral (values(lb)) y(lbcopy); +-- select * from (values(1)) x(lb), +-- lateral (select lb from int4_tbl) y(lbcopy); +-- select * from +-- int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1, +-- lateral (values(x.q1,y.q1,y.q2)) v(xq1,yq1,yq2); +select * from + int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1, + lateral (select x.q1,y.q1,y.q2) v(xq1,yq1,yq2); +select x.* from + int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1, + lateral (select x.q1,y.q1,y.q2) v(xq1,yq1,yq2); +-- Accessing outer query column is not allowed in Union +-- select v.* from +-- (int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1) +-- left join int4_tbl z on z.f1 = x.q2, +-- lateral (select x.q1,y.q1 union all select x.q2,y.q2) v(vx,vy); +-- Accessing outer query column is not allowed in Union +-- select v.* from +-- (int8_tbl x left join (select q1,(select coalesce(q2,0)) q2 from int8_tbl) y on x.q2 = y.q1) +-- left join int4_tbl z on z.f1 = x.q2, +-- lateral (select x.q1,y.q1 union all select x.q2,y.q2) v(vx,vy); +-- Expressions referencing the outer query are not supported outside of WHERE/HAVING clauses +-- select v.* from +-- (int8_tbl x left join (select q1,(select coalesce(q2,0)) q2 from int8_tbl) y on x.q2 = y.q1) +-- left join int4_tbl z on z.f1 = x.q2, +-- lateral (select x.q1,y.q1 from onerow union all select x.q2,y.q2 from onerow) v(vx,vy); + +-- explain (verbose, costs off) +-- select * from +-- int8_tbl a left join +-- lateral (select *, a.q2 as x from int8_tbl b) ss on a.q2 = ss.q1; +select * from + int8_tbl a left join + lateral (select *, a.q2 as x from int8_tbl b) ss on a.q2 = ss.q1; +-- explain (verbose, costs off) +-- select * from +-- int8_tbl a left join +-- lateral (select *, coalesce(a.q2, 42) as x from int8_tbl b) ss on a.q2 = ss.q1; +select * from + int8_tbl a left join + lateral (select *, coalesce(a.q2, 42) as x from int8_tbl b) ss on a.q2 = ss.q1; + +-- lateral can result in join conditions appearing below their +-- real semantic level +-- explain (verbose, costs off) +-- select * from int4_tbl i left join +-- lateral (select * from int2_tbl j where i.f1 = j.f1) k on true; +select * from int4_tbl i left join + lateral (select * from int2_tbl j where i.f1 = j.f1) k on true; +-- explain (verbose, costs off) +-- select * from int4_tbl i left join +-- lateral (select coalesce(i) from int2_tbl j where i.f1 = j.f1) k on true; +-- select * from int4_tbl i left join +-- lateral (select coalesce(i) from int2_tbl j where i.f1 = j.f1) k on true; +-- explain (verbose, costs off) +-- select * from int4_tbl a, +-- lateral ( +-- select * from int4_tbl b left join int8_tbl c on (b.f1 = q1 and a.f1 = q2) +-- ) ss; +-- select * from int4_tbl a, +-- lateral ( +-- select * from int4_tbl b left join int8_tbl c on (b.f1 = q1 and a.f1 = q2) +-- ) ss; + +-- lateral reference in a PlaceHolderVar evaluated at join level +-- explain (verbose, costs off) +-- select * from +-- int8_tbl a left join lateral +-- (select b.q1 as bq1, c.q1 as cq1, least(a.q1,b.q1,c.q1) from +-- int8_tbl b cross join int8_tbl c) ss +-- on a.q2 = ss.bq1; +select * from + int8_tbl a left join lateral + (select b.q1 as bq1, c.q1 as cq1, least(a.q1,b.q1,c.q1) from + int8_tbl b cross join int8_tbl c) ss + on a.q2 = ss.bq1; + +-- case requiring nested PlaceHolderVars +-- explain (verbose, costs off) +-- select * from +-- int8_tbl c left join ( +-- int8_tbl a left join (select q1, coalesce(q2,42) as x from int8_tbl b) ss1 +-- on a.q2 = ss1.q1 +-- cross join +-- lateral (select q1, coalesce(ss1.x,q2) as y from int8_tbl d) ss2 +-- ) on c.q2 = ss2.q1, +-- lateral (select ss2.y offset 0) ss3; + +-- case that breaks the old ph_may_need optimization +-- explain (verbose, costs off) +-- select c.*,a.*,ss1.q1,ss2.q1,ss3.* from +-- int8_tbl c left join ( +-- int8_tbl a left join +-- (select q1, coalesce(q2,f1) as x from int8_tbl b, int4_tbl b2 +-- where q1 < f1) ss1 +-- on a.q2 = ss1.q1 +-- cross join +-- lateral (select q1, coalesce(ss1.x,q2) as y from int8_tbl d) ss2 +-- ) on c.q2 = ss2.q1, +-- lateral (select * from int4_tbl i where ss2.y > f1) ss3; + +-- check processing of postponed quals (bug #9041) +-- explain (verbose, costs off) +-- select * from +-- (select 1 as x offset 0) x cross join (select 2 as y offset 0) y +-- left join lateral ( +-- select * from (select 3 as z offset 0) z where z.z = x.x +-- ) zz on zz.z = y.y; + +-- check dummy rels with lateral references (bug #15694) +-- explain (verbose, costs off) +-- select * from int8_tbl i8 left join lateral +-- (select *, i8.q2 from int4_tbl where false) ss on true; +-- explain (verbose, costs off) +-- select * from int8_tbl i8 left join lateral +-- (select *, i8.q2 from int4_tbl i1, int4_tbl i2 where false) ss on true; + +-- check handling of nested appendrels inside LATERAL +-- select * from +-- ((select 2 as v) union all (select 3 as v)) as q1 +-- cross join lateral +-- ((select * from +-- ((select 4 as v) union all (select 5 as v)) as q3) +-- union all +-- (select q1.v) +-- ) as q2; + +-- check we don't try to do a unique-ified semijoin with LATERAL +-- explain (verbose, costs off) +-- select * from +-- (values (0,9998), (1,1000)) v(id,x), +-- lateral (select f1 from int4_tbl +-- where f1 = any (select unique1 from tenk1 +-- where unique2 = v.x offset 0)) ss; +-- select * from +-- (values (0,9998), (1,1000)) v(id,x), +-- lateral (select f1 from int4_tbl +-- where f1 = any (select unique1 from tenk1 +-- where unique2 = v.x offset 0)) ss; + +-- check proper extParam/allParam handling (this isn't exactly a LATERAL issue, +-- but we can make the test case much more compact with LATERAL) +-- explain (verbose, costs off) +-- select * from (values (0), (1)) v(id), +-- lateral (select * from int8_tbl t1, +-- lateral (select * from +-- (select * from int8_tbl t2 +-- where q1 = any (select q2 from int8_tbl t3 +-- where q2 = (select greatest(t1.q1,t2.q2)) +-- and (select v.id=0)) offset 0) ss2) ss +-- where t1.q1 = ss.q2) ss0; + +-- select * from (values (0), (1)) v(id), +-- lateral (select * from int8_tbl t1, +-- lateral (select * from +-- (select * from int8_tbl t2 +-- where q1 = any (select q2 from int8_tbl t3 +-- where q2 = (select greatest(t1.q1,t2.q2)) +-- and (select v.id=0)) offset 0) ss2) ss +-- where t1.q1 = ss.q2) ss0; + +-- test some error cases where LATERAL should have been used but wasn't +select f1,g from int4_tbl a, (select f1 as g) ss; +select f1,g from int4_tbl a, (select a.f1 as g) ss; +select f1,g from int4_tbl a cross join (select f1 as g) ss; +select f1,g from int4_tbl a cross join (select a.f1 as g) ss; +-- SQL:2008 says the left table is in scope but illegal to access here +-- select f1,g from int4_tbl a right join lateral generate_series(0, a.f1) g on true; +-- select f1,g from int4_tbl a full join lateral generate_series(0, a.f1) g on true; +-- check we complain about ambiguous table references +-- select * from +-- int8_tbl x cross join (int4_tbl x cross join lateral (select x.f1) ss); +-- LATERAL can be used to put an aggregate into the FROM clause of its query +-- select 1 from tenk1 a, lateral (select max(a.unique1) from int4_tbl b) ss; + +-- check behavior of LATERAL in UPDATE/DELETE + +-- create temp table xx1 as select f1 as x1, -f1 as x2 from int4_tbl; + +-- error, can't do this: +-- update xx1 set x2 = f1 from (select * from int4_tbl where f1 = x1) ss; +-- update xx1 set x2 = f1 from (select * from int4_tbl where f1 = xx1.x1) ss; +-- can't do it even with LATERAL: +-- update xx1 set x2 = f1 from lateral (select * from int4_tbl where f1 = x1) ss; +-- we might in future allow something like this, but for now it's an error: +-- update xx1 set x2 = f1 from xx1, lateral (select * from int4_tbl where f1 = x1) ss; + +-- also errors: +-- delete from xx1 using (select * from int4_tbl where f1 = x1) ss; +-- delete from xx1 using (select * from int4_tbl where f1 = xx1.x1) ss; +-- delete from xx1 using lateral (select * from int4_tbl where f1 = x1) ss; + +-- [SPARK-25411] Implement range partition in Spark +-- +-- test LATERAL reference propagation down a multi-level inheritance hierarchy +-- produced for a multi-level partitioned table hierarchy. +-- +-- create table join_pt1 (a int, b int, c varchar) partition by range(a); +-- create table join_pt1p1 partition of join_pt1 for values from (0) to (100) partition by range(b); +-- create table join_pt1p2 partition of join_pt1 for values from (100) to (200); +-- create table join_pt1p1p1 partition of join_pt1p1 for values from (0) to (100); +-- insert into join_pt1 values (1, 1, 'x'), (101, 101, 'y'); +-- create table join_ut1 (a int, b int, c varchar); +-- insert into join_ut1 values (101, 101, 'y'), (2, 2, 'z'); +-- explain (verbose, costs off) +-- select t1.b, ss.phv from join_ut1 t1 left join lateral +-- (select t2.a as t2a, t3.a t3a, least(t1.a, t2.a, t3.a) phv +-- from join_pt1 t2 join join_ut1 t3 on t2.a = t3.b) ss +-- on t1.a = ss.t2a order by t1.a; +-- select t1.b, ss.phv from join_ut1 t1 left join lateral +-- (select t2.a as t2a, t3.a t3a, least(t1.a, t2.a, t3.a) phv +-- from join_pt1 t2 join join_ut1 t3 on t2.a = t3.b) ss +-- on t1.a = ss.t2a order by t1.a; +-- +-- drop table join_pt1; +-- drop table join_ut1; +-- +-- test that foreign key join estimation performs sanely for outer joins +-- + +-- begin; + +-- create table fkest (a int, b int, c int unique, primary key(a,b)); +-- create table fkest1 (a int, b int, primary key(a,b)); + +-- insert into fkest select x/10, x%10, x from generate_series(1,1000) x; +-- insert into fkest1 select x/10, x%10 from generate_series(1,1000) x; + +-- alter table fkest1 +-- add constraint fkest1_a_b_fkey foreign key (a,b) references fkest; + +-- analyze fkest; +-- analyze fkest1; + +-- explain (costs off) +-- select * +-- from fkest f +-- left join fkest1 f1 on f.a = f1.a and f.b = f1.b +-- left join fkest1 f2 on f.a = f2.a and f.b = f2.b +-- left join fkest1 f3 on f.a = f3.a and f.b = f3.b +-- where f.c = 1; + +-- rollback; + +-- Skip these test because it only test explain +-- +-- test planner's ability to mark joins as unique +-- + +-- create table j1 (id int primary key); +-- create table j2 (id int primary key); +-- create table j3 (id int); + +-- insert into j1 values(1),(2),(3); +-- insert into j2 values(1),(2),(3); +-- insert into j3 values(1),(1); + +-- analyze j1; +-- analyze j2; +-- analyze j3; + +-- ensure join is properly marked as unique +-- explain (verbose, costs off) +-- select * from j1 inner join j2 on j1.id = j2.id; + +-- ensure join is not unique when not an equi-join +-- explain (verbose, costs off) +-- select * from j1 inner join j2 on j1.id > j2.id; + +-- ensure non-unique rel is not chosen as inner +-- explain (verbose, costs off) +-- select * from j1 inner join j3 on j1.id = j3.id; + +-- ensure left join is marked as unique +-- explain (verbose, costs off) +-- select * from j1 left join j2 on j1.id = j2.id; + +-- ensure right join is marked as unique +-- explain (verbose, costs off) +-- select * from j1 right join j2 on j1.id = j2.id; + +-- ensure full join is marked as unique +-- explain (verbose, costs off) +-- select * from j1 full join j2 on j1.id = j2.id; + +-- a clauseless (cross) join can't be unique +-- explain (verbose, costs off) +-- select * from j1 cross join j2; + +-- ensure a natural join is marked as unique +-- explain (verbose, costs off) +-- select * from j1 natural join j2; + +-- ensure a distinct clause allows the inner to become unique +-- explain (verbose, costs off) +-- select * from j1 +-- inner join (select distinct id from j3) j3 on j1.id = j3.id; + +-- ensure group by clause allows the inner to become unique +-- explain (verbose, costs off) +-- select * from j1 +-- inner join (select id from j3 group by id) j3 on j1.id = j3.id; + +-- drop table if exists j1; +-- drop table if exists j2; +-- drop table if exists j3; + +-- test more complex permutations of unique joins + +CREATE TABLE j1 (id1 int, id2 int) USING parquet; +CREATE TABLE j2 (id1 int, id2 int) USING parquet; +-- create table j3 (id1 int, id2 int) using parquet; + +INSERT INTO j1 values(1,1),(1,2); +INSERT INTO j2 values(1,1); +-- insert into j3 values(1,1); + +-- analyze j1; +-- analyze j2; +-- analyze j3; + +-- ensure there's no unique join when not all columns which are part of the +-- unique index are seen in the join clause +-- explain (verbose, costs off) +-- select * from j1 +-- inner join j2 on j1.id1 = j2.id1; + +-- ensure proper unique detection with multiple join quals +-- explain (verbose, costs off) +-- select * from j1 +-- inner join j2 on j1.id1 = j2.id1 and j1.id2 = j2.id2; + +-- ensure we don't detect the join to be unique when quals are not part of the +-- join condition +-- explain (verbose, costs off) +-- select * from j1 +-- inner join j2 on j1.id1 = j2.id1 where j1.id2 = 1; + +-- as above, but for left joins. +-- explain (verbose, costs off) +-- select * from j1 +-- left join j2 on j1.id1 = j2.id1 where j1.id2 = 1; + +-- validate logic in merge joins which skips mark and restore. +-- it should only do this if all quals which were used to detect the unique +-- are present as join quals, and not plain quals. +-- set enable_nestloop to 0; +-- set enable_hashjoin to 0; +-- set enable_sort to 0; + +-- create indexes that will be preferred over the PKs to perform the join +-- create index j1_id1_idx on j1 (id1) where id1 % 1000 = 1; +-- create index j2_id1_idx on j2 (id1) where id1 % 1000 = 1; + +-- need an additional row in j2, if we want j2_id1_idx to be preferred +INSERT INTO j2 values(1,2); +-- analyze j2; + +-- explain (costs off) select * from j1 +-- inner join j2 on j1.id1 = j2.id1 and j1.id2 = j2.id2 +-- where j1.id1 % 1000 = 1 and j2.id1 % 1000 = 1; + +select * from j1 +inner join j2 on j1.id1 = j2.id1 and j1.id2 = j2.id2 +where j1.id1 % 1000 = 1 and j2.id1 % 1000 = 1; + +-- reset enable_nestloop; +-- reset enable_hashjoin; +-- reset enable_sort; + +drop table j1; +drop table j2; +-- drop table j3; + +-- Skip these tests because it only test explain +-- check that semijoin inner is not seen as unique for a portion of the outerrel +-- explain (verbose, costs off) +-- select t1.unique1, t2.hundred +-- from onek t1, tenk1 t2 +-- where exists (select 1 from tenk1 t3 +-- where t3.thousand = t1.unique1 and t3.tenthous = t2.hundred) +-- and t1.unique1 < 1; + +-- ... unless it actually is unique +-- create table j3 as select unique1, tenthous from onek; +-- vacuum analyze j3; +-- create unique index on j3(unique1, tenthous); + +-- explain (verbose, costs off) +-- select t1.unique1, t2.hundred +-- from onek t1, tenk1 t2 +-- where exists (select 1 from j3 +-- where j3.unique1 = t1.unique1 and j3.tenthous = t2.hundred) +-- and t1.unique1 < 1; + +-- drop table j3; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/limit.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/limit.sql new file mode 100644 index 000000000000..f59575817d6b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/limit.sql @@ -0,0 +1,162 @@ +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- LIMIT +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/limit.sql + +SELECT '' AS two, unique1, unique2, stringu1 + FROM onek WHERE unique1 > 50 + ORDER BY unique1 LIMIT 2; +SELECT '' AS five, unique1, unique2, stringu1 + FROM onek WHERE unique1 > 60 + ORDER BY unique1 LIMIT 5; +SELECT '' AS two, unique1, unique2, stringu1 + FROM onek WHERE unique1 > 60 AND unique1 < 63 + ORDER BY unique1 LIMIT 5; +SELECT '' AS three, unique1, unique2, stringu1 + FROM onek WHERE unique1 > 100 + ORDER BY unique1 LIMIT 3 OFFSET 20; +SELECT '' AS zero, unique1, unique2, stringu1 + FROM onek WHERE unique1 < 50 + ORDER BY unique1 DESC LIMIT 8 OFFSET 99; +SELECT '' AS eleven, unique1, unique2, stringu1 + FROM onek WHERE unique1 < 50 + ORDER BY unique1 DESC LIMIT 20 OFFSET 39; +SELECT '' AS ten, unique1, unique2, stringu1 + FROM onek + ORDER BY unique1 OFFSET 990; +-- SELECT '' AS five, unique1, unique2, stringu1 +-- FROM onek +-- ORDER BY unique1 OFFSET 990 LIMIT 5; +SELECT '' AS five, unique1, unique2, stringu1 + FROM onek + ORDER BY unique1 LIMIT 5 OFFSET 900; + +CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM + (VALUES + (123, 456), + (123, 4567890123456789), + (4567890123456789, 123), + (4567890123456789, 4567890123456789), + (4567890123456789, -4567890123456789)) + AS v(q1, q2); + +-- Test null limit and offset. The planner would discard a simple null +-- constant, so to ensure executor is exercised, do this: +-- [SPARK-29650] Discard a NULL constant in LIMIT +select * from int8_tbl limit (case when random() < 0.5 then bigint(null) end); +select * from int8_tbl offset (case when random() < 0.5 then bigint(null) end); + +-- Test assorted cases involving backwards fetch from a LIMIT plan node +-- [SPARK-20965] Support PREPARE/EXECUTE/DECLARE/FETCH statements +-- begin; +-- +-- declare c1 cursor for select * from int8_tbl limit 10; +-- fetch all in c1; +-- fetch 1 in c1; +-- fetch backward 1 in c1; +-- fetch backward all in c1; +-- fetch backward 1 in c1; +-- fetch all in c1; +-- +-- declare c2 cursor for select * from int8_tbl limit 3; +-- fetch all in c2; +-- fetch 1 in c2; +-- fetch backward 1 in c2; +-- fetch backward all in c2; +-- fetch backward 1 in c2; +-- fetch all in c2; +-- +-- declare c3 cursor for select * from int8_tbl offset 3; +-- fetch all in c3; +-- fetch 1 in c3; +-- fetch backward 1 in c3; +-- fetch backward all in c3; +-- fetch backward 1 in c3; +-- fetch all in c3; +-- +-- declare c4 cursor for select * from int8_tbl offset 10; +-- fetch all in c4; +-- fetch 1 in c4; +-- fetch backward 1 in c4; +-- fetch backward all in c4; +-- fetch backward 1 in c4; +-- fetch all in c4; +-- +-- rollback; + +DROP VIEW INT8_TBL; + +-- Stress test for variable LIMIT in conjunction with bounded-heap sorting + +-- [SPARK-27767] Built-in function: generate_series +-- SELECT +-- (SELECT n +-- FROM (VALUES (1)) AS x, +-- (SELECT n FROM generate_series(1,10) AS n +-- ORDER BY n LIMIT 1 OFFSET s-1) AS y) AS z +-- FROM generate_series(1,10) AS s; + +-- +-- Test behavior of volatile and set-returning functions in conjunction +-- with ORDER BY and LIMIT. +-- + +-- [SPARK-29631] Support ANSI SQL CREATE SEQUENCE +-- create temp sequence testseq; + +-- explain (verbose, costs off) +-- select unique1, unique2, nextval('testseq') +-- from tenk1 order by unique2 limit 10; + +-- select unique1, unique2, nextval('testseq') +-- from tenk1 order by unique2 limit 10; + +-- select currval('testseq'); + +-- explain (verbose, costs off) +-- select unique1, unique2, nextval('testseq') +-- from tenk1 order by tenthous limit 10; + +-- select unique1, unique2, nextval('testseq') +-- from tenk1 order by tenthous limit 10; + +-- select currval('testseq'); + +-- explain (verbose, costs off) +-- select unique1, unique2, generate_series(1,10) +-- from tenk1 order by unique2 limit 7; + +-- [SPARK-27767] Built-in function: generate_series +-- select unique1, unique2, generate_series(1,10) +-- from tenk1 order by unique2 limit 7; + +-- explain (verbose, costs off) +-- select unique1, unique2, generate_series(1,10) +-- from tenk1 order by tenthous limit 7; + +-- [SPARK-27767] Built-in function: generate_series +-- select unique1, unique2, generate_series(1,10) +-- from tenk1 order by tenthous limit 7; + +-- use of random() is to keep planner from folding the expressions together +-- explain (verbose, costs off) +-- select generate_series(0,2) as s1, generate_series((random()*.1)::int,2) as s2; + +-- [SPARK-27767] Built-in function: generate_series +-- select generate_series(0,2) as s1, generate_series((random()*.1)::int,2) as s2; + +-- explain (verbose, costs off) +-- select generate_series(0,2) as s1, generate_series((random()*.1)::int,2) as s2 +-- order by s2 desc; + +-- [SPARK-27767] Built-in function: generate_series +-- select generate_series(0,2) as s1, generate_series((random()*.1)::int,2) as s2 +-- order by s2 desc; + +-- test for failure to set all aggregates' aggtranstype +-- explain (verbose, costs off) +-- select sum(tenthous) as s1, sum(tenthous) + random()*0 as s2 +-- from tenk1 group by thousand order by thousand limit 3; + +select sum(tenthous) as s1, sum(tenthous) + random()*0 as s2 + from tenk1 group by thousand order by thousand limit 3; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/numeric.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/numeric.sql new file mode 100644 index 000000000000..cbc9c2a26f6c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/numeric.sql @@ -0,0 +1,1165 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- NUMERIC +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/numeric.sql +-- + +-- [SPARK-28318] Decimal can only support precision up to 38. We rewrite numeric(210,10) to decimal(38,10). +CREATE TABLE num_data (id int, val decimal(38,10)) USING parquet; +CREATE TABLE num_exp_add (id1 int, id2 int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_sub (id1 int, id2 int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_div (id1 int, id2 int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_mul (id1 int, id2 int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_sqrt (id int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_ln (id int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_log10 (id int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_power_10_ln (id int, expected decimal(38,10)) USING parquet; + +CREATE TABLE num_result (id1 int, id2 int, result decimal(38,10)) USING parquet; + + +-- ****************************** +-- * The following EXPECTED results are computed by bc(1) +-- * with a scale of 200 +-- ****************************** + +-- BEGIN TRANSACTION; +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO num_exp_add VALUES (0,0,0); +INSERT INTO num_exp_sub VALUES (0,0,0); +INSERT INTO num_exp_mul VALUES (0,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (0,0,double('NaN')); +INSERT INTO num_exp_add VALUES (0,1,0); +INSERT INTO num_exp_sub VALUES (0,1,0); +INSERT INTO num_exp_mul VALUES (0,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (0,1,double('NaN')); +INSERT INTO num_exp_add VALUES (0,2,-34338492.215397047); +INSERT INTO num_exp_sub VALUES (0,2,34338492.215397047); +INSERT INTO num_exp_mul VALUES (0,2,0); +INSERT INTO num_exp_div VALUES (0,2,0); +INSERT INTO num_exp_add VALUES (0,3,4.31); +INSERT INTO num_exp_sub VALUES (0,3,-4.31); +INSERT INTO num_exp_mul VALUES (0,3,0); +INSERT INTO num_exp_div VALUES (0,3,0); +INSERT INTO num_exp_add VALUES (0,4,7799461.4119); +INSERT INTO num_exp_sub VALUES (0,4,-7799461.4119); +INSERT INTO num_exp_mul VALUES (0,4,0); +INSERT INTO num_exp_div VALUES (0,4,0); +INSERT INTO num_exp_add VALUES (0,5,16397.038491); +INSERT INTO num_exp_sub VALUES (0,5,-16397.038491); +INSERT INTO num_exp_mul VALUES (0,5,0); +INSERT INTO num_exp_div VALUES (0,5,0); +INSERT INTO num_exp_add VALUES (0,6,93901.57763026); +INSERT INTO num_exp_sub VALUES (0,6,-93901.57763026); +INSERT INTO num_exp_mul VALUES (0,6,0); +INSERT INTO num_exp_div VALUES (0,6,0); +INSERT INTO num_exp_add VALUES (0,7,-83028485); +INSERT INTO num_exp_sub VALUES (0,7,83028485); +INSERT INTO num_exp_mul VALUES (0,7,0); +INSERT INTO num_exp_div VALUES (0,7,0); +INSERT INTO num_exp_add VALUES (0,8,74881); +INSERT INTO num_exp_sub VALUES (0,8,-74881); +INSERT INTO num_exp_mul VALUES (0,8,0); +INSERT INTO num_exp_div VALUES (0,8,0); +INSERT INTO num_exp_add VALUES (0,9,-24926804.045047420); +INSERT INTO num_exp_sub VALUES (0,9,24926804.045047420); +INSERT INTO num_exp_mul VALUES (0,9,0); +INSERT INTO num_exp_div VALUES (0,9,0); +INSERT INTO num_exp_add VALUES (1,0,0); +INSERT INTO num_exp_sub VALUES (1,0,0); +INSERT INTO num_exp_mul VALUES (1,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (1,0,double('NaN')); +INSERT INTO num_exp_add VALUES (1,1,0); +INSERT INTO num_exp_sub VALUES (1,1,0); +INSERT INTO num_exp_mul VALUES (1,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (1,1,double('NaN')); +INSERT INTO num_exp_add VALUES (1,2,-34338492.215397047); +INSERT INTO num_exp_sub VALUES (1,2,34338492.215397047); +INSERT INTO num_exp_mul VALUES (1,2,0); +INSERT INTO num_exp_div VALUES (1,2,0); +INSERT INTO num_exp_add VALUES (1,3,4.31); +INSERT INTO num_exp_sub VALUES (1,3,-4.31); +INSERT INTO num_exp_mul VALUES (1,3,0); +INSERT INTO num_exp_div VALUES (1,3,0); +INSERT INTO num_exp_add VALUES (1,4,7799461.4119); +INSERT INTO num_exp_sub VALUES (1,4,-7799461.4119); +INSERT INTO num_exp_mul VALUES (1,4,0); +INSERT INTO num_exp_div VALUES (1,4,0); +INSERT INTO num_exp_add VALUES (1,5,16397.038491); +INSERT INTO num_exp_sub VALUES (1,5,-16397.038491); +INSERT INTO num_exp_mul VALUES (1,5,0); +INSERT INTO num_exp_div VALUES (1,5,0); +INSERT INTO num_exp_add VALUES (1,6,93901.57763026); +INSERT INTO num_exp_sub VALUES (1,6,-93901.57763026); +INSERT INTO num_exp_mul VALUES (1,6,0); +INSERT INTO num_exp_div VALUES (1,6,0); +INSERT INTO num_exp_add VALUES (1,7,-83028485); +INSERT INTO num_exp_sub VALUES (1,7,83028485); +INSERT INTO num_exp_mul VALUES (1,7,0); +INSERT INTO num_exp_div VALUES (1,7,0); +INSERT INTO num_exp_add VALUES (1,8,74881); +INSERT INTO num_exp_sub VALUES (1,8,-74881); +INSERT INTO num_exp_mul VALUES (1,8,0); +INSERT INTO num_exp_div VALUES (1,8,0); +INSERT INTO num_exp_add VALUES (1,9,-24926804.045047420); +INSERT INTO num_exp_sub VALUES (1,9,24926804.045047420); +INSERT INTO num_exp_mul VALUES (1,9,0); +INSERT INTO num_exp_div VALUES (1,9,0); +INSERT INTO num_exp_add VALUES (2,0,-34338492.215397047); +INSERT INTO num_exp_sub VALUES (2,0,-34338492.215397047); +INSERT INTO num_exp_mul VALUES (2,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (2,0,double('NaN')); +INSERT INTO num_exp_add VALUES (2,1,-34338492.215397047); +INSERT INTO num_exp_sub VALUES (2,1,-34338492.215397047); +INSERT INTO num_exp_mul VALUES (2,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (2,1,double('NaN')); +INSERT INTO num_exp_add VALUES (2,2,-68676984.430794094); +INSERT INTO num_exp_sub VALUES (2,2,0); +INSERT INTO num_exp_mul VALUES (2,2,1179132047626883.596862135856320209); +INSERT INTO num_exp_div VALUES (2,2,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (2,3,-34338487.905397047); +INSERT INTO num_exp_sub VALUES (2,3,-34338496.525397047); +INSERT INTO num_exp_mul VALUES (2,3,-147998901.44836127257); +INSERT INTO num_exp_div VALUES (2,3,-7967167.56737750510440835266); +INSERT INTO num_exp_add VALUES (2,4,-26539030.803497047); +INSERT INTO num_exp_sub VALUES (2,4,-42137953.627297047); +INSERT INTO num_exp_mul VALUES (2,4,-267821744976817.8111137106593); +INSERT INTO num_exp_div VALUES (2,4,-4.40267480046830116685); +INSERT INTO num_exp_add VALUES (2,5,-34322095.176906047); +INSERT INTO num_exp_sub VALUES (2,5,-34354889.253888047); +INSERT INTO num_exp_mul VALUES (2,5,-563049578578.769242506736077); +INSERT INTO num_exp_div VALUES (2,5,-2094.18866914563535496429); +INSERT INTO num_exp_add VALUES (2,6,-34244590.637766787); +INSERT INTO num_exp_sub VALUES (2,6,-34432393.793027307); +INSERT INTO num_exp_mul VALUES (2,6,-3224438592470.18449811926184222); +INSERT INTO num_exp_div VALUES (2,6,-365.68599891479766440940); +INSERT INTO num_exp_add VALUES (2,7,-117366977.215397047); +INSERT INTO num_exp_sub VALUES (2,7,48689992.784602953); +INSERT INTO num_exp_mul VALUES (2,7,2851072985828710.485883795); +INSERT INTO num_exp_div VALUES (2,7,.41357483778485235518); +INSERT INTO num_exp_add VALUES (2,8,-34263611.215397047); +INSERT INTO num_exp_sub VALUES (2,8,-34413373.215397047); +INSERT INTO num_exp_mul VALUES (2,8,-2571300635581.146276407); +INSERT INTO num_exp_div VALUES (2,8,-458.57416721727870888476); +INSERT INTO num_exp_add VALUES (2,9,-59265296.260444467); +INSERT INTO num_exp_sub VALUES (2,9,-9411688.170349627); +INSERT INTO num_exp_mul VALUES (2,9,855948866655588.453741509242968740); +INSERT INTO num_exp_div VALUES (2,9,1.37757299946438931811); +INSERT INTO num_exp_add VALUES (3,0,4.31); +INSERT INTO num_exp_sub VALUES (3,0,4.31); +INSERT INTO num_exp_mul VALUES (3,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (3,0,double('NaN')); +INSERT INTO num_exp_add VALUES (3,1,4.31); +INSERT INTO num_exp_sub VALUES (3,1,4.31); +INSERT INTO num_exp_mul VALUES (3,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (3,1,double('NaN')); +INSERT INTO num_exp_add VALUES (3,2,-34338487.905397047); +INSERT INTO num_exp_sub VALUES (3,2,34338496.525397047); +INSERT INTO num_exp_mul VALUES (3,2,-147998901.44836127257); +INSERT INTO num_exp_div VALUES (3,2,-.00000012551512084352); +INSERT INTO num_exp_add VALUES (3,3,8.62); +INSERT INTO num_exp_sub VALUES (3,3,0); +INSERT INTO num_exp_mul VALUES (3,3,18.5761); +INSERT INTO num_exp_div VALUES (3,3,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (3,4,7799465.7219); +INSERT INTO num_exp_sub VALUES (3,4,-7799457.1019); +INSERT INTO num_exp_mul VALUES (3,4,33615678.685289); +INSERT INTO num_exp_div VALUES (3,4,.00000055260225961552); +INSERT INTO num_exp_add VALUES (3,5,16401.348491); +INSERT INTO num_exp_sub VALUES (3,5,-16392.728491); +INSERT INTO num_exp_mul VALUES (3,5,70671.23589621); +INSERT INTO num_exp_div VALUES (3,5,.00026285234387695504); +INSERT INTO num_exp_add VALUES (3,6,93905.88763026); +INSERT INTO num_exp_sub VALUES (3,6,-93897.26763026); +INSERT INTO num_exp_mul VALUES (3,6,404715.7995864206); +INSERT INTO num_exp_div VALUES (3,6,.00004589912234457595); +INSERT INTO num_exp_add VALUES (3,7,-83028480.69); +INSERT INTO num_exp_sub VALUES (3,7,83028489.31); +INSERT INTO num_exp_mul VALUES (3,7,-357852770.35); +INSERT INTO num_exp_div VALUES (3,7,-.00000005190989574240); +INSERT INTO num_exp_add VALUES (3,8,74885.31); +INSERT INTO num_exp_sub VALUES (3,8,-74876.69); +INSERT INTO num_exp_mul VALUES (3,8,322737.11); +INSERT INTO num_exp_div VALUES (3,8,.00005755799201399553); +INSERT INTO num_exp_add VALUES (3,9,-24926799.735047420); +INSERT INTO num_exp_sub VALUES (3,9,24926808.355047420); +INSERT INTO num_exp_mul VALUES (3,9,-107434525.43415438020); +INSERT INTO num_exp_div VALUES (3,9,-.00000017290624149854); +INSERT INTO num_exp_add VALUES (4,0,7799461.4119); +INSERT INTO num_exp_sub VALUES (4,0,7799461.4119); +INSERT INTO num_exp_mul VALUES (4,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (4,0,double('NaN')); +INSERT INTO num_exp_add VALUES (4,1,7799461.4119); +INSERT INTO num_exp_sub VALUES (4,1,7799461.4119); +INSERT INTO num_exp_mul VALUES (4,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (4,1,double('NaN')); +INSERT INTO num_exp_add VALUES (4,2,-26539030.803497047); +INSERT INTO num_exp_sub VALUES (4,2,42137953.627297047); +INSERT INTO num_exp_mul VALUES (4,2,-267821744976817.8111137106593); +INSERT INTO num_exp_div VALUES (4,2,-.22713465002993920385); +INSERT INTO num_exp_add VALUES (4,3,7799465.7219); +INSERT INTO num_exp_sub VALUES (4,3,7799457.1019); +INSERT INTO num_exp_mul VALUES (4,3,33615678.685289); +INSERT INTO num_exp_div VALUES (4,3,1809619.81714617169373549883); +INSERT INTO num_exp_add VALUES (4,4,15598922.8238); +INSERT INTO num_exp_sub VALUES (4,4,0); +INSERT INTO num_exp_mul VALUES (4,4,60831598315717.14146161); +INSERT INTO num_exp_div VALUES (4,4,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (4,5,7815858.450391); +INSERT INTO num_exp_sub VALUES (4,5,7783064.373409); +INSERT INTO num_exp_mul VALUES (4,5,127888068979.9935054429); +INSERT INTO num_exp_div VALUES (4,5,475.66281046305802686061); +INSERT INTO num_exp_add VALUES (4,6,7893362.98953026); +INSERT INTO num_exp_sub VALUES (4,6,7705559.83426974); +INSERT INTO num_exp_mul VALUES (4,6,732381731243.745115764094); +INSERT INTO num_exp_div VALUES (4,6,83.05996138436129499606); +INSERT INTO num_exp_add VALUES (4,7,-75229023.5881); +INSERT INTO num_exp_sub VALUES (4,7,90827946.4119); +INSERT INTO num_exp_mul VALUES (4,7,-647577464846017.9715); +INSERT INTO num_exp_div VALUES (4,7,-.09393717604145131637); +INSERT INTO num_exp_add VALUES (4,8,7874342.4119); +INSERT INTO num_exp_sub VALUES (4,8,7724580.4119); +INSERT INTO num_exp_mul VALUES (4,8,584031469984.4839); +INSERT INTO num_exp_div VALUES (4,8,104.15808298366741897143); +INSERT INTO num_exp_add VALUES (4,9,-17127342.633147420); +INSERT INTO num_exp_sub VALUES (4,9,32726265.456947420); +INSERT INTO num_exp_mul VALUES (4,9,-194415646271340.1815956522980); +INSERT INTO num_exp_div VALUES (4,9,-.31289456112403769409); +INSERT INTO num_exp_add VALUES (5,0,16397.038491); +INSERT INTO num_exp_sub VALUES (5,0,16397.038491); +INSERT INTO num_exp_mul VALUES (5,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (5,0,double('NaN')); +INSERT INTO num_exp_add VALUES (5,1,16397.038491); +INSERT INTO num_exp_sub VALUES (5,1,16397.038491); +INSERT INTO num_exp_mul VALUES (5,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (5,1,double('NaN')); +INSERT INTO num_exp_add VALUES (5,2,-34322095.176906047); +INSERT INTO num_exp_sub VALUES (5,2,34354889.253888047); +INSERT INTO num_exp_mul VALUES (5,2,-563049578578.769242506736077); +INSERT INTO num_exp_div VALUES (5,2,-.00047751189505192446); +INSERT INTO num_exp_add VALUES (5,3,16401.348491); +INSERT INTO num_exp_sub VALUES (5,3,16392.728491); +INSERT INTO num_exp_mul VALUES (5,3,70671.23589621); +INSERT INTO num_exp_div VALUES (5,3,3804.41728329466357308584); +INSERT INTO num_exp_add VALUES (5,4,7815858.450391); +INSERT INTO num_exp_sub VALUES (5,4,-7783064.373409); +INSERT INTO num_exp_mul VALUES (5,4,127888068979.9935054429); +INSERT INTO num_exp_div VALUES (5,4,.00210232958726897192); +INSERT INTO num_exp_add VALUES (5,5,32794.076982); +INSERT INTO num_exp_sub VALUES (5,5,0); +INSERT INTO num_exp_mul VALUES (5,5,268862871.275335557081); +INSERT INTO num_exp_div VALUES (5,5,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (5,6,110298.61612126); +INSERT INTO num_exp_sub VALUES (5,6,-77504.53913926); +INSERT INTO num_exp_mul VALUES (5,6,1539707782.76899778633766); +INSERT INTO num_exp_div VALUES (5,6,.17461941433576102689); +INSERT INTO num_exp_add VALUES (5,7,-83012087.961509); +INSERT INTO num_exp_sub VALUES (5,7,83044882.038491); +INSERT INTO num_exp_mul VALUES (5,7,-1361421264394.416135); +INSERT INTO num_exp_div VALUES (5,7,-.00019748690453643710); +INSERT INTO num_exp_add VALUES (5,8,91278.038491); +INSERT INTO num_exp_sub VALUES (5,8,-58483.961509); +INSERT INTO num_exp_mul VALUES (5,8,1227826639.244571); +INSERT INTO num_exp_div VALUES (5,8,.21897461960978085228); +INSERT INTO num_exp_add VALUES (5,9,-24910407.006556420); +INSERT INTO num_exp_sub VALUES (5,9,24943201.083538420); +INSERT INTO num_exp_mul VALUES (5,9,-408725765384.257043660243220); +INSERT INTO num_exp_div VALUES (5,9,-.00065780749354660427); +INSERT INTO num_exp_add VALUES (6,0,93901.57763026); +INSERT INTO num_exp_sub VALUES (6,0,93901.57763026); +INSERT INTO num_exp_mul VALUES (6,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (6,0,double('NaN')); +INSERT INTO num_exp_add VALUES (6,1,93901.57763026); +INSERT INTO num_exp_sub VALUES (6,1,93901.57763026); +INSERT INTO num_exp_mul VALUES (6,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (6,1,double('NaN')); +INSERT INTO num_exp_add VALUES (6,2,-34244590.637766787); +INSERT INTO num_exp_sub VALUES (6,2,34432393.793027307); +INSERT INTO num_exp_mul VALUES (6,2,-3224438592470.18449811926184222); +INSERT INTO num_exp_div VALUES (6,2,-.00273458651128995823); +INSERT INTO num_exp_add VALUES (6,3,93905.88763026); +INSERT INTO num_exp_sub VALUES (6,3,93897.26763026); +INSERT INTO num_exp_mul VALUES (6,3,404715.7995864206); +INSERT INTO num_exp_div VALUES (6,3,21786.90896293735498839907); +INSERT INTO num_exp_add VALUES (6,4,7893362.98953026); +INSERT INTO num_exp_sub VALUES (6,4,-7705559.83426974); +INSERT INTO num_exp_mul VALUES (6,4,732381731243.745115764094); +INSERT INTO num_exp_div VALUES (6,4,.01203949512295682469); +INSERT INTO num_exp_add VALUES (6,5,110298.61612126); +INSERT INTO num_exp_sub VALUES (6,5,77504.53913926); +INSERT INTO num_exp_mul VALUES (6,5,1539707782.76899778633766); +INSERT INTO num_exp_div VALUES (6,5,5.72674008674192359679); +INSERT INTO num_exp_add VALUES (6,6,187803.15526052); +INSERT INTO num_exp_sub VALUES (6,6,0); +INSERT INTO num_exp_mul VALUES (6,6,8817506281.4517452372676676); +INSERT INTO num_exp_div VALUES (6,6,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (6,7,-82934583.42236974); +INSERT INTO num_exp_sub VALUES (6,7,83122386.57763026); +INSERT INTO num_exp_mul VALUES (6,7,-7796505729750.37795610); +INSERT INTO num_exp_div VALUES (6,7,-.00113095617281538980); +INSERT INTO num_exp_add VALUES (6,8,168782.57763026); +INSERT INTO num_exp_sub VALUES (6,8,19020.57763026); +INSERT INTO num_exp_mul VALUES (6,8,7031444034.53149906); +INSERT INTO num_exp_div VALUES (6,8,1.25401073209839612184); +INSERT INTO num_exp_add VALUES (6,9,-24832902.467417160); +INSERT INTO num_exp_sub VALUES (6,9,25020705.622677680); +INSERT INTO num_exp_mul VALUES (6,9,-2340666225110.29929521292692920); +INSERT INTO num_exp_div VALUES (6,9,-.00376709254265256789); +INSERT INTO num_exp_add VALUES (7,0,-83028485); +INSERT INTO num_exp_sub VALUES (7,0,-83028485); +INSERT INTO num_exp_mul VALUES (7,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (7,0,double('NaN')); +INSERT INTO num_exp_add VALUES (7,1,-83028485); +INSERT INTO num_exp_sub VALUES (7,1,-83028485); +INSERT INTO num_exp_mul VALUES (7,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (7,1,double('NaN')); +INSERT INTO num_exp_add VALUES (7,2,-117366977.215397047); +INSERT INTO num_exp_sub VALUES (7,2,-48689992.784602953); +INSERT INTO num_exp_mul VALUES (7,2,2851072985828710.485883795); +INSERT INTO num_exp_div VALUES (7,2,2.41794207151503385700); +INSERT INTO num_exp_add VALUES (7,3,-83028480.69); +INSERT INTO num_exp_sub VALUES (7,3,-83028489.31); +INSERT INTO num_exp_mul VALUES (7,3,-357852770.35); +INSERT INTO num_exp_div VALUES (7,3,-19264149.65197215777262180974); +INSERT INTO num_exp_add VALUES (7,4,-75229023.5881); +INSERT INTO num_exp_sub VALUES (7,4,-90827946.4119); +INSERT INTO num_exp_mul VALUES (7,4,-647577464846017.9715); +INSERT INTO num_exp_div VALUES (7,4,-10.64541262725136247686); +INSERT INTO num_exp_add VALUES (7,5,-83012087.961509); +INSERT INTO num_exp_sub VALUES (7,5,-83044882.038491); +INSERT INTO num_exp_mul VALUES (7,5,-1361421264394.416135); +INSERT INTO num_exp_div VALUES (7,5,-5063.62688881730941836574); +INSERT INTO num_exp_add VALUES (7,6,-82934583.42236974); +INSERT INTO num_exp_sub VALUES (7,6,-83122386.57763026); +INSERT INTO num_exp_mul VALUES (7,6,-7796505729750.37795610); +INSERT INTO num_exp_div VALUES (7,6,-884.20756174009028770294); +INSERT INTO num_exp_add VALUES (7,7,-166056970); +INSERT INTO num_exp_sub VALUES (7,7,0); +INSERT INTO num_exp_mul VALUES (7,7,6893729321395225); +INSERT INTO num_exp_div VALUES (7,7,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (7,8,-82953604); +INSERT INTO num_exp_sub VALUES (7,8,-83103366); +INSERT INTO num_exp_mul VALUES (7,8,-6217255985285); +INSERT INTO num_exp_div VALUES (7,8,-1108.80577182462841041118); +INSERT INTO num_exp_add VALUES (7,9,-107955289.045047420); +INSERT INTO num_exp_sub VALUES (7,9,-58101680.954952580); +INSERT INTO num_exp_mul VALUES (7,9,2069634775752159.035758700); +INSERT INTO num_exp_div VALUES (7,9,3.33089171198810413382); +INSERT INTO num_exp_add VALUES (8,0,74881); +INSERT INTO num_exp_sub VALUES (8,0,74881); +INSERT INTO num_exp_mul VALUES (8,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (8,0,double('NaN')); +INSERT INTO num_exp_add VALUES (8,1,74881); +INSERT INTO num_exp_sub VALUES (8,1,74881); +INSERT INTO num_exp_mul VALUES (8,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (8,1,double('NaN')); +INSERT INTO num_exp_add VALUES (8,2,-34263611.215397047); +INSERT INTO num_exp_sub VALUES (8,2,34413373.215397047); +INSERT INTO num_exp_mul VALUES (8,2,-2571300635581.146276407); +INSERT INTO num_exp_div VALUES (8,2,-.00218067233500788615); +INSERT INTO num_exp_add VALUES (8,3,74885.31); +INSERT INTO num_exp_sub VALUES (8,3,74876.69); +INSERT INTO num_exp_mul VALUES (8,3,322737.11); +INSERT INTO num_exp_div VALUES (8,3,17373.78190255220417633410); +INSERT INTO num_exp_add VALUES (8,4,7874342.4119); +INSERT INTO num_exp_sub VALUES (8,4,-7724580.4119); +INSERT INTO num_exp_mul VALUES (8,4,584031469984.4839); +INSERT INTO num_exp_div VALUES (8,4,.00960079113741758956); +INSERT INTO num_exp_add VALUES (8,5,91278.038491); +INSERT INTO num_exp_sub VALUES (8,5,58483.961509); +INSERT INTO num_exp_mul VALUES (8,5,1227826639.244571); +INSERT INTO num_exp_div VALUES (8,5,4.56673929509287019456); +INSERT INTO num_exp_add VALUES (8,6,168782.57763026); +INSERT INTO num_exp_sub VALUES (8,6,-19020.57763026); +INSERT INTO num_exp_mul VALUES (8,6,7031444034.53149906); +INSERT INTO num_exp_div VALUES (8,6,.79744134113322314424); +INSERT INTO num_exp_add VALUES (8,7,-82953604); +INSERT INTO num_exp_sub VALUES (8,7,83103366); +INSERT INTO num_exp_mul VALUES (8,7,-6217255985285); +INSERT INTO num_exp_div VALUES (8,7,-.00090187120721280172); +INSERT INTO num_exp_add VALUES (8,8,149762); +INSERT INTO num_exp_sub VALUES (8,8,0); +INSERT INTO num_exp_mul VALUES (8,8,5607164161); +INSERT INTO num_exp_div VALUES (8,8,1.00000000000000000000); +INSERT INTO num_exp_add VALUES (8,9,-24851923.045047420); +INSERT INTO num_exp_sub VALUES (8,9,25001685.045047420); +INSERT INTO num_exp_mul VALUES (8,9,-1866544013697.195857020); +INSERT INTO num_exp_div VALUES (8,9,-.00300403532938582735); +INSERT INTO num_exp_add VALUES (9,0,-24926804.045047420); +INSERT INTO num_exp_sub VALUES (9,0,-24926804.045047420); +INSERT INTO num_exp_mul VALUES (9,0,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (9,0,double('NaN')); +INSERT INTO num_exp_add VALUES (9,1,-24926804.045047420); +INSERT INTO num_exp_sub VALUES (9,1,-24926804.045047420); +INSERT INTO num_exp_mul VALUES (9,1,0); +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_div VALUES (9,1,double('NaN')); +INSERT INTO num_exp_add VALUES (9,2,-59265296.260444467); +INSERT INTO num_exp_sub VALUES (9,2,9411688.170349627); +INSERT INTO num_exp_mul VALUES (9,2,855948866655588.453741509242968740); +INSERT INTO num_exp_div VALUES (9,2,.72591434384152961526); +INSERT INTO num_exp_add VALUES (9,3,-24926799.735047420); +INSERT INTO num_exp_sub VALUES (9,3,-24926808.355047420); +INSERT INTO num_exp_mul VALUES (9,3,-107434525.43415438020); +INSERT INTO num_exp_div VALUES (9,3,-5783481.21694835730858468677); +INSERT INTO num_exp_add VALUES (9,4,-17127342.633147420); +INSERT INTO num_exp_sub VALUES (9,4,-32726265.456947420); +INSERT INTO num_exp_mul VALUES (9,4,-194415646271340.1815956522980); +INSERT INTO num_exp_div VALUES (9,4,-3.19596478892958416484); +INSERT INTO num_exp_add VALUES (9,5,-24910407.006556420); +INSERT INTO num_exp_sub VALUES (9,5,-24943201.083538420); +INSERT INTO num_exp_mul VALUES (9,5,-408725765384.257043660243220); +INSERT INTO num_exp_div VALUES (9,5,-1520.20159364322004505807); +INSERT INTO num_exp_add VALUES (9,6,-24832902.467417160); +INSERT INTO num_exp_sub VALUES (9,6,-25020705.622677680); +INSERT INTO num_exp_mul VALUES (9,6,-2340666225110.29929521292692920); +INSERT INTO num_exp_div VALUES (9,6,-265.45671195426965751280); +INSERT INTO num_exp_add VALUES (9,7,-107955289.045047420); +INSERT INTO num_exp_sub VALUES (9,7,58101680.954952580); +INSERT INTO num_exp_mul VALUES (9,7,2069634775752159.035758700); +INSERT INTO num_exp_div VALUES (9,7,.30021990699995814689); +INSERT INTO num_exp_add VALUES (9,8,-24851923.045047420); +INSERT INTO num_exp_sub VALUES (9,8,-25001685.045047420); +INSERT INTO num_exp_mul VALUES (9,8,-1866544013697.195857020); +INSERT INTO num_exp_div VALUES (9,8,-332.88556569820675471748); +INSERT INTO num_exp_add VALUES (9,9,-49853608.090094840); +INSERT INTO num_exp_sub VALUES (9,9,0); +INSERT INTO num_exp_mul VALUES (9,9,621345559900192.420120630048656400); +INSERT INTO num_exp_div VALUES (9,9,1.00000000000000000000); +-- COMMIT TRANSACTION; +-- BEGIN TRANSACTION; +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO num_exp_sqrt VALUES (0,0); +INSERT INTO num_exp_sqrt VALUES (1,0); +INSERT INTO num_exp_sqrt VALUES (2,5859.90547836712524903505); +INSERT INTO num_exp_sqrt VALUES (3,2.07605394920266944396); +INSERT INTO num_exp_sqrt VALUES (4,2792.75158435189147418923); +INSERT INTO num_exp_sqrt VALUES (5,128.05092147657509145473); +INSERT INTO num_exp_sqrt VALUES (6,306.43364311096782703406); +INSERT INTO num_exp_sqrt VALUES (7,9111.99676251039939975230); +INSERT INTO num_exp_sqrt VALUES (8,273.64392922189960397542); +INSERT INTO num_exp_sqrt VALUES (9,4992.67503899937593364766); +-- COMMIT TRANSACTION; +-- BEGIN TRANSACTION; +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_ln VALUES (0,double('NaN')); +INSERT INTO num_exp_ln VALUES (1,double('NaN')); +INSERT INTO num_exp_ln VALUES (2,17.35177750493897715514); +INSERT INTO num_exp_ln VALUES (3,1.46093790411565641971); +INSERT INTO num_exp_ln VALUES (4,15.86956523951936572464); +INSERT INTO num_exp_ln VALUES (5,9.70485601768871834038); +INSERT INTO num_exp_ln VALUES (6,11.45000246622944403127); +INSERT INTO num_exp_ln VALUES (7,18.23469429965478772991); +INSERT INTO num_exp_ln VALUES (8,11.22365546576315513668); +INSERT INTO num_exp_ln VALUES (9,17.03145425013166006962); +-- COMMIT TRANSACTION; +-- BEGIN TRANSACTION; +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_log10 VALUES (0,double('NaN')); +INSERT INTO num_exp_log10 VALUES (1,double('NaN')); +INSERT INTO num_exp_log10 VALUES (2,7.53578122160797276459); +INSERT INTO num_exp_log10 VALUES (3,.63447727016073160075); +INSERT INTO num_exp_log10 VALUES (4,6.89206461372691743345); +INSERT INTO num_exp_log10 VALUES (5,4.21476541614777768626); +INSERT INTO num_exp_log10 VALUES (6,4.97267288886207207671); +INSERT INTO num_exp_log10 VALUES (7,7.91922711353275546914); +INSERT INTO num_exp_log10 VALUES (8,4.87437163556421004138); +INSERT INTO num_exp_log10 VALUES (9,7.39666659961986567059); +-- COMMIT TRANSACTION; +-- BEGIN TRANSACTION; +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +-- [SPARK-28315] Decimal can not accept NaN as input +INSERT INTO num_exp_power_10_ln VALUES (0,double('NaN')); +INSERT INTO num_exp_power_10_ln VALUES (1,double('NaN')); +INSERT INTO num_exp_power_10_ln VALUES (2,224790267919917955.13261618583642653184); +INSERT INTO num_exp_power_10_ln VALUES (3,28.90266599445155957393); +INSERT INTO num_exp_power_10_ln VALUES (4,7405685069594999.07733999469386277636); +INSERT INTO num_exp_power_10_ln VALUES (5,5068226527.32127265408584640098); +INSERT INTO num_exp_power_10_ln VALUES (6,281839893606.99372343357047819067); +-- In Spark, decimal can only support precision up to 38 +INSERT INTO num_exp_power_10_ln VALUES (7,1716699575118597095.42330819910640247627); +INSERT INTO num_exp_power_10_ln VALUES (8,167361463828.07491320069016125952); +INSERT INTO num_exp_power_10_ln VALUES (9,107511333880052007.04141124673540337457); +-- COMMIT TRANSACTION; +-- BEGIN TRANSACTION; +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO num_data VALUES (0, 0); +INSERT INTO num_data VALUES (1, 0); +INSERT INTO num_data VALUES (2, -34338492.215397047); +INSERT INTO num_data VALUES (3, 4.31); +INSERT INTO num_data VALUES (4, 7799461.4119); +INSERT INTO num_data VALUES (5, 16397.038491); +INSERT INTO num_data VALUES (6, 93901.57763026); +INSERT INTO num_data VALUES (7, -83028485); +INSERT INTO num_data VALUES (8, 74881); +INSERT INTO num_data VALUES (9, -24926804.045047420); +-- COMMIT TRANSACTION; + +SELECT * FROM num_data; + +-- ****************************** +-- * Create indices for faster checks +-- ****************************** + +-- CREATE UNIQUE INDEX num_exp_add_idx ON num_exp_add (id1, id2); +-- CREATE UNIQUE INDEX num_exp_sub_idx ON num_exp_sub (id1, id2); +-- CREATE UNIQUE INDEX num_exp_div_idx ON num_exp_div (id1, id2); +-- CREATE UNIQUE INDEX num_exp_mul_idx ON num_exp_mul (id1, id2); +-- CREATE UNIQUE INDEX num_exp_sqrt_idx ON num_exp_sqrt (id); +-- CREATE UNIQUE INDEX num_exp_ln_idx ON num_exp_ln (id); +-- CREATE UNIQUE INDEX num_exp_log10_idx ON num_exp_log10 (id); +-- CREATE UNIQUE INDEX num_exp_power_10_ln_idx ON num_exp_power_10_ln (id); + +-- VACUUM ANALYZE num_exp_add; +-- VACUUM ANALYZE num_exp_sub; +-- VACUUM ANALYZE num_exp_div; +-- VACUUM ANALYZE num_exp_mul; +-- VACUUM ANALYZE num_exp_sqrt; +-- VACUUM ANALYZE num_exp_ln; +-- VACUUM ANALYZE num_exp_log10; +-- VACUUM ANALYZE num_exp_power_10_ln; + +-- ****************************** +-- * Now check the behaviour of the NUMERIC type +-- ****************************** + +-- ****************************** +-- * Addition check +-- ****************************** +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, t1.val + t2.val + FROM num_data t1, num_data t2; +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_add t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected; + +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val + t2.val, 10) + FROM num_data t1, num_data t2; +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 10) as expected + FROM num_result t1, num_exp_add t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 10); + +-- ****************************** +-- * Subtraction check +-- ****************************** +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, t1.val - t2.val + FROM num_data t1, num_data t2; +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_sub t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected; + +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val - t2.val, 40) + FROM num_data t1, num_data t2; +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 40) + FROM num_result t1, num_exp_sub t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 40); + +-- ****************************** +-- * Multiply check +-- ****************************** +-- [SPARK-28316] Decimal precision issue +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, t1.val, t2.val, t1.val * t2.val + FROM num_data t1, num_data t2; +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_mul t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected; + +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val * t2.val, 30) + FROM num_data t1, num_data t2; +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 30) as expected + FROM num_result t1, num_exp_mul t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 30); + +-- ****************************** +-- * Division check +-- ****************************** +-- [SPARK-28316] Decimal precision issue +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, t1.val / t2.val + FROM num_data t1, num_data t2 + WHERE t2.val != '0.0'; +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_div t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected; + +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val / t2.val, 80) + FROM num_data t1, num_data t2 + WHERE t2.val != '0.0'; +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 80) as expected + FROM num_result t1, num_exp_div t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 80); + +-- ****************************** +-- * Square root check +-- ****************************** +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT id, 0, SQRT(ABS(val)) + FROM num_data; +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_sqrt t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected; + +-- ****************************** +-- * Natural logarithm check +-- ****************************** +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT id, 0, LN(ABS(val)) + FROM num_data + WHERE val != '0.0'; +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_ln t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected; + +-- ****************************** +-- * Logarithm base 10 check +-- ****************************** +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT id, 0, LOG(cast('10' as decimal(38, 18)), ABS(val)) + FROM num_data + WHERE val != '0.0'; +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_log10 t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected; + +-- ****************************** +-- * POWER(10, LN(value)) check +-- ****************************** +-- [SPARK-28316] Decimal precision issue +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT id, 0, POWER(cast('10' as decimal(38, 18)), LN(ABS(round(val,200)))) + FROM num_data + WHERE val != '0.0'; +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_power_10_ln t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected; + +-- ****************************** +-- * miscellaneous checks for things that have been broken in the past... +-- ****************************** +-- numeric AVG used to fail on some platforms +SELECT AVG(val) FROM num_data; +-- [SPARK-28316] STDDEV and VARIANCE returns double type +-- Skip it because: Expected "2.779120328758835[]E7", but got "2.779120328758835[4]E7" +-- SELECT STDDEV(val) FROM num_data; +-- Skip it because: Expected "7.72350980172061[8]E14", but got "7.72350980172061[6]E14" +-- SELECT VARIANCE(val) FROM num_data; + +-- Check for appropriate rounding and overflow +CREATE TABLE fract_only (id int, val decimal(4,4)) USING parquet; +INSERT INTO fract_only VALUES (1, 0.0); +INSERT INTO fract_only VALUES (2, 0.1); +-- [SPARK-27923] PostgreSQL throws an exception but Spark SQL is NULL +-- INSERT INTO fract_only VALUES (3, '1.0'); -- should fail +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO fract_only VALUES (4, -0.9999); +INSERT INTO fract_only VALUES (5, 0.99994); +-- [SPARK-27923] PostgreSQL throws an exception but Spark SQL is NULL +-- INSERT INTO fract_only VALUES (6, '0.99995'); -- should fail +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO fract_only VALUES (7, 0.00001); +INSERT INTO fract_only VALUES (8, 0.00017); +SELECT * FROM fract_only; +DROP TABLE fract_only; + +-- [SPARK-28315] Decimal can not accept NaN as input +-- [SPARK-27923] Decimal type can not accept Infinity and -Infinity +-- Check inf/nan conversion behavior +SELECT decimal(double('NaN')); +SELECT decimal(double('Infinity')); +SELECT decimal(double('-Infinity')); +SELECT decimal(float('NaN')); +SELECT decimal(float('Infinity')); +SELECT decimal(float('-Infinity')); + +-- Simple check that ceil(), floor(), and round() work correctly +CREATE TABLE ceil_floor_round (a decimal(38, 18)) USING parquet; +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO ceil_floor_round VALUES (-5.5); +INSERT INTO ceil_floor_round VALUES (-5.499999); +INSERT INTO ceil_floor_round VALUES (9.5); +INSERT INTO ceil_floor_round VALUES (9.4999999); +INSERT INTO ceil_floor_round VALUES (0.0); +INSERT INTO ceil_floor_round VALUES (0.0000001); +INSERT INTO ceil_floor_round VALUES (-0.000001); +SELECT a, ceil(a), ceiling(a), floor(a), round(a) FROM ceil_floor_round; +DROP TABLE ceil_floor_round; + +-- [SPARK-28007] Caret operator (^) means bitwise XOR in Spark and exponentiation in Postgres +-- Check rounding, it should round ties away from zero. +-- SELECT i as pow, +-- round((-2.5 * 10 ^ i)::numeric, -i), +-- round((-1.5 * 10 ^ i)::numeric, -i), +-- round((-0.5 * 10 ^ i)::numeric, -i), +-- round((0.5 * 10 ^ i)::numeric, -i), +-- round((1.5 * 10 ^ i)::numeric, -i), +-- round((2.5 * 10 ^ i)::numeric, -i) +-- FROM generate_series(-5,5) AS t(i); + +-- Testing for width_bucket(). For convenience, we test both the +-- numeric and float8 versions of the function in this file. + +-- errors +SELECT width_bucket(5.0, 3.0, 4.0, 0); +SELECT width_bucket(5.0, 3.0, 4.0, -5); +SELECT width_bucket(3.5, 3.0, 3.0, 888); +SELECT width_bucket(double(5.0), double(3.0), double(4.0), 0); +SELECT width_bucket(double(5.0), double(3.0), double(4.0), -5); +SELECT width_bucket(double(3.5), double(3.0), double(3.0), 888); +SELECT width_bucket('NaN', 3.0, 4.0, 888); +SELECT width_bucket(double(0), 'NaN', double(4.0), 888); + +-- normal operation +-- CREATE TABLE width_bucket_test (operand_num numeric, operand_f8 float8); +CREATE TABLE width_bucket_test (operand_num decimal(30,15), operand_f8 double) USING parquet; + +-- COPY width_bucket_test (operand_num) FROM stdin; +-- -5.2 +-- -0.0000000001 +-- 0.000000000001 +-- 1 +-- 1.99999999999999 +-- 2 +-- 2.00000000000001 +-- 3 +-- 4 +-- 4.5 +-- 5 +-- 5.5 +-- 6 +-- 7 +-- 8 +-- 9 +-- 9.99999999999999 +-- 10 +-- 10.0000000000001 +-- \. + +-- UPDATE width_bucket_test SET operand_f8 = operand_num::float8; + +INSERT INTO width_bucket_test VALUES + (-5.2, -5.2), + (-0.0000000001, -0.0000000001), + (0.000000000001, 0.000000000001), + (1, 1), + (1.99999999999999, 1.99999999999999), + (2, 2), + (2.00000000000001, 2.00000000000001), + (3, 3), + (4, 4), + (4.5, 4.5), + (5, 5), + (5.5, 5.5), + (6, 6), + (7, 7), + (8, 8), + (9, 9), + (9.99999999999999, 9.99999999999999), + (10, 10), + (10.0000000000001, 10.0000000000001); + +SELECT + operand_num, + width_bucket(operand_num, 0, 10, 5) AS wb_1, + width_bucket(operand_f8, 0, 10, 5) AS wb_1f, + width_bucket(operand_num, 10, 0, 5) AS wb_2, + width_bucket(operand_f8, 10, 0, 5) AS wb_2f, + width_bucket(operand_num, 2, 8, 4) AS wb_3, + width_bucket(operand_f8, 2, 8, 4) AS wb_3f, + width_bucket(operand_num, 5.0, 5.5, 20) AS wb_4, + width_bucket(operand_f8, 5.0, 5.5, 20) AS wb_4f, + width_bucket(operand_num, -25, 25, 10) AS wb_5, + width_bucket(operand_f8, -25, 25, 10) AS wb_5f + FROM width_bucket_test + ORDER BY operand_num ASC; + +-- for float8 only, check positive and negative infinity: we require +-- finite bucket bounds, but allow an infinite operand +SELECT width_bucket(double(0.0), double('Infinity'), 5, 10); -- error +SELECT width_bucket(double(0.0), 5, double('-Infinity'), 20); -- error +SELECT width_bucket(double('Infinity'), 1, 10, 10), + width_bucket(double('-Infinity'), 1, 10, 10); + +DROP TABLE width_bucket_test; + +-- TO_CHAR() +-- some queries are commented out as the format string is not supported by Spark +SELECT '' AS to_char_3, to_char(val, '9999999999999999.999999999999999PR'), val +FROM num_data; + +SELECT '' AS to_char_4, to_char(val, '9999999999999999.999999999999999S'), val +FROM num_data; + +SELECT '' AS to_char_5, to_char(val, 'MI9999999999999999.999999999999999'), val FROM num_data; +-- SELECT '' AS to_char_6, to_char(val, 'FMS9999999999999999.999999999999999'), val FROM num_data; +-- SELECT '' AS to_char_7, to_char(val, 'FM9999999999999999.999999999999999THPR'), val FROM num_data; +-- SELECT '' AS to_char_8, to_char(val, 'SG9999999999999999.999999999999999th'), val FROM num_data; +SELECT '' AS to_char_9, to_char(val, '0999999999999999.999999999999999'), val FROM num_data; +SELECT '' AS to_char_10, to_char(val, 'S0999999999999999.999999999999999'), val FROM num_data; +-- SELECT '' AS to_char_11, to_char(val, 'FM0999999999999999.999999999999999'), val FROM num_data; +-- SELECT '' AS to_char_12, to_char(val, 'FM9999999999999999.099999999999999'), val FROM num_data; +-- SELECT '' AS to_char_13, to_char(val, 'FM9999999999990999.990999999999999'), val FROM num_data; +-- SELECT '' AS to_char_14, to_char(val, 'FM0999999999999999.999909999999999'), val FROM num_data; +-- SELECT '' AS to_char_15, to_char(val, 'FM9999999990999999.099999999999999'), val FROM num_data; +-- SELECT '' AS to_char_16, to_char(val, 'L9999999999999999.099999999999999'), val FROM num_data; +-- SELECT '' AS to_char_17, to_char(val, 'FM9999999999999999.99999999999999'), val FROM num_data; +-- SELECT '' AS to_char_18, to_char(val, 'S 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9'), val FROM num_data; +-- SELECT '' AS to_char_19, to_char(val, 'FMS 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9'), val FROM num_data; +-- SELECT '' AS to_char_20, to_char(val, E'99999 "text" 9999 "9999" 999 "\\"text between quote marks\\"" 9999'), val FROM num_data; +-- SELECT '' AS to_char_21, to_char(val, '999999SG9999999999'), val FROM num_data; +-- SELECT '' AS to_char_22, to_char(val, 'FM9999999999999999.999999999999999'), val FROM num_data; +-- SELECT '' AS to_char_23, to_char(val, '9.999EEEE'), val FROM num_data; + +-- SELECT '' AS to_char_24, to_char('100'::numeric, 'FM999.9'); +-- SELECT '' AS to_char_25, to_char('100'::numeric, 'FM999.'); +-- SELECT '' AS to_char_26, to_char('100'::numeric, 'FM999'); + +-- Check parsing of literal text in a format string +-- SELECT '' AS to_char_27, to_char('100'::numeric, 'foo999'); +-- SELECT '' AS to_char_28, to_char('100'::numeric, 'f\oo999'); +-- SELECT '' AS to_char_29, to_char('100'::numeric, 'f\\oo999'); +-- SELECT '' AS to_char_30, to_char('100'::numeric, 'f\"oo999'); +-- SELECT '' AS to_char_31, to_char('100'::numeric, 'f\\"oo999'); +-- SELECT '' AS to_char_32, to_char('100'::numeric, 'f"ool"999'); +-- SELECT '' AS to_char_33, to_char('100'::numeric, 'f"\ool"999'); +-- SELECT '' AS to_char_34, to_char('100'::numeric, 'f"\\ool"999'); +-- SELECT '' AS to_char_35, to_char('100'::numeric, 'f"ool\"999'); +-- SELECT '' AS to_char_36, to_char('100'::numeric, 'f"ool\\"999'); + +-- [SPARK-28137] Missing Data Type Formatting Functions: TO_NUMBER +-- TO_NUMBER() +-- some queries are commented out as the format string is not supported by Spark +-- SET lc_numeric = 'C'; +SELECT '' AS to_number_1, to_number('-34,338,492', '99G999G999'); +SELECT '' AS to_number_2, to_number('-34,338,492.654,878', '99G999G999D999G999'); +-- SELECT '' AS to_number_3, to_number('<564646.654564>', '999999.999999PR'); +SELECT '' AS to_number_4, to_number('0.00001-', '9.999999S'); +-- SELECT '' AS to_number_5, to_number('5.01-', 'FM9.999999S'); +-- SELECT '' AS to_number_5, to_number('5.01-', 'FM9.999999MI'); +-- SELECT '' AS to_number_7, to_number('5 4 4 4 4 8 . 7 8', '9 9 9 9 9 9 . 9 9'); +-- SELECT '' AS to_number_8, to_number('.01', 'FM9.99'); +SELECT '' AS to_number_9, to_number('.0', '99999999.99999999'); +SELECT '' AS to_number_10, to_number('0', '99.99'); +-- SELECT '' AS to_number_11, to_number('.-01', 'S99.99'); +SELECT '' AS to_number_12, to_number('.01-', '99.99S'); +-- SELECT '' AS to_number_13, to_number(' . 0 1-', ' 9 9 . 9 9 S'); +SELECT '' AS to_number_14, to_number('34,50','999,99'); +SELECT '' AS to_number_15, to_number('123,000','999G'); +SELECT '' AS to_number_16, to_number('123456','999G999'); +-- SELECT '' AS to_number_17, to_number('$1234.56','L9,999.99'); +-- SELECT '' AS to_number_18, to_number('$1234.56','L99,999.99'); +-- SELECT '' AS to_number_19, to_number('$1,234.56','L99,999.99'); +-- SELECT '' AS to_number_20, to_number('1234.56','L99,999.99'); +-- SELECT '' AS to_number_21, to_number('1,234.56','L99,999.99'); +-- SELECT '' AS to_number_22, to_number('42nd', '99th'); +-- RESET lc_numeric; + +-- +-- Input syntax +-- + +CREATE TABLE num_input_test (n1 decimal(38, 18)) USING parquet; + +-- good inputs +-- PostgreSQL implicitly casts string literals to data with decimal types, but +-- Spark does not support that kind of implicit casts. To test all the INSERT queries below, +-- we rewrote them into the other typed literals. +INSERT INTO num_input_test VALUES (double(trim(' 123'))); +INSERT INTO num_input_test VALUES (double(trim(' 3245874 '))); +INSERT INTO num_input_test VALUES (double(trim(' -93853'))); +INSERT INTO num_input_test VALUES (555.50); +INSERT INTO num_input_test VALUES (-555.50); +-- [SPARK-28315] Decimal can not accept NaN as input +-- INSERT INTO num_input_test VALUES (trim('NaN ')); +-- INSERT INTO num_input_test VALUES (trim(' nan')); + +-- [SPARK-27923] Spark SQL accept bad inputs to NULL +-- bad inputs +-- INSERT INTO num_input_test VALUES (' '); +-- INSERT INTO num_input_test VALUES (' 1234 %'); +-- INSERT INTO num_input_test VALUES ('xyz'); +-- INSERT INTO num_input_test VALUES ('- 1234'); +-- INSERT INTO num_input_test VALUES ('5 . 0'); +-- INSERT INTO num_input_test VALUES ('5. 0 '); +-- INSERT INTO num_input_test VALUES (''); +-- INSERT INTO num_input_test VALUES (' N aN '); + +SELECT * FROM num_input_test; + +-- [SPARK-28318] Decimal can only support precision up to 38 +-- +-- Test some corner cases for multiplication +-- + +-- select 4790999999999999999999999999999999999999999999999999999999999999999999999999999999999999 * 9999999999999999999999999999999999999999999999999999999999999999999999999999999999999999; + +-- select 4789999999999999999999999999999999999999999999999999999999999999999999999999999999999999 * 9999999999999999999999999999999999999999999999999999999999999999999999999999999999999999; + +-- select 4770999999999999999999999999999999999999999999999999999999999999999999999999999999999999 * 9999999999999999999999999999999999999999999999999999999999999999999999999999999999999999; + +-- select 4769999999999999999999999999999999999999999999999999999999999999999999999999999999999999 * 9999999999999999999999999999999999999999999999999999999999999999999999999999999999999999; + +-- +-- Test some corner cases for division +-- +-- 999999999999999999999 is overflow for SYSTEM_DEFAULT(decimal(38, 18)), we use BigIntDecimal(decimal(38, 0)). +select cast(999999999999999999999 as decimal(38, 0))/1000000000000000000000; + +select div(cast(999999999999999999999 as decimal(38, 0)),1000000000000000000000); +select mod(cast(999999999999999999999 as decimal(38, 0)),1000000000000000000000); +select div(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000); +select mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000); +select div(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000)*1000000000000000000000 + mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000); +select mod (70.0,70) ; +select div (70.0,70) ; +select 70.0 / 70 ; +select 12345678901234567890 % 123; +-- [SPARK-2659] HiveQL: Division operator should always perform fractional division +-- select 12345678901234567890 DIV 123; +-- select div(12345678901234567890, 123); +-- select div(12345678901234567890, 123) * 123 + 12345678901234567890 % 123; + +-- [SPARK-28007] Caret operator (^) means bitwise XOR in Spark and exponentiation in Postgres +-- +-- Test code path for raising to integer powers +-- + +-- select 10.0 ^ -2147483648 as rounds_to_zero; +-- select 10.0 ^ -2147483647 as rounds_to_zero; +-- select 10.0 ^ 2147483647 as overflows; +-- select 117743296169.0 ^ 1000000000 as overflows; + +-- cases that used to return inaccurate results +-- select 3.789 ^ 21; +-- select 3.789 ^ 35; +-- select 1.2 ^ 345; +-- select 0.12 ^ (-20); + +-- cases that used to error out +-- select 0.12 ^ (-25); +-- select 0.5678 ^ (-85); + +-- +-- Tests for raising to non-integer powers +-- + +-- special cases +-- select 0.0 ^ 0.0; +-- select (-12.34) ^ 0.0; +-- select 12.34 ^ 0.0; +-- select 0.0 ^ 12.34; + +-- NaNs +-- select 'NaN'::numeric ^ 'NaN'::numeric; +-- select 'NaN'::numeric ^ 0; +-- select 'NaN'::numeric ^ 1; +-- select 0 ^ 'NaN'::numeric; +-- select 1 ^ 'NaN'::numeric; + +-- invalid inputs +-- select 0.0 ^ (-12.34); +-- select (-12.34) ^ 1.2; + +-- cases that used to generate inaccurate results +-- select 32.1 ^ 9.8; +-- select 32.1 ^ (-9.8); +-- select 12.3 ^ 45.6; +-- select 12.3 ^ (-45.6); + +-- big test +-- select 1.234 ^ 5678; + +-- +-- Tests for EXP() +-- + +-- special cases +select exp(0.0); +select exp(1.0); +-- [SPARK-28316] EXP returns double type for decimal input +-- [SPARK-28318] Decimal can only support precision up to 38 +-- select exp(1.0::numeric(71,70)); + +-- cases that used to generate inaccurate results +select exp(32.999); +select exp(-32.999); +select exp(123.456); +select exp(-123.456); + +-- big test +select exp(1234.5678); + +-- +-- Tests for generate_series +-- +select * from range(cast(0.0 as decimal(38, 18)), cast(4.0 as decimal(38, 18))); +select * from range(cast(0.1 as decimal(38, 18)), cast(4.0 as decimal(38, 18)), cast(1.3 as decimal(38, 18))); +select * from range(cast(4.0 as decimal(38, 18)), cast(-1.5 as decimal(38, 18)), cast(-2.2 as decimal(38, 18))); +-- Trigger errors +-- select * from generate_series(-100::numeric, 100::numeric, 0::numeric); +-- select * from generate_series(-100::numeric, 100::numeric, 'nan'::numeric); +-- select * from generate_series('nan'::numeric, 100::numeric, 10::numeric); +-- select * from generate_series(0::numeric, 'nan'::numeric, 10::numeric); +-- [SPARK-28007] Caret operator (^) means bitwise XOR in Spark and exponentiation in Postgres +-- Checks maximum, output is truncated +-- select (i / (10::numeric ^ 131071))::numeric(1,0) +-- from generate_series(6 * (10::numeric ^ 131071), +-- 9 * (10::numeric ^ 131071), +-- 10::numeric ^ 131071) as a(i); +-- Check usage with variables +-- select * from generate_series(1::numeric, 3::numeric) i, generate_series(i,3) j; +-- select * from generate_series(1::numeric, 3::numeric) i, generate_series(1,i) j; +-- select * from generate_series(1::numeric, 3::numeric) i, generate_series(1,5,i) j; + +-- +-- Tests for LN() +-- + +-- [SPARK-27923] Invalid inputs for LN throws exception at PostgreSQL +-- Invalid inputs +-- select ln(-12.34); +-- select ln(0.0); + +-- Some random tests +select ln(1.2345678e-28); +select ln(0.0456789); +-- [SPARK-28318] Decimal can only support precision up to 38 +-- select ln(0.349873948359354029493948309745709580730482050975); +select ln(0.99949452); +select ln(1.00049687395); +select ln(1234.567890123456789); +select ln(5.80397490724e5); +select ln(9.342536355e34); + +-- +-- Tests for LOG() (base 10) +-- + +-- [SPARK-27923] Invalid inputs for LOG throws exception at PostgreSQL +-- invalid inputs +-- select log(-12.34); +-- select log(0.0); + +-- some random tests +-- [SPARK-28318] Decimal can only support precision up to 38 +-- select log(1.234567e-89); +-- [SPARK-28324] The LOG function using 10 as the base, but Spark using E +select log(3.4634998359873254962349856073435545); +select log(9.999999999999999999); +select log(10.00000000000000000); +select log(10.00000000000000001); +select log(590489.45235237); + +-- +-- Tests for LOG() (arbitrary base) +-- + +-- [SPARK-27923] Invalid inputs for LOG throws exception at PostgreSQL +-- invalid inputs +-- select log(-12.34, 56.78); +-- select log(-12.34, -56.78); +-- select log(12.34, -56.78); +-- select log(0.0, 12.34); +-- select log(12.34, 0.0); +-- select log(1.0, 12.34); + +-- some random tests +-- [SPARK-28318] Decimal can only support precision up to 38 +-- select log(1.23e-89, 6.4689e45); +select log(0.99923, 4.58934e34); +select log(1.000016, 8.452010e18); +-- [SPARK-28318] Decimal can only support precision up to 38 +-- select log(3.1954752e47, 9.4792021e-73); + +-- [SPARK-28317] Built-in Mathematical Functions: SCALE +-- +-- Tests for scale() +-- + +-- select scale(numeric 'NaN'); +-- select scale(NULL::numeric); +-- select scale(1.12); +-- select scale(0); +-- select scale(0.00); +-- select scale(1.12345); +-- select scale(110123.12475871856128); +-- select scale(-1123.12471856128); +-- select scale(-13.000000000000000); + +-- +-- Tests for SUM() +-- + +-- cases that need carry propagation +SELECT SUM(decimal(9999)) FROM range(1, 100001); +SELECT SUM(decimal(-9999)) FROM range(1, 100001); + +DROP TABLE num_data; +DROP TABLE num_exp_add; +DROP TABLE num_exp_sub; +DROP TABLE num_exp_div; +DROP TABLE num_exp_mul; +DROP TABLE num_exp_sqrt; +DROP TABLE num_exp_ln; +DROP TABLE num_exp_log10; +DROP TABLE num_exp_power_10_ln; +DROP TABLE num_result; +DROP TABLE num_input_test; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/select.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/select.sql new file mode 100644 index 000000000000..1f83d6c41661 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/select.sql @@ -0,0 +1,285 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- SELECT +-- Test int8 64-bit integers. +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select.sql +-- +create or replace temporary view onek2 as select * from onek; +create or replace temporary view INT8_TBL as select * from values + (cast(trim(' 123 ') as bigint), cast(trim(' 456') as bigint)), + (cast(trim('123 ') as bigint),cast('4567890123456789' as bigint)), + (cast('4567890123456789' as bigint),cast('123' as bigint)), + (cast(+4567890123456789 as bigint),cast('4567890123456789' as bigint)), + (cast('+4567890123456789' as bigint),cast('-4567890123456789' as bigint)) + as INT8_TBL(q1, q2); + +-- btree index +-- awk '{if($1<10){print;}else{next;}}' onek.data | sort +0n -1 +-- +SELECT * FROM onek + WHERE onek.unique1 < 10 + ORDER BY onek.unique1; + +-- [SPARK-28010] Support ORDER BY ... USING syntax +-- +-- awk '{if($1<20){print $1,$14;}else{next;}}' onek.data | sort +0nr -1 +-- +SELECT onek.unique1, onek.stringu1 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 DESC; + +-- +-- awk '{if($1>980){print $1,$14;}else{next;}}' onek.data | sort +1d -2 +-- +SELECT onek.unique1, onek.stringu1 FROM onek + WHERE onek.unique1 > 980 + ORDER BY stringu1 ASC; + +-- +-- awk '{if($1>980){print $1,$16;}else{next;}}' onek.data | +-- sort +1d -2 +0nr -1 +-- +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 > 980 + ORDER BY string4 ASC, unique1 DESC; + +-- +-- awk '{if($1>980){print $1,$16;}else{next;}}' onek.data | +-- sort +1dr -2 +0n -1 +-- +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 > 980 + ORDER BY string4 DESC, unique1 ASC; + +-- +-- awk '{if($1<20){print $1,$16;}else{next;}}' onek.data | +-- sort +0nr -1 +1d -2 +-- +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 DESC, string4 ASC; + +-- +-- awk '{if($1<20){print $1,$16;}else{next;}}' onek.data | +-- sort +0n -1 +1dr -2 +-- +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 ASC, string4 DESC; + +-- +-- test partial btree indexes +-- +-- As of 7.2, planner probably won't pick an indexscan without stats, +-- so ANALYZE first. Also, we want to prevent it from picking a bitmapscan +-- followed by sort, because that could hide index ordering problems. +-- +-- ANALYZE onek2; + +-- SET enable_seqscan TO off; +-- SET enable_bitmapscan TO off; +-- SET enable_sort TO off; + +-- +-- awk '{if($1<10){print $0;}else{next;}}' onek.data | sort +0n -1 +-- +SELECT onek2.* FROM onek2 WHERE onek2.unique1 < 10; + +-- +-- awk '{if($1<20){print $1,$14;}else{next;}}' onek.data | sort +0nr -1 +-- +SELECT onek2.unique1, onek2.stringu1 FROM onek2 + WHERE onek2.unique1 < 20 + ORDER BY unique1 DESC; + +-- +-- awk '{if($1>980){print $1,$14;}else{next;}}' onek.data | sort +1d -2 +-- +SELECT onek2.unique1, onek2.stringu1 FROM onek2 + WHERE onek2.unique1 > 980; + +-- RESET enable_seqscan; +-- RESET enable_bitmapscan; +-- RESET enable_sort; + +-- [SPARK-28329] SELECT INTO syntax +-- SELECT two, stringu1, ten, string4 +-- INTO TABLE tmp +-- FROM onek; +CREATE TABLE tmp USING parquet AS +SELECT two, stringu1, ten, string4 +FROM onek; + +-- Skip the person table because there is a point data type that we don't support. +-- +-- awk '{print $1,$2;}' person.data | +-- awk '{if(NF!=2){print $3,$2;}else{print;}}' - emp.data | +-- awk '{if(NF!=2){print $3,$2;}else{print;}}' - student.data | +-- awk 'BEGIN{FS=" ";}{if(NF!=2){print $4,$5;}else{print;}}' - stud_emp.data +-- +-- SELECT name, age FROM person*; ??? check if different +-- SELECT p.name, p.age FROM person* p; + +-- +-- awk '{print $1,$2;}' person.data | +-- awk '{if(NF!=2){print $3,$2;}else{print;}}' - emp.data | +-- awk '{if(NF!=2){print $3,$2;}else{print;}}' - student.data | +-- awk 'BEGIN{FS=" ";}{if(NF!=1){print $4,$5;}else{print;}}' - stud_emp.data | +-- sort +1nr -2 +-- +-- SELECT p.name, p.age FROM person* p ORDER BY age DESC, name; + +-- [SPARK-28330] Enhance query limit +-- +-- Test some cases involving whole-row Var referencing a subquery +-- +select foo.* from (select 1) as foo; +select foo.* from (select null) as foo; +select foo.* from (select 'xyzzy',1,null) as foo; + +-- +-- Test VALUES lists +-- +select * from onek, values(147, 'RFAAAA'), (931, 'VJAAAA') as v (i, j) + WHERE onek.unique1 = v.i and onek.stringu1 = v.j; + +-- [SPARK-28296] Improved VALUES support +-- a more complex case +-- looks like we're coding lisp :-) +-- select * from onek, +-- (values ((select i from +-- (values(10000), (2), (389), (1000), (2000), ((select 10029))) as foo(i) +-- order by i asc limit 1))) bar (i) +-- where onek.unique1 = bar.i; + +-- try VALUES in a subquery +-- select * from onek +-- where (unique1,ten) in (values (1,1), (20,0), (99,9), (17,99)) +-- order by unique1; + +-- VALUES is also legal as a standalone query or a set-operation member +VALUES (1,2), (3,4+4), (7,77.7); + +VALUES (1,2), (3,4+4), (7,77.7) +UNION ALL +SELECT 2+2, 57 +UNION ALL +TABLE int8_tbl; + +-- +-- Test ORDER BY options +-- + +CREATE OR REPLACE TEMPORARY VIEW foo AS +SELECT * FROM (values(42),(3),(10),(7),(null),(null),(1)) as foo (f1); + +-- [SPARK-28333] NULLS FIRST for DESC and NULLS LAST for ASC +SELECT * FROM foo ORDER BY f1; +SELECT * FROM foo ORDER BY f1 ASC; -- same thing +SELECT * FROM foo ORDER BY f1 NULLS FIRST; +SELECT * FROM foo ORDER BY f1 DESC; +SELECT * FROM foo ORDER BY f1 DESC NULLS LAST; + +-- check if indexscans do the right things +-- CREATE INDEX fooi ON foo (f1); +-- SET enable_sort = false; + +-- SELECT * FROM foo ORDER BY f1; +-- SELECT * FROM foo ORDER BY f1 NULLS FIRST; +-- SELECT * FROM foo ORDER BY f1 DESC; +-- SELECT * FROM foo ORDER BY f1 DESC NULLS LAST; + +-- DROP INDEX fooi; +-- CREATE INDEX fooi ON foo (f1 DESC); + +-- SELECT * FROM foo ORDER BY f1; +-- SELECT * FROM foo ORDER BY f1 NULLS FIRST; +-- SELECT * FROM foo ORDER BY f1 DESC; +-- SELECT * FROM foo ORDER BY f1 DESC NULLS LAST; + +-- DROP INDEX fooi; +-- CREATE INDEX fooi ON foo (f1 DESC NULLS LAST); + +-- SELECT * FROM foo ORDER BY f1; +-- SELECT * FROM foo ORDER BY f1 NULLS FIRST; +-- SELECT * FROM foo ORDER BY f1 DESC; +-- SELECT * FROM foo ORDER BY f1 DESC NULLS LAST; + +-- +-- Test planning of some cases with partial indexes +-- + +-- partial index is usable +-- explain (costs off) +-- select * from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +select * from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +-- actually run the query with an analyze to use the partial index +-- explain (costs off, analyze on, timing off, summary off) +-- select * from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +select unique2 from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +-- partial index predicate implies clause, so no need for retest +-- explain (costs off) +-- select * from onek2 where unique2 = 11 and stringu1 < 'B'; +select * from onek2 where unique2 = 11 and stringu1 < 'B'; +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'B'; +select unique2 from onek2 where unique2 = 11 and stringu1 < 'B'; +-- but if it's an update target, must retest anyway +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' for update; +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' for update; +-- partial index is not applicable +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'C'; +select unique2 from onek2 where unique2 = 11 and stringu1 < 'C'; +-- partial index implies clause, but bitmap scan must recheck predicate anyway +-- SET enable_indexscan TO off; +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'B'; +select unique2 from onek2 where unique2 = 11 and stringu1 < 'B'; +-- RESET enable_indexscan; +-- check multi-index cases too +-- explain (costs off) +-- select unique1, unique2 from onek2 +-- where (unique2 = 11 or unique1 = 0) and stringu1 < 'B'; +select unique1, unique2 from onek2 + where (unique2 = 11 or unique1 = 0) and stringu1 < 'B'; +-- explain (costs off) +-- select unique1, unique2 from onek2 +-- where (unique2 = 11 and stringu1 < 'B') or unique1 = 0; +select unique1, unique2 from onek2 + where (unique2 = 11 and stringu1 < 'B') or unique1 = 0; + +-- +-- Test some corner cases that have been known to confuse the planner +-- + +-- ORDER BY on a constant doesn't really need any sorting +SELECT 1 AS x ORDER BY x; + +-- But ORDER BY on a set-valued expression does +-- create function sillysrf(int) returns setof int as +-- 'values (1),(10),(2),($1)' language sql immutable; + +-- select sillysrf(42); +-- select sillysrf(-1) order by 1; + +-- drop function sillysrf(int); + +-- X = X isn't a no-op, it's effectively X IS NOT NULL assuming = is strict +-- (see bug #5084) +select * from (values (2),(null),(1)) v(k) where k = k order by k; +select * from (values (2),(null),(1)) v(k) where k = k; + +-- Test partitioned tables with no partitions, which should be handled the +-- same as the non-inheritance case when expanding its RTE. +-- create table list_parted_tbl (a int,b int) partition by list (a); +-- create table list_parted_tbl1 partition of list_parted_tbl +-- for values in (1) partition by list(b); +-- explain (costs off) select * from list_parted_tbl; +-- drop table list_parted_tbl; +drop table tmp; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/select_distinct.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/select_distinct.sql new file mode 100644 index 000000000000..5306028e5bd7 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/select_distinct.sql @@ -0,0 +1,86 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- SELECT_DISTINCT +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_distinct.sql +-- + +CREATE OR REPLACE TEMPORARY VIEW tmp AS +SELECT two, stringu1, ten, string4 +FROM onek; + +-- +-- awk '{print $3;}' onek.data | sort -n | uniq +-- +SELECT DISTINCT two FROM tmp ORDER BY 1; + +-- +-- awk '{print $5;}' onek.data | sort -n | uniq +-- +SELECT DISTINCT ten FROM tmp ORDER BY 1; + +-- +-- awk '{print $16;}' onek.data | sort -d | uniq +-- +SELECT DISTINCT string4 FROM tmp ORDER BY 1; + +-- [SPARK-28010] Support ORDER BY ... USING syntax +-- +-- awk '{print $3,$16,$5;}' onek.data | sort -d | uniq | +-- sort +0n -1 +1d -2 +2n -3 +-- +-- SELECT DISTINCT two, string4, ten +-- FROM tmp +-- ORDER BY two using <, string4 using <, ten using <; +SELECT DISTINCT two, string4, ten + FROM tmp + ORDER BY two ASC, string4 ASC, ten ASC; + +-- Skip the person table because there is a point data type that we don't support. +-- +-- awk '{print $2;}' person.data | +-- awk '{if(NF!=1){print $2;}else{print;}}' - emp.data | +-- awk '{if(NF!=1){print $2;}else{print;}}' - student.data | +-- awk 'BEGIN{FS=" ";}{if(NF!=1){print $5;}else{print;}}' - stud_emp.data | +-- sort -n -r | uniq +-- +-- SELECT DISTINCT p.age FROM person* p ORDER BY age using >; + +-- +-- Check mentioning same column more than once +-- + +-- EXPLAIN (VERBOSE, COSTS OFF) +-- SELECT count(*) FROM +-- (SELECT DISTINCT two, four, two FROM tenk1) ss; + +SELECT count(*) FROM + (SELECT DISTINCT two, four, two FROM tenk1) ss; + +-- +-- Also, some tests of IS DISTINCT FROM, which doesn't quite deserve its +-- very own regression file. +-- + +CREATE OR REPLACE TEMPORARY VIEW disttable AS SELECT * FROM + (VALUES (1), (2), (3), (NULL)) + AS v(f1); + +-- basic cases +SELECT f1, f1 IS DISTINCT FROM 2 as `not 2` FROM disttable; +SELECT f1, f1 IS DISTINCT FROM NULL as `not null` FROM disttable; +SELECT f1, f1 IS DISTINCT FROM f1 as `false` FROM disttable; +SELECT f1, f1 IS DISTINCT FROM f1+1 as `not null` FROM disttable; + +-- check that optimizer constant-folds it properly +SELECT 1 IS DISTINCT FROM 2 as `yes`; +SELECT 2 IS DISTINCT FROM 2 as `no`; +SELECT 2 IS DISTINCT FROM null as `yes`; +SELECT null IS DISTINCT FROM null as `no`; + +-- negated form +SELECT 1 IS NOT DISTINCT FROM 2 as `no`; +SELECT 2 IS NOT DISTINCT FROM 2 as `yes`; +SELECT 2 IS NOT DISTINCT FROM null as `no`; +SELECT null IS NOT DISTINCT FROM null as `yes`; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/select_having.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/select_having.sql new file mode 100644 index 000000000000..0efe0877e9b3 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/select_having.sql @@ -0,0 +1,56 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- SELECT_HAVING +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql +-- + +-- load test data +CREATE TABLE test_having (a int, b int, c string, d string) USING parquet; +INSERT INTO test_having VALUES (0, 1, 'XXXX', 'A'); +INSERT INTO test_having VALUES (1, 2, 'AAAA', 'b'); +INSERT INTO test_having VALUES (2, 2, 'AAAA', 'c'); +INSERT INTO test_having VALUES (3, 3, 'BBBB', 'D'); +INSERT INTO test_having VALUES (4, 3, 'BBBB', 'e'); +INSERT INTO test_having VALUES (5, 3, 'bbbb', 'F'); +INSERT INTO test_having VALUES (6, 4, 'cccc', 'g'); +INSERT INTO test_having VALUES (7, 4, 'cccc', 'h'); +INSERT INTO test_having VALUES (8, 4, 'CCCC', 'I'); +INSERT INTO test_having VALUES (9, 4, 'CCCC', 'j'); + +SELECT b, c FROM test_having + GROUP BY b, c HAVING count(*) = 1 ORDER BY b, c; + +-- HAVING is effectively equivalent to WHERE in this case +SELECT b, c FROM test_having + GROUP BY b, c HAVING b = 3 ORDER BY b, c; + +-- [SPARK-28386] Cannot resolve ORDER BY columns with GROUP BY and HAVING +-- SELECT lower(c), count(c) FROM test_having +-- GROUP BY lower(c) HAVING count(*) > 2 OR min(a) = max(a) +-- ORDER BY lower(c); + +SELECT c, max(a) FROM test_having + GROUP BY c HAVING count(*) > 2 OR min(a) = max(a) + ORDER BY c; + +-- test degenerate cases involving HAVING without GROUP BY +-- Per SQL spec, these should generate 0 or 1 row, even without aggregates + +SELECT min(a), max(a) FROM test_having HAVING min(a) = max(a); +SELECT min(a), max(a) FROM test_having HAVING min(a) < max(a); + +-- errors: ungrouped column references +SELECT a FROM test_having HAVING min(a) < max(a); +SELECT 1 AS one FROM test_having HAVING a > 1; + +-- the really degenerate case: need not scan table at all +SELECT 1 AS one FROM test_having HAVING 1 > 2; +SELECT 1 AS one FROM test_having HAVING 1 < 2; + +-- [SPARK-33008] Spark SQL throws an exception +-- and just to prove that we aren't scanning the table: +SELECT 1 AS one FROM test_having WHERE 1/a = 1 HAVING 1 < 2; + +DROP TABLE test_having; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/select_implicit.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/select_implicit.sql new file mode 100644 index 000000000000..54b3083a9f4a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/select_implicit.sql @@ -0,0 +1,160 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- SELECT_IMPLICIT +-- Test cases for queries with ordering terms missing from the target list. +-- This used to be called "junkfilter.sql". +-- The parser uses the term "resjunk" to handle these cases. +-- - thomas 1998-07-09 +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_implicit.sql +-- + +-- load test data +CREATE TABLE test_missing_target (a int, b int, c string, d string) using parquet; +INSERT INTO test_missing_target VALUES (0, 1, 'XXXX', 'A'); +INSERT INTO test_missing_target VALUES (1, 2, 'ABAB', 'b'); +INSERT INTO test_missing_target VALUES (2, 2, 'ABAB', 'c'); +INSERT INTO test_missing_target VALUES (3, 3, 'BBBB', 'D'); +INSERT INTO test_missing_target VALUES (4, 3, 'BBBB', 'e'); +INSERT INTO test_missing_target VALUES (5, 3, 'bbbb', 'F'); +INSERT INTO test_missing_target VALUES (6, 4, 'cccc', 'g'); +INSERT INTO test_missing_target VALUES (7, 4, 'cccc', 'h'); +INSERT INTO test_missing_target VALUES (8, 4, 'CCCC', 'I'); +INSERT INTO test_missing_target VALUES (9, 4, 'CCCC', 'j'); + + +-- w/ existing GROUP BY target +SELECT c, count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c; + +-- w/o existing GROUP BY target using a relation name in GROUP BY clause +SELECT count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c; + +-- w/o existing GROUP BY target and w/o existing a different ORDER BY target +-- failure expected +SELECT count(*) FROM test_missing_target GROUP BY a ORDER BY b; + +-- w/o existing GROUP BY target and w/o existing same ORDER BY target +SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b; + +-- w/ existing GROUP BY target using a relation name in target +SELECT test_missing_target.b, count(*) + FROM test_missing_target GROUP BY b ORDER BY b; + +-- w/o existing GROUP BY target +SELECT c FROM test_missing_target ORDER BY a; + +-- w/o existing ORDER BY target +SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b desc; + +-- group using reference number +SELECT count(*) FROM test_missing_target ORDER BY 1 desc; + +-- order using reference number +SELECT c, count(*) FROM test_missing_target GROUP BY 1 ORDER BY 1; + +-- group using reference number out of range +-- failure expected +SELECT c, count(*) FROM test_missing_target GROUP BY 3; + +-- group w/o existing GROUP BY and ORDER BY target under ambiguous condition +-- failure expected +SELECT count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY b ORDER BY b; + +-- order w/ target under ambiguous condition +-- failure NOT expected +SELECT a, a FROM test_missing_target + ORDER BY a; + +-- order expression w/ target under ambiguous condition +-- failure NOT expected +SELECT a/2, a/2 FROM test_missing_target + ORDER BY a/2; + +-- group expression w/ target under ambiguous condition +-- failure NOT expected +SELECT a/2, a/2 FROM test_missing_target + GROUP BY a/2 ORDER BY a/2; + +-- group w/ existing GROUP BY target under ambiguous condition +SELECT x.b, count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b ORDER BY x.b; + +-- group w/o existing GROUP BY target under ambiguous condition +SELECT count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b ORDER BY x.b; + +-- [SPARK-28329] SELECT INTO syntax +-- group w/o existing GROUP BY target under ambiguous condition +-- into a table +-- SELECT count(*) INTO TABLE test_missing_target2 +-- FROM test_missing_target x, test_missing_target y +-- WHERE x.a = y.a +-- GROUP BY x.b ORDER BY x.b; +-- SELECT * FROM test_missing_target2; + + +-- Functions and expressions + +-- w/ existing GROUP BY target +SELECT a%2, count(b) FROM test_missing_target +GROUP BY test_missing_target.a%2 +ORDER BY test_missing_target.a%2; + +-- w/o existing GROUP BY target using a relation name in GROUP BY clause +SELECT count(c) FROM test_missing_target +GROUP BY lower(test_missing_target.c) +ORDER BY lower(test_missing_target.c); + +-- w/o existing GROUP BY target and w/o existing a different ORDER BY target +-- failure expected +SELECT count(a) FROM test_missing_target GROUP BY a ORDER BY b; + +-- w/o existing GROUP BY target and w/o existing same ORDER BY target +SELECT count(b) FROM test_missing_target GROUP BY b/2 ORDER BY b/2; + +-- w/ existing GROUP BY target using a relation name in target +SELECT lower(test_missing_target.c), count(c) + FROM test_missing_target GROUP BY lower(c) ORDER BY lower(c); + +-- w/o existing GROUP BY target +SELECT a FROM test_missing_target ORDER BY upper(d); + +-- w/o existing ORDER BY target +SELECT count(b) FROM test_missing_target + GROUP BY (b + 1) / 2 ORDER BY (b + 1) / 2 desc; + +-- group w/o existing GROUP BY and ORDER BY target under ambiguous condition +-- failure expected +SELECT count(x.a) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY b/2 ORDER BY b/2; + +-- group w/ existing GROUP BY target under ambiguous condition +SELECT x.b/2, count(x.b) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b/2 ORDER BY x.b/2; + +-- group w/o existing GROUP BY target under ambiguous condition +-- failure expected due to ambiguous b in count(b) +SELECT count(b) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b/2; + +-- [SPARK-28329] SELECT INTO syntax +-- group w/o existing GROUP BY target under ambiguous condition +-- into a table +-- SELECT count(x.b) INTO TABLE test_missing_target3 +-- FROM test_missing_target x, test_missing_target y +-- WHERE x.a = y.a +-- GROUP BY x.b/2 ORDER BY x.b/2; +-- SELECT * FROM test_missing_target3; + +-- Cleanup +DROP TABLE test_missing_target; +-- DROP TABLE test_missing_target2; +-- DROP TABLE test_missing_target3; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/strings.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/strings.sql new file mode 100644 index 000000000000..c252db7d3a8a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/strings.sql @@ -0,0 +1,659 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- STRINGS +-- -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/strings.sql +-- Test various data entry syntaxes. +-- + +-- SQL string continuation syntax +-- E021-03 character string literals +SELECT 'first line' +' - next line' + ' - third line' + AS `Three lines to one`; + +-- Spark SQL support this string continuation syntax +-- illegal string continuation syntax +SELECT 'first line' +' - next line' /* this comment is not allowed here */ +' - third line' + AS `Illegal comment within continuation`; + +-- [SPARK-28447] ANSI SQL: Unicode escapes in literals +-- Unicode escapes +-- SET standard_conforming_strings TO on; + +-- SELECT U&'d\0061t\+000061' AS U&"d\0061t\+000061"; +-- SELECT U&'d!0061t\+000061' UESCAPE '!' AS U&"d*0061t\+000061" UESCAPE '*'; + +-- SELECT U&' \' UESCAPE '!' AS "tricky"; +-- SELECT 'tricky' AS U&"\" UESCAPE '!'; + +-- SELECT U&'wrong: \061'; +-- SELECT U&'wrong: \+0061'; +-- SELECT U&'wrong: +0061' UESCAPE '+'; + +-- SET standard_conforming_strings TO off; + +-- SELECT U&'d\0061t\+000061' AS U&"d\0061t\+000061"; +-- SELECT U&'d!0061t\+000061' UESCAPE '!' AS U&"d*0061t\+000061" UESCAPE '*'; + +-- SELECT U&' \' UESCAPE '!' AS "tricky"; +-- SELECT 'tricky' AS U&"\" UESCAPE '!'; + +-- SELECT U&'wrong: \061'; +-- SELECT U&'wrong: \+0061'; +-- SELECT U&'wrong: +0061' UESCAPE '+'; + +-- RESET standard_conforming_strings; + +-- Spark SQL only support escape mode +-- bytea +-- SET bytea_output TO hex; +-- SELECT E'\\xDeAdBeEf'::bytea; +-- SELECT E'\\x De Ad Be Ef '::bytea; +-- SELECT E'\\xDeAdBeE'::bytea; +-- SELECT E'\\xDeAdBeEx'::bytea; +-- SELECT E'\\xDe00BeEf'::bytea; +-- SELECT E'DeAdBeEf'::bytea; +-- SELECT E'De\\000dBeEf'::bytea; +-- SELECT E'De\123dBeEf'::bytea; +-- SELECT E'De\\123dBeEf'::bytea; +-- SELECT E'De\\678dBeEf'::bytea; + +-- SET bytea_output TO escape; +SELECT binary('\\xDeAdBeEf'); +SELECT binary('\\x De Ad Be Ef '); +SELECT binary('\\xDe00BeEf'); +SELECT binary('DeAdBeEf'); +SELECT binary('De\\000dBeEf'); +SELECT binary('De\\123dBeEf'); + +-- Skip these tests because we do not have char/varchar type +-- +-- test conversions between various string types +-- E021-10 implicit casting among the character data types +-- + +-- SELECT CAST(f1 AS text) AS "text(char)" FROM CHAR_TBL; + +-- SELECT CAST(f1 AS text) AS "text(varchar)" FROM VARCHAR_TBL; + +-- SELECT CAST(name 'namefield' AS text) AS "text(name)"; + +-- since this is an explicit cast, it should truncate w/o error: +-- SELECT CAST(f1 AS char(10)) AS "char(text)" FROM TEXT_TBL; +-- note: implicit-cast case is tested in char.sql + +-- SELECT CAST(f1 AS char(20)) AS "char(text)" FROM TEXT_TBL; + +-- SELECT CAST(f1 AS char(10)) AS "char(varchar)" FROM VARCHAR_TBL; + +-- SELECT CAST(name 'namefield' AS char(10)) AS "char(name)"; + +-- SELECT CAST(f1 AS varchar) AS "varchar(text)" FROM TEXT_TBL; + +-- SELECT CAST(f1 AS varchar) AS "varchar(char)" FROM CHAR_TBL; + +-- SELECT CAST(name 'namefield' AS varchar) AS "varchar(name)"; + +-- +-- test SQL string functions +-- E### and T### are feature reference numbers from SQL99 +-- + +-- E021-09 trim function +SELECT TRIM(BOTH FROM ' bunch o blanks ') = 'bunch o blanks' AS `bunch o blanks`; + +SELECT TRIM(LEADING FROM ' bunch o blanks ') = 'bunch o blanks ' AS `bunch o blanks `; + +SELECT TRIM(TRAILING FROM ' bunch o blanks ') = ' bunch o blanks' AS ` bunch o blanks`; + +SELECT TRIM(BOTH 'x' FROM 'xxxxxsome Xsxxxxx') = 'some Xs' AS `some Xs`; + +-- E021-06 substring expression +SELECT SUBSTRING('1234567890' FROM 3) = '34567890' AS `34567890`; + +SELECT SUBSTRING('1234567890' FROM 4 FOR 3) = '456' AS `456`; + +-- [SPARK-28076] Support regular expression substring +-- T581 regular expression substring (with SQL's bizarre regexp syntax) +-- SELECT SUBSTRING('abcdefg' FROM 'a#"(b_d)#"%' FOR '#') AS "bcd"; + +-- No match should return NULL +-- SELECT SUBSTRING('abcdefg' FROM '#"(b_d)#"%' FOR '#') IS NULL AS "True"; + +-- Null inputs should return NULL +-- SELECT SUBSTRING('abcdefg' FROM '%' FOR NULL) IS NULL AS "True"; +-- SELECT SUBSTRING(NULL FROM '%' FOR '#') IS NULL AS "True"; +-- SELECT SUBSTRING('abcdefg' FROM NULL FOR '#') IS NULL AS "True"; + +-- The first and last parts should act non-greedy +-- SELECT SUBSTRING('abcdefg' FROM 'a#"%#"g' FOR '#') AS "bcdef"; +-- SELECT SUBSTRING('abcdefg' FROM 'a*#"%#"g*' FOR '#') AS "abcdefg"; + +-- Vertical bar in any part affects only that part +-- SELECT SUBSTRING('abcdefg' FROM 'a|b#"%#"g' FOR '#') AS "bcdef"; +-- SELECT SUBSTRING('abcdefg' FROM 'a#"%#"x|g' FOR '#') AS "bcdef"; +-- SELECT SUBSTRING('abcdefg' FROM 'a#"%|ab#"g' FOR '#') AS "bcdef"; + +-- Can't have more than two part separators +-- SELECT SUBSTRING('abcdefg' FROM 'a*#"%#"g*#"x' FOR '#') AS "error"; + +-- Postgres extension: with 0 or 1 separator, assume parts 1 and 3 are empty +-- SELECT SUBSTRING('abcdefg' FROM 'a#"%g' FOR '#') AS "bcdefg"; +-- SELECT SUBSTRING('abcdefg' FROM 'a%g' FOR '#') AS "abcdefg"; + +-- substring() with just two arguments is not allowed by SQL spec; +-- we accept it, but we interpret the pattern as a POSIX regexp not SQL +-- SELECT SUBSTRING('abcdefg' FROM 'c.e') AS "cde"; + +-- With a parenthesized subexpression, return only what matches the subexpr +-- SELECT SUBSTRING('abcdefg' FROM 'b(.*)f') AS "cde"; + +-- [SPARK-27952] String Functions: regexp_replace is not compatible +-- PostgreSQL extension to allow using back reference in replace string; +-- SELECT regexp_replace('1112223333', E'(\\d{3})(\\d{3})(\\d{4})', E'(\\1) \\2-\\3'); +-- SELECT regexp_replace('AAA BBB CCC ', E'\\s+', ' ', 'g'); +-- SELECT regexp_replace('AAA', '^|$', 'Z', 'g'); +-- SELECT regexp_replace('AAA aaa', 'A+', 'Z', 'gi'); +-- invalid regexp option +-- SELECT regexp_replace('AAA aaa', 'A+', 'Z', 'z'); + +-- set so we can tell NULL from empty string +-- \pset null '\\N' + +-- [SPARK-28078] Add support other 4 REGEXP functions +-- return all matches from regexp +-- SELECT regexp_matches('foobarbequebaz', $re$(bar)(beque)$re$); + +-- test case insensitive +-- SELECT regexp_matches('foObARbEqUEbAz', $re$(bar)(beque)$re$, 'i'); + +-- global option - more than one match +-- SELECT regexp_matches('foobarbequebazilbarfbonk', $re$(b[^b]+)(b[^b]+)$re$, 'g'); + +-- empty capture group (matched empty string) +-- SELECT regexp_matches('foobarbequebaz', $re$(bar)(.*)(beque)$re$); +-- no match +-- SELECT regexp_matches('foobarbequebaz', $re$(bar)(.+)(beque)$re$); +-- optional capture group did not match, null entry in array +-- SELECT regexp_matches('foobarbequebaz', $re$(bar)(.+)?(beque)$re$); + +-- no capture groups +-- SELECT regexp_matches('foobarbequebaz', $re$barbeque$re$); + +-- start/end-of-line matches are of zero length +-- SELECT regexp_matches('foo' || chr(10) || 'bar' || chr(10) || 'bequq' || chr(10) || 'baz', '^', 'mg'); +-- SELECT regexp_matches('foo' || chr(10) || 'bar' || chr(10) || 'bequq' || chr(10) || 'baz', '$', 'mg'); +-- SELECT regexp_matches('1' || chr(10) || '2' || chr(10) || '3' || chr(10) || '4' || chr(10), '^.?', 'mg'); +-- SELECT regexp_matches(chr(10) || '1' || chr(10) || '2' || chr(10) || '3' || chr(10) || '4' || chr(10), '.?$', 'mg'); +-- SELECT regexp_matches(chr(10) || '1' || chr(10) || '2' || chr(10) || '3' || chr(10) || '4', '.?$', 'mg'); + +-- give me errors +-- SELECT regexp_matches('foobarbequebaz', $re$(bar)(beque)$re$, 'gz'); +-- SELECT regexp_matches('foobarbequebaz', $re$(barbeque$re$); +-- SELECT regexp_matches('foobarbequebaz', $re$(bar)(beque){2,1}$re$); + +-- split string on regexp +-- SELECT foo, length(foo) FROM regexp_split_to_table('the quick brown fox jumps over the lazy dog', $re$\s+$re$) AS foo; +-- SELECT regexp_split_to_array('the quick brown fox jumps over the lazy dog', $re$\s+$re$); + +-- SELECT foo, length(foo) FROM regexp_split_to_table('the quick brown fox jumps over the lazy dog', $re$\s*$re$) AS foo; +-- SELECT regexp_split_to_array('the quick brown fox jumps over the lazy dog', $re$\s*$re$); +-- SELECT foo, length(foo) FROM regexp_split_to_table('the quick brown fox jumps over the lazy dog', '') AS foo; +-- SELECT regexp_split_to_array('the quick brown fox jumps over the lazy dog', ''); +-- case insensitive +-- SELECT foo, length(foo) FROM regexp_split_to_table('thE QUick bROWn FOx jUMPs ovEr The lazy dOG', 'e', 'i') AS foo; +-- -- SELECT regexp_split_to_array('thE QUick bROWn FOx jUMPs ovEr The lazy dOG', 'e', 'i'); +-- no match of pattern +-- SELECT foo, length(foo) FROM regexp_split_to_table('the quick brown fox jumps over the lazy dog', 'nomatch') AS foo; +-- SELECT regexp_split_to_array('the quick brown fox jumps over the lazy dog', 'nomatch'); +-- some corner cases +-- SELECT regexp_split_to_array('123456','1'); +-- SELECT regexp_split_to_array('123456','6'); +-- SELECT regexp_split_to_array('123456','.'); +-- SELECT regexp_split_to_array('123456',''); +-- SELECT regexp_split_to_array('123456','(?:)'); +-- SELECT regexp_split_to_array('1',''); +-- errors +-- SELECT foo, length(foo) FROM regexp_split_to_table('thE QUick bROWn FOx jUMPs ovEr The lazy dOG', 'e', 'zippy') AS foo; +-- SELECT regexp_split_to_array('thE QUick bROWn FOx jUMPs ovEr The lazy dOG', 'e', 'iz'); +-- global option meaningless for regexp_split +-- SELECT foo, length(foo) FROM regexp_split_to_table('thE QUick bROWn FOx jUMPs ovEr The lazy dOG', 'e', 'g') AS foo; +-- SELECT regexp_split_to_array('thE QUick bROWn FOx jUMPs ovEr The lazy dOG', 'e', 'g'); + +-- change NULL-display back +-- \pset null '' + +-- E021-11 position expression +SELECT POSITION('4' IN '1234567890') = '4' AS `4`; + +SELECT POSITION('5' IN '1234567890') = '5' AS `5`; + +-- [SPARK-28077] Add support string functions: OVERLAY +-- T312 character overlay function +SELECT OVERLAY('abcdef' PLACING '45' FROM 4) AS `abc45f`; + +SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5) AS `yabadaba`; + +SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5 FOR 0) AS `yabadabadoo`; + +SELECT OVERLAY('babosa' PLACING 'ubb' FROM 2 FOR 4) AS `bubba`; + +-- +-- test LIKE +-- Be sure to form every test as a LIKE/NOT LIKE pair. +-- + +-- simplest examples +-- E061-04 like predicate +SELECT 'hawkeye' LIKE 'h%' AS `true`; +SELECT 'hawkeye' NOT LIKE 'h%' AS `false`; + +SELECT 'hawkeye' LIKE 'H%' AS `false`; +SELECT 'hawkeye' NOT LIKE 'H%' AS `true`; + +SELECT 'hawkeye' LIKE 'indio%' AS `false`; +SELECT 'hawkeye' NOT LIKE 'indio%' AS `true`; + +SELECT 'hawkeye' LIKE 'h%eye' AS `true`; +SELECT 'hawkeye' NOT LIKE 'h%eye' AS `false`; + +SELECT 'indio' LIKE '_ndio' AS `true`; +SELECT 'indio' NOT LIKE '_ndio' AS `false`; + +SELECT 'indio' LIKE 'in__o' AS `true`; +SELECT 'indio' NOT LIKE 'in__o' AS `false`; + +SELECT 'indio' LIKE 'in_o' AS `false`; +SELECT 'indio' NOT LIKE 'in_o' AS `true`; + +-- unused escape character +SELECT 'hawkeye' LIKE 'h%' ESCAPE '#' AS `true`; +SELECT 'hawkeye' NOT LIKE 'h%' ESCAPE '#' AS `false`; + +SELECT 'indio' LIKE 'ind_o' ESCAPE '$' AS `true`; +SELECT 'indio' NOT LIKE 'ind_o' ESCAPE '$' AS `false`; + +-- escape character +-- E061-05 like predicate with escape clause +SELECT 'h%' LIKE 'h#%' ESCAPE '#' AS `true`; +SELECT 'h%' NOT LIKE 'h#%' ESCAPE '#' AS `false`; + +SELECT 'h%wkeye' LIKE 'h#%' ESCAPE '#' AS `false`; +SELECT 'h%wkeye' NOT LIKE 'h#%' ESCAPE '#' AS `true`; + +SELECT 'h%wkeye' LIKE 'h#%%' ESCAPE '#' AS `true`; +SELECT 'h%wkeye' NOT LIKE 'h#%%' ESCAPE '#' AS `false`; + +SELECT 'h%awkeye' LIKE 'h#%a%k%e' ESCAPE '#' AS `true`; +SELECT 'h%awkeye' NOT LIKE 'h#%a%k%e' ESCAPE '#' AS `false`; + +SELECT 'indio' LIKE '_ndio' ESCAPE '$' AS `true`; +SELECT 'indio' NOT LIKE '_ndio' ESCAPE '$' AS `false`; + +SELECT 'i_dio' LIKE 'i$_d_o' ESCAPE '$' AS `true`; +SELECT 'i_dio' NOT LIKE 'i$_d_o' ESCAPE '$' AS `false`; + +SELECT 'i_dio' LIKE 'i$_nd_o' ESCAPE '$' AS `false`; +SELECT 'i_dio' NOT LIKE 'i$_nd_o' ESCAPE '$' AS `true`; + +SELECT 'i_dio' LIKE 'i$_d%o' ESCAPE '$' AS `true`; +SELECT 'i_dio' NOT LIKE 'i$_d%o' ESCAPE '$' AS `false`; + +-- escape character same as pattern character +SELECT 'maca' LIKE 'm%aca' ESCAPE '%' AS `true`; +SELECT 'maca' NOT LIKE 'm%aca' ESCAPE '%' AS `false`; + +SELECT 'ma%a' LIKE 'm%a%%a' ESCAPE '%' AS `true`; +SELECT 'ma%a' NOT LIKE 'm%a%%a' ESCAPE '%' AS `false`; + +SELECT 'bear' LIKE 'b_ear' ESCAPE '_' AS `true`; +SELECT 'bear' NOT LIKE 'b_ear' ESCAPE '_' AS `false`; + +SELECT 'be_r' LIKE 'b_e__r' ESCAPE '_' AS `true`; +SELECT 'be_r' NOT LIKE 'b_e__r' ESCAPE '_' AS `false`; + +SELECT 'be_r' LIKE '__e__r' ESCAPE '_' AS `false`; +SELECT 'be_r' NOT LIKE '__e__r' ESCAPE '_' AS `true`; + +-- [SPARK-28448] Implement ILIKE operator +-- +-- test ILIKE (case-insensitive LIKE) +-- Be sure to form every test as an ILIKE/NOT ILIKE pair. +-- + +-- SELECT 'hawkeye' ILIKE 'h%' AS "true"; +-- SELECT 'hawkeye' NOT ILIKE 'h%' AS "false"; + +-- SELECT 'hawkeye' ILIKE 'H%' AS "true"; +-- SELECT 'hawkeye' NOT ILIKE 'H%' AS "false"; + +-- SELECT 'hawkeye' ILIKE 'H%Eye' AS "true"; +-- SELECT 'hawkeye' NOT ILIKE 'H%Eye' AS "false"; + +-- SELECT 'Hawkeye' ILIKE 'h%' AS "true"; +-- SELECT 'Hawkeye' NOT ILIKE 'h%' AS "false"; + +-- +-- test %/_ combination cases, cf bugs #4821 and #5478 +-- + +SELECT 'foo' LIKE '_%' as t, 'f' LIKE '_%' as t, '' LIKE '_%' as f; +SELECT 'foo' LIKE '%_' as t, 'f' LIKE '%_' as t, '' LIKE '%_' as f; + +SELECT 'foo' LIKE '__%' as t, 'foo' LIKE '___%' as t, 'foo' LIKE '____%' as f; +SELECT 'foo' LIKE '%__' as t, 'foo' LIKE '%___' as t, 'foo' LIKE '%____' as f; + +SELECT 'jack' LIKE '%____%' AS t; + + +-- Skip tests of LIKE with indexes +-- +-- basic tests of LIKE with indexes +-- + +-- CREATE TABLE texttest (a text PRIMARY KEY, b int); +-- SELECT * FROM texttest WHERE a LIKE '%1%'; + +-- CREATE TABLE byteatest (a bytea PRIMARY KEY, b int); +-- SELECT * FROM byteatest WHERE a LIKE '%1%'; + +-- DROP TABLE texttest, byteatest; + + +-- +-- test implicit type conversion +-- + +-- E021-07 character concatenation +SELECT 'unknown' || ' and unknown' AS `Concat unknown types`; + +SELECT string('text') || ' and unknown' AS `Concat text to unknown type`; + +-- Spark SQL does not have char and varchar type +-- SELECT char(20) 'characters' || ' and text' AS "Concat char to unknown type"; + +-- SELECT text 'text' || char(20) ' and characters' AS "Concat text to char"; + +-- SELECT text 'text' || varchar ' and varchar' AS "Concat text to varchar"; + +-- +-- test substr with toasted text values +-- +CREATE TABLE toasttest(f1 string) USING parquet; + +insert into toasttest values(repeat('1234567890',10000)); +insert into toasttest values(repeat('1234567890',10000)); + +-- +-- Ensure that some values are uncompressed, to test the faster substring +-- operation used in that case +-- +-- alter table toasttest alter column f1 set storage external; +insert into toasttest values(repeat('1234567890',10000)); +insert into toasttest values(repeat('1234567890',10000)); + +-- [SPARK-28451] substr returns different values +-- If the starting position is zero or less, then return from the start of the string +-- adjusting the length to be consistent with the "negative start" per SQL. +-- SELECT substr(f1, -1, 5) from toasttest; + +-- If the length is less than zero, an ERROR is thrown. +-- SELECT substr(f1, 5, -1) from toasttest; + +-- If no third argument (length) is provided, the length to the end of the +-- string is assumed. +SELECT substr(f1, 99995) from toasttest; + +-- If start plus length is > string length, the result is truncated to +-- string length +SELECT substr(f1, 99995, 10) from toasttest; + +-- Skip these tests +-- TRUNCATE TABLE toasttest; +-- INSERT INTO toasttest values (repeat('1234567890',300)); +-- INSERT INTO toasttest values (repeat('1234567890',300)); +-- INSERT INTO toasttest values (repeat('1234567890',300)); +-- INSERT INTO toasttest values (repeat('1234567890',300)); +-- expect >0 blocks +-- SELECT pg_relation_size(reltoastrelid) = 0 AS is_empty +-- FROM pg_class where relname = 'toasttest'; + +-- TRUNCATE TABLE toasttest; +-- ALTER TABLE toasttest set (toast_tuple_target = 4080); +-- INSERT INTO toasttest values (repeat('1234567890',300)); +-- INSERT INTO toasttest values (repeat('1234567890',300)); +-- INSERT INTO toasttest values (repeat('1234567890',300)); +-- INSERT INTO toasttest values (repeat('1234567890',300)); +-- expect 0 blocks +-- SELECT pg_relation_size(reltoastrelid) = 0 AS is_empty +-- FROM pg_class where relname = 'toasttest'; + +-- DROP TABLE toasttest; + +-- [SPARK-28121] decode can not accept 'escape' as charset +-- +-- +-- test substr with toasted bytea values +-- +-- CREATE TABLE toasttest(f1 binary) USING parquet; + +-- insert into toasttest values(decode(repeat('1234567890',10000),'escape')); +-- insert into toasttest values(decode(repeat('1234567890',10000),'escape')); + +-- +-- Ensure that some values are uncompressed, to test the faster substring +-- operation used in that case +-- +-- alter table toasttest alter column f1 set storage external; +-- insert into toasttest values(decode(repeat('1234567890',10000),'escape')); +-- insert into toasttest values(decode(repeat('1234567890',10000),'escape')); + +-- If the starting position is zero or less, then return from the start of the string +-- adjusting the length to be consistent with the "negative start" per SQL. +-- SELECT substr(f1, -1, 5) from toasttest; + +-- If the length is less than zero, an ERROR is thrown. +-- SELECT substr(f1, 5, -1) from toasttest; + +-- If no third argument (length) is provided, the length to the end of the +-- string is assumed. +-- SELECT substr(f1, 99995) from toasttest; + +-- If start plus length is > string length, the result is truncated to +-- string length +-- SELECT substr(f1, 99995, 10) from toasttest; + +-- DROP TABLE toasttest; + +-- Skip these tests because we do not support char type +-- test internally compressing datums + +-- this tests compressing a datum to a very small size which exercises a +-- corner case in packed-varlena handling: even though small, the compressed +-- datum must be given a 4-byte header because there are no bits to indicate +-- compression in a 1-byte header + +-- CREATE TABLE toasttest (c char(4096)); +-- INSERT INTO toasttest VALUES('x'); +-- SELECT length(c), c::text FROM toasttest; +-- SELECT c FROM toasttest; +-- DROP TABLE toasttest; + +-- +-- test length +-- + +SELECT length('abcdef') AS `length_6`; + +-- [SPARK-27930] Replace strpos with locate or position in Spark SQL +-- +-- test strpos +-- + +SELECT position('cd', 'abcdef') AS `pos_3`; + +SELECT position('xy', 'abcdef') AS `pos_0`; + +-- +-- test replace +-- +SELECT replace('abcdef', 'de', '45') AS `abc45f`; + +SELECT replace('yabadabadoo', 'ba', '123') AS `ya123da123doo`; + +SELECT replace('yabadoo', 'bad', '') AS `yaoo`; + +-- [SPARK-28087] Add support split_part +-- +-- test split_part +-- +-- select split_part('joeuser@mydatabase','@',0) AS "an error"; + +-- select split_part('joeuser@mydatabase','@',1) AS "joeuser"; + +-- select split_part('joeuser@mydatabase','@',2) AS "mydatabase"; + +-- select split_part('joeuser@mydatabase','@',3) AS "empty string"; + +-- select split_part('@joeuser@mydatabase@','@',2) AS "joeuser"; + +-- [SPARK-27930] Spark SQL use hex +-- +-- test to_hex +-- +select hex(256*256*256 - 1) AS `ffffff`; + +select hex(bigint(bigint(bigint(bigint(256)*256)*256)*256) - 1) AS `ffffffff`; + +-- +-- MD5 test suite - from IETF RFC 1321 +-- (see: ftp://ftp.rfc-editor.org/in-notes/rfc1321.txt) +-- +select md5('') = 'd41d8cd98f00b204e9800998ecf8427e' AS `TRUE`; + +select md5('a') = '0cc175b9c0f1b6a831c399e269772661' AS `TRUE`; + +select md5('abc') = '900150983cd24fb0d6963f7d28e17f72' AS `TRUE`; + +select md5('message digest') = 'f96b697d7cb7938d525a2f31aaf161d0' AS `TRUE`; + +select md5('abcdefghijklmnopqrstuvwxyz') = 'c3fcd3d76192e4007dfb496cca67e13b' AS `TRUE`; + +select md5('ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789') = 'd174ab98d277d9f5a5611c2c9f419d9f' AS `TRUE`; + +select md5('12345678901234567890123456789012345678901234567890123456789012345678901234567890') = '57edf4a22be3c955ac49da2e2107b67a' AS `TRUE`; + +select md5(binary('')) = 'd41d8cd98f00b204e9800998ecf8427e' AS `TRUE`; + +select md5(binary('a')) = '0cc175b9c0f1b6a831c399e269772661' AS `TRUE`; + +select md5(binary('abc')) = '900150983cd24fb0d6963f7d28e17f72' AS `TRUE`; + +select md5(binary('message digest')) = 'f96b697d7cb7938d525a2f31aaf161d0' AS `TRUE`; + +select md5(binary('abcdefghijklmnopqrstuvwxyz')) = 'c3fcd3d76192e4007dfb496cca67e13b' AS `TRUE`; + +select md5(binary('ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789')) = 'd174ab98d277d9f5a5611c2c9f419d9f' AS `TRUE`; + +select md5(binary('12345678901234567890123456789012345678901234567890123456789012345678901234567890')) = '57edf4a22be3c955ac49da2e2107b67a' AS `TRUE`; + +-- [SPARK-28122] missing SHA functions +-- +-- SHA-2 +-- +-- SET bytea_output TO hex; + +-- SELECT sha224(''); +-- SELECT sha224('The quick brown fox jumps over the lazy dog.'); + +-- SELECT sha256(''); +-- SELECT sha256('The quick brown fox jumps over the lazy dog.'); + +-- SELECT sha384(''); +-- SELECT sha384('The quick brown fox jumps over the lazy dog.'); + +-- SELECT sha512(''); +-- SELECT sha512('The quick brown fox jumps over the lazy dog.'); + +-- [SPARK-28449] Missing escape_string_warning and standard_conforming_strings config +-- +-- test behavior of escape_string_warning and standard_conforming_strings options +-- +-- set escape_string_warning = off; +-- set standard_conforming_strings = off; + +-- show escape_string_warning; +-- show standard_conforming_strings; + +-- set escape_string_warning = on; +-- set standard_conforming_strings = on; + +-- show escape_string_warning; +-- show standard_conforming_strings; + +-- select 'a\bcd' as f1, 'a\b''cd' as f2, 'a\b''''cd' as f3, 'abcd\' as f4, 'ab\''cd' as f5, '\\' as f6; + +-- set standard_conforming_strings = off; + +-- select 'a\\bcd' as f1, 'a\\b\'cd' as f2, 'a\\b\'''cd' as f3, 'abcd\\' as f4, 'ab\\\'cd' as f5, '\\\\' as f6; + +-- set escape_string_warning = off; +-- set standard_conforming_strings = on; + +-- select 'a\bcd' as f1, 'a\b''cd' as f2, 'a\b''''cd' as f3, 'abcd\' as f4, 'ab\''cd' as f5, '\\' as f6; + +-- set standard_conforming_strings = off; + +-- select 'a\\bcd' as f1, 'a\\b\'cd' as f2, 'a\\b\'''cd' as f3, 'abcd\\' as f4, 'ab\\\'cd' as f5, '\\\\' as f6; + + +-- +-- Additional string functions +-- +-- SET bytea_output TO escape; + +SELECT initcap('hi THOMAS'); + +SELECT lpad('hi', 5, 'xy'); +SELECT lpad('hi', 5); +SELECT lpad('hi', -5, 'xy'); +SELECT lpad('hello', 2); +SELECT lpad('hi', 5, ''); + +SELECT rpad('hi', 5, 'xy'); +SELECT rpad('hi', 5); +SELECT rpad('hi', -5, 'xy'); +SELECT rpad('hello', 2); +SELECT rpad('hi', 5, ''); + +-- skip this test because PostgreSQL has different parameter order compares to SparkSQL +-- SELECT ltrim('zzzytrim', 'xyz'); + +SELECT translate('', '14', 'ax'); +SELECT translate('12345', '14', 'ax'); + +SELECT ascii('x'); +SELECT ascii(''); + +SELECT chr(65); +-- PostgreSQL throws: ERROR: null character not permitted +SELECT chr(0); + +SELECT repeat('Pg', 4); +SELECT repeat('Pg', -4); + +SELECT trim(binary('\\000') from binary('\\000Tom\\000')); +SELECT btrim(binary('\\000trim\\000'), binary('\\000')); +SELECT btrim(binary(''), binary('\\000')); +SELECT btrim(binary('\\000trim\\000'), binary('')); +-- [SPARK-28121] decode can not accept 'escape' as charset +-- [SPARK-28412][SQL] ANSI SQL: OVERLAY function support byte array +-- SELECT encode(overlay(E'Th\\000omas'::bytea placing E'Th\\001omas'::bytea from 2),'escape'); +-- SELECT encode(overlay(E'Th\\000omas'::bytea placing E'\\002\\003'::bytea from 8),'escape'); +-- SELECT encode(overlay(E'Th\\000omas'::bytea placing E'\\002\\003'::bytea from 5 for 3),'escape'); + +DROP TABLE toasttest; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/text.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/text.sql new file mode 100644 index 000000000000..05953123da86 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/text.sql @@ -0,0 +1,133 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- TEXT +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/text.sql + +SELECT string('this is a text string') = string('this is a text string') AS true; + +SELECT string('this is a text string') = string('this is a text strin') AS `false`; + +CREATE TABLE TEXT_TBL (f1 string) USING parquet; + +INSERT INTO TEXT_TBL VALUES ('doh!'); +INSERT INTO TEXT_TBL VALUES ('hi de ho neighbor'); + +SELECT '' AS two, * FROM TEXT_TBL; + +-- As of 8.3 we have removed most implicit casts to text, so that for example +-- this no longer works: +-- Spark SQL implicit cast integer to string +select length(42); + +-- But as a special exception for usability's sake, we still allow implicit +-- casting to text in concatenations, so long as the other input is text or +-- an unknown literal. So these work: +-- [SPARK-28033] String concatenation low priority than other arithmeticBinary +select string('four: ') || 2+2; +select 'four: ' || 2+2; + +-- but not this: +-- Spark SQL implicit cast both side to string +select 3 || 4.0; + +/* + * various string functions + */ +select concat('one'); +-- Spark SQL does not support YYYYMMDD, we replace it to yyyyMMdd +select concat(1,2,3,'hello',true, false, to_date('20100309','yyyyMMdd')); +select concat_ws('#','one'); +select concat_ws('#',1,2,3,'hello',true, false, to_date('20100309','yyyyMMdd')); +select concat_ws(',',10,20,null,30); +select concat_ws('',10,20,null,30); +select concat_ws(NULL,10,20,null,30) is null; +select reverse('abcde'); +select i, left('ahoj', i), right('ahoj', i) from range(-5, 6) t(i) order by i; +-- [SPARK-28037] Add built-in String Functions: quote_literal +-- select quote_literal(''); +-- select quote_literal('abc'''); +-- select quote_literal(e'\\'); + +-- Skip these tests because Spark does not support variadic labeled argument +-- check variadic labeled argument +-- select concat(variadic array[1,2,3]); +-- select concat_ws(',', variadic array[1,2,3]); +-- select concat_ws(',', variadic NULL::int[]); +-- select concat(variadic NULL::int[]) is NULL; +-- select concat(variadic '{}'::int[]) = ''; +--should fail +-- select concat_ws(',', variadic 10); + +-- [SPARK-27930] Replace format to format_string +/* + * format + */ +select format_string(NULL); +select format_string('Hello'); +select format_string('Hello %s', 'World'); +select format_string('Hello %%'); +select format_string('Hello %%%%'); +-- should fail +select format_string('Hello %s %s', 'World'); +select format_string('Hello %s'); +select format_string('Hello %x', 20); +-- check literal and sql identifiers +-- [SPARK-27930] format_string can not fully support PostgreSQL's format +-- select format_string('INSERT INTO %I VALUES(%L,%L)', 'mytab', 10, 'Hello'); +-- select format_string('%s%s%s','Hello', NULL,'World'); +-- select format_string('INSERT INTO %I VALUES(%L,%L)', 'mytab', 10, NULL); +-- select format_string('INSERT INTO %I VALUES(%L,%L)', 'mytab', NULL, 'Hello'); +-- should fail, sql identifier cannot be NULL +-- select format_string('INSERT INTO %I VALUES(%L,%L)', NULL, 10, 'Hello'); +-- check positional placeholders +select format_string('%1$s %3$s', 1, 2, 3); +select format_string('%1$s %12$s', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12); +-- should fail +select format_string('%1$s %4$s', 1, 2, 3); +select format_string('%1$s %13$s', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12); +--PostgreSQL throw ERROR: format specifies argument 0, but arguments are numbered from 1 +select format_string('%0$s', 'Hello'); +-- [SPARK-27930] format_string can not fully support PostgreSQL's format +-- select format_string('%*0$s', 'Hello'); +-- select format_string('%1$', 1); +-- select format_string('%1$1', 1); +-- check mix of positional and ordered placeholders +select format_string('Hello %s %1$s %s', 'World', 'Hello again'); +select format_string('Hello %s %s, %2$s %2$s', 'World', 'Hello again'); +-- Skip these tests because Spark does not support variadic labeled argument +-- check variadic labeled arguments +-- select format('%s, %s', variadic array['Hello','World']); +-- select format('%s, %s', variadic array[1, 2]); +-- select format('%s, %s', variadic array[true, false]); +-- select format('%s, %s', variadic array[true, false]::text[]); +-- check variadic with positional placeholders +-- select format('%2$s, %1$s', variadic array['first', 'second']); +-- select format('%2$s, %1$s', variadic array[1, 2]); +-- variadic argument can be array type NULL, but should not be referenced +-- select format('Hello', variadic NULL::int[]); +-- variadic argument allows simulating more than FUNC_MAX_ARGS parameters +-- select format(string_agg('%s',','), variadic array_agg(i)) +-- from generate_series(1,200) g(i); +-- check field widths and left, right alignment +select format_string('>>%10s<<', 'Hello'); +select format_string('>>%10s<<', NULL); +select format_string('>>%10s<<', ''); +select format_string('>>%-10s<<', ''); +select format_string('>>%-10s<<', 'Hello'); +select format_string('>>%-10s<<', NULL); +select format_string('>>%1$10s<<', 'Hello'); +-- [SPARK-27930] format_string can not fully support PostgreSQL's format +-- select format_string('>>%1$-10I<<', 'Hello'); +-- select format_string('>>%2$*1$L<<', 10, 'Hello'); +-- select format_string('>>%2$*1$L<<', 10, NULL); +-- select format_string('>>%2$*1$L<<', -10, NULL); +-- select format_string('>>%*s<<', 10, 'Hello'); +-- select format_string('>>%*1$s<<', 10, 'Hello'); +-- select format_string('>>%-s<<', 'Hello'); +-- select format_string('>>%10L<<', NULL); +-- select format_string('>>%2$*1$L<<', NULL, 'Hello'); +-- select format_string('>>%2$*1$L<<', 0, 'Hello'); + +DROP TABLE TEXT_TBL; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/timestamp.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/timestamp.sql new file mode 100644 index 000000000000..0630262a2953 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/timestamp.sql @@ -0,0 +1,252 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- TIMESTAMP +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/timestamp.sql + +CREATE TABLE TIMESTAMP_TBL (d1 timestamp) USING parquet; + +-- Test shorthand input values +-- We can't just "select" the results since they aren't constants; test for +-- equality instead. We can do that by running the test inside a transaction +-- block, within which the value of 'now' shouldn't change. We also check +-- that 'now' *does* change over a reasonable interval such as 100 msec. +-- NOTE: it is possible for this part of the test to fail if the transaction +-- block is entered exactly at local midnight; then 'now' and 'today' have +-- the same values and the counts will come out different. + +-- PostgreSQL implicitly casts string literals to data with timestamp types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'now'); +-- SELECT pg_sleep(0.1); + +-- BEGIN; + +-- PostgreSQL implicitly casts string literals to data with timestamp types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'now'); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'today'); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'yesterday'); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'tomorrow'); +-- time zone should be ignored by this data type +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'tomorrow EST'); +-- [SPARK-29024] Ignore case while resolving time zones +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'tomorrow Zulu'); + +SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'today'; +SELECT count(*) AS Three FROM TIMESTAMP_TBL WHERE d1 = timestamp 'tomorrow'; +SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'yesterday'; +-- [SPARK-29025] Support seconds precision by the timestamp type +-- SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp(2) 'now'; + +-- COMMIT; + +-- DELETE FROM TIMESTAMP_TBL; + +-- verify uniform transaction time within transaction block +-- BEGIN; +-- INSERT INTO TIMESTAMP_TBL VALUES ('now'); +-- SELECT pg_sleep(0.1); +-- INSERT INTO TIMESTAMP_TBL VALUES ('now'); +-- SELECT pg_sleep(0.1); +-- SELECT count(*) AS two FROM TIMESTAMP_TBL WHERE d1 = timestamp(2) 'now'; +-- COMMIT; + +TRUNCATE TABLE TIMESTAMP_TBL; + +-- Special values +-- INSERT INTO TIMESTAMP_TBL VALUES ('-infinity'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('infinity'); +-- PostgreSQL implicitly casts string literals to data with timestamp types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'epoch'); +-- [SPARK-27923] Spark SQL insert there obsolete special values to NULL +-- Obsolete special values +-- INSERT INTO TIMESTAMP_TBL VALUES ('invalid'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('undefined'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('current'); + +-- [SPARK-28259] Date/Time Output Styles and Date Order Conventions +-- Postgres v6.0 standard output format +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01 1997 PST'); + +-- Variations on Postgres v6.1 standard output format +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.000001 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.999999 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.4 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.5 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.6 1997 PST'); + +-- ISO 8601 format +-- PostgreSQL implicitly casts string literals to data with timestamp types, but +-- Spark does not support that kind of implicit casts. +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-01-02')); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-01-02 03:04:05')); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-02-10 17:32:01-08')); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01 -08:00'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 -0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 17:32:01 -07:00'); +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('2001-09-22T18:19:20')); + +-- POSIX format (note that the timezone abbrev is just decoration here) +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 08:14:01 GMT+8'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 13:14:02 GMT-1'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 12:14:03 GMT-2'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 03:14:04 PST+8'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 02:14:05 MST+7:00'); + +-- Variations for acceptable input formats +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997 -0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 5:32PM 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997/02/10 17:32:01-0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb-10-1997 17:32:01 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('02-10-1997 17:32:01 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 PST'); +-- set datestyle to ymd; +-- INSERT INTO TIMESTAMP_TBL VALUES ('97FEB10 5:32:01PM UTC'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('97/02/10 17:32:01 UTC'); +-- reset datestyle; +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997.041 17:32:01 UTC'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 America/New_York'); +-- this fails (even though TZ is a no-op, we still look it up) +-- INSERT INTO TIMESTAMP_TBL VALUES ('19970710 173201 America/Does_not_exist'); + +-- Check date conversion and date arithmetic +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 18:32:01 PDT'); + +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 11 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 12 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 13 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 14 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 15 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1997'); + +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 0097 BC'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 0097'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 0597'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1097'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1697'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1797'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1897'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 2097'); + +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 28 17:32:01 1996'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 29 17:32:01 1996'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mar 01 17:32:01 1996'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Dec 30 17:32:01 1996'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 1996'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Jan 01 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 28 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 29 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mar 01 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Dec 30 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 1999'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Jan 01 17:32:01 2000'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 2000'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Jan 01 17:32:01 2001'); + +-- Currently unsupported syntax and ranges +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 -0097'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 5097 BC'); + +SELECT '' AS `64`, d1 FROM TIMESTAMP_TBL; + +-- [SPARK-28253] Date/Timestamp type have different low value and high value with Spark +-- Check behavior at the lower boundary of the timestamp range +-- SELECT '4714-11-24 00:00:00 BC'::timestamp; +-- SELECT '4714-11-23 23:59:59 BC'::timestamp; -- out of range +-- The upper boundary differs between integer and float timestamps, so no check + +-- Demonstrate functions and operators +SELECT '' AS `48`, d1 FROM TIMESTAMP_TBL + WHERE d1 > timestamp '1997-01-02'; + +SELECT '' AS `15`, d1 FROM TIMESTAMP_TBL + WHERE d1 < timestamp '1997-01-02'; + +SELECT '' AS one, d1 FROM TIMESTAMP_TBL + WHERE d1 = timestamp '1997-01-02'; + +SELECT '' AS `63`, d1 FROM TIMESTAMP_TBL + WHERE d1 != timestamp '1997-01-02'; + +SELECT '' AS `16`, d1 FROM TIMESTAMP_TBL + WHERE d1 <= timestamp '1997-01-02'; + +SELECT '' AS `49`, d1 FROM TIMESTAMP_TBL + WHERE d1 >= timestamp '1997-01-02'; + +SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff + FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; + +SELECT '' AS date_trunc_week, date_trunc( 'week', timestamp '2004-02-29 15:44:17.71393' ) AS week_trunc; + +-- Test casting within a BETWEEN qualifier +SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff + FROM TIMESTAMP_TBL + WHERE d1 BETWEEN timestamp '1902-01-01' + AND timestamp '2038-01-01'; + +SELECT '' AS `54`, d1 as `timestamp`, + date_part( 'year', d1) AS `year`, date_part( 'month', d1) AS `month`, + date_part( 'day', d1) AS `day`, date_part( 'hour', d1) AS `hour`, + date_part( 'minute', d1) AS `minute`, date_part( 'second', d1) AS `second` + FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; + +-- SELECT '' AS `54`, d1 as `timestamp`, +-- date_part( 'quarter', d1) AS quarter, date_part( 'msec', d1) AS msec, +-- date_part( 'usec', d1) AS usec +-- FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; + +-- SELECT '' AS `54`, d1 as `timestamp`, +-- date_part( 'isoyear', d1) AS isoyear, date_part( 'week', d1) AS week, +-- date_part( 'dow', d1) AS dow +-- FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; + +-- [SPARK-28137] Data Type Formatting Functions +-- TO_CHAR() +-- SELECT '' AS to_char_1, to_char(d1, 'DAY Day day DY Dy dy MONTH Month month RM MON Mon mon') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_2, to_char(d1, 'FMDAY FMDay FMday FMMONTH FMMonth FMmonth FMRM') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_3, to_char(d1, 'Y,YYY YYYY YYY YY Y CC Q MM WW DDD DD D J') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_4, to_char(d1, 'FMY,YYY FMYYYY FMYYY FMYY FMY FMCC FMQ FMMM FMWW FMDDD FMDD FMD FMJ') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_5, to_char(d1, 'HH HH12 HH24 MI SS SSSS') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_6, to_char(d1, E'"HH:MI:SS is" HH:MI:SS "\\"text between quote marks\\""') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_7, to_char(d1, 'HH24--text--MI--text--SS') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_8, to_char(d1, 'YYYYTH YYYYth Jth') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_9, to_char(d1, 'YYYY A.D. YYYY a.d. YYYY bc HH:MI:SS P.M. HH:MI:SS p.m. HH:MI:SS pm') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_10, to_char(d1, 'IYYY IYY IY I IW IDDD ID') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_11, to_char(d1, 'FMIYYY FMIYY FMIY FMI FMIW FMIDDD FMID') +-- FROM TIMESTAMP_TBL; + + +-- timestamp numeric fields constructor +SELECT make_timestamp(2014,12,28,6,30,45.887); + +DROP TABLE TIMESTAMP_TBL; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/union.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/union.sql new file mode 100644 index 000000000000..41d8f6eb680c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/union.sql @@ -0,0 +1,472 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- UNION (also INTERSECT, EXCEPT) +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/union.sql +-- + +CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM + (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) + AS v(f1); +CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM + (VALUES + (123, 456), + (123, 4567890123456789), + (4567890123456789, 123), + (4567890123456789, 4567890123456789), + (4567890123456789, -4567890123456789)) + AS v(q1, q2); +CREATE OR REPLACE TEMPORARY VIEW FLOAT8_TBL AS SELECT * FROM + (VALUES (0.0), (-34.84), (-1004.30), + (CAST('-1.2345678901234e+200' AS DOUBLE)), (CAST('-1.2345678901234e-200' AS DOUBLE))) + AS v(f1); + +-- Simple UNION constructs + +SELECT 1 AS two UNION SELECT 2 ORDER BY 1; + +SELECT 1 AS one UNION SELECT 1 ORDER BY 1; + +SELECT 1 AS two UNION ALL SELECT 2; + +SELECT 1 AS two UNION ALL SELECT 1; + +SELECT 1 AS three UNION SELECT 2 UNION SELECT 3 ORDER BY 1; + +SELECT 1 AS two UNION SELECT 2 UNION SELECT 2 ORDER BY 1; + +SELECT 1 AS three UNION SELECT 2 UNION ALL SELECT 2 ORDER BY 1; + +SELECT 1.1 AS two UNION SELECT 2.2 ORDER BY 1; + +-- Mixed types + +SELECT 1.1 AS two UNION SELECT 2 ORDER BY 1; + +SELECT 1 AS two UNION SELECT 2.2 ORDER BY 1; + +SELECT 1 AS one UNION SELECT double(1.0) ORDER BY 1; + +SELECT 1.1 AS two UNION ALL SELECT 2 ORDER BY 1; + +SELECT double(1.0) AS two UNION ALL SELECT 1 ORDER BY 1; + +SELECT 1.1 AS three UNION SELECT 2 UNION SELECT 3 ORDER BY 1; + +SELECT double(1.1) AS two UNION SELECT 2 UNION SELECT double(2.0) ORDER BY 1; + +SELECT 1.1 AS three UNION SELECT 2 UNION ALL SELECT 2 ORDER BY 1; + +SELECT 1.1 AS two UNION (SELECT 2 UNION ALL SELECT 2) ORDER BY 1; + +-- +-- Try testing from tables... +-- + +SELECT f1 AS five FROM FLOAT8_TBL +UNION +SELECT f1 FROM FLOAT8_TBL +ORDER BY 1; + +SELECT f1 AS ten FROM FLOAT8_TBL +UNION ALL +SELECT f1 FROM FLOAT8_TBL; + +SELECT f1 AS nine FROM FLOAT8_TBL +UNION +SELECT f1 FROM INT4_TBL +ORDER BY 1; + +SELECT f1 AS ten FROM FLOAT8_TBL +UNION ALL +SELECT f1 FROM INT4_TBL; + +SELECT f1 AS five FROM FLOAT8_TBL + WHERE f1 BETWEEN -1e6 AND 1e6 +UNION +SELECT f1 FROM INT4_TBL + WHERE f1 BETWEEN 0 AND 1000000 +ORDER BY 1; + +-- [SPARK-28298] Fully support char and varchar types +-- SELECT CAST(f1 AS char(4)) AS three FROM VARCHAR_TBL +-- UNION +-- SELECT f1 FROM CHAR_TBL +-- ORDER BY 1; + +-- SELECT f1 AS three FROM VARCHAR_TBL +-- UNION +-- SELECT CAST(f1 AS varchar) FROM CHAR_TBL +-- ORDER BY 1; + +-- SELECT f1 AS eight FROM VARCHAR_TBL +-- UNION ALL +-- SELECT f1 FROM CHAR_TBL; + +-- SELECT f1 AS five FROM TEXT_TBL +-- UNION +-- SELECT f1 FROM VARCHAR_TBL +-- UNION +-- SELECT TRIM(TRAILING FROM f1) FROM CHAR_TBL +-- ORDER BY 1; + +-- +-- INTERSECT and EXCEPT +-- + +SELECT q2 FROM int8_tbl INTERSECT SELECT q1 FROM int8_tbl ORDER BY 1; + +SELECT q2 FROM int8_tbl INTERSECT ALL SELECT q1 FROM int8_tbl ORDER BY 1; + +SELECT q2 FROM int8_tbl EXCEPT SELECT q1 FROM int8_tbl ORDER BY 1; + +SELECT q2 FROM int8_tbl EXCEPT ALL SELECT q1 FROM int8_tbl ORDER BY 1; + +SELECT q2 FROM int8_tbl EXCEPT ALL SELECT DISTINCT q1 FROM int8_tbl ORDER BY 1; + +SELECT q1 FROM int8_tbl EXCEPT SELECT q2 FROM int8_tbl ORDER BY 1; + +SELECT q1 FROM int8_tbl EXCEPT ALL SELECT q2 FROM int8_tbl ORDER BY 1; + +SELECT q1 FROM int8_tbl EXCEPT ALL SELECT DISTINCT q2 FROM int8_tbl ORDER BY 1; + +-- Spark SQL do not support update +-- SELECT q1 FROM int8_tbl EXCEPT ALL SELECT q1 FROM int8_tbl FOR NO KEY UPDATE; + +-- nested cases +(SELECT 1,2,3 UNION SELECT 4,5,6) INTERSECT SELECT 4,5,6; +(SELECT 1,2,3 UNION SELECT 4,5,6 ORDER BY 1,2) INTERSECT SELECT 4,5,6; +(SELECT 1,2,3 UNION SELECT 4,5,6) EXCEPT SELECT 4,5,6; +(SELECT 1,2,3 UNION SELECT 4,5,6 ORDER BY 1,2) EXCEPT SELECT 4,5,6; + +-- exercise both hashed and sorted implementations of INTERSECT/EXCEPT + +-- set enable_hashagg to on; + +-- explain (costs off) +-- select count(*) from +-- ( select unique1 from tenk1 intersect select fivethous from tenk1 ) ss; +select count(*) from + ( select unique1 from tenk1 intersect select fivethous from tenk1 ) ss; + +-- explain (costs off) +-- select unique1 from tenk1 except select unique2 from tenk1 where unique2 != 10; +select unique1 from tenk1 except select unique2 from tenk1 where unique2 != 10; + +-- set enable_hashagg to off; + +-- explain (costs off) +-- select count(*) from +-- ( select unique1 from tenk1 intersect select fivethous from tenk1 ) ss; +select count(*) from + ( select unique1 from tenk1 intersect select fivethous from tenk1 ) ss; + +-- explain (costs off) +-- select unique1 from tenk1 except select unique2 from tenk1 where unique2 != 10; +select unique1 from tenk1 except select unique2 from tenk1 where unique2 != 10; + +-- reset enable_hashagg; + +-- +-- Mixed types +-- + +SELECT f1 FROM float8_tbl INTERSECT SELECT f1 FROM int4_tbl ORDER BY 1; + +SELECT f1 FROM float8_tbl EXCEPT SELECT f1 FROM int4_tbl ORDER BY 1; + +-- +-- Operator precedence and (((((extra))))) parentheses +-- + +SELECT q1 FROM int8_tbl INTERSECT SELECT q2 FROM int8_tbl UNION ALL SELECT q2 FROM int8_tbl ORDER BY 1; + +SELECT q1 FROM int8_tbl INTERSECT (((SELECT q2 FROM int8_tbl UNION ALL SELECT q2 FROM int8_tbl))) ORDER BY 1; + +(((SELECT q1 FROM int8_tbl INTERSECT SELECT q2 FROM int8_tbl ORDER BY 1))) UNION ALL SELECT q2 FROM int8_tbl; + +SELECT q1 FROM int8_tbl UNION ALL SELECT q2 FROM int8_tbl EXCEPT SELECT q1 FROM int8_tbl ORDER BY 1; + +SELECT q1 FROM int8_tbl UNION ALL (((SELECT q2 FROM int8_tbl EXCEPT SELECT q1 FROM int8_tbl ORDER BY 1))); + +(((SELECT q1 FROM int8_tbl UNION ALL SELECT q2 FROM int8_tbl))) EXCEPT SELECT q1 FROM int8_tbl ORDER BY 1; + +-- +-- Subqueries with ORDER BY & LIMIT clauses +-- + +-- In this syntax, ORDER BY/LIMIT apply to the result of the EXCEPT +SELECT q1,q2 FROM int8_tbl EXCEPT SELECT q2,q1 FROM int8_tbl +ORDER BY q2,q1; + +-- This should fail, because q2 isn't a name of an EXCEPT output column +SELECT q1 FROM int8_tbl EXCEPT SELECT q2 FROM int8_tbl ORDER BY q2 LIMIT 1; + +-- But this should work: +SELECT q1 FROM int8_tbl EXCEPT (((SELECT q2 FROM int8_tbl ORDER BY q2 LIMIT 1))) ORDER BY 1; + +-- +-- New syntaxes (7.1) permit new tests +-- + +(((((select * from int8_tbl))))); + +-- [SPARK-28557] Support empty select list +-- +-- Check behavior with empty select list (allowed since 9.4) +-- + +-- select union select; +-- select intersect select; +-- select except select; + +-- check hashed implementation +-- set enable_hashagg = true; +-- set enable_sort = false; + +-- explain (costs off) +-- select from generate_series(1,5) union select from generate_series(1,3); +-- explain (costs off) +-- select from generate_series(1,5) intersect select from generate_series(1,3); + +-- [SPARK-28409] SELECT FROM syntax +-- [SPARK-27767] Built-in function: generate_series +select * from range(1,5) union select * from range(1,3); +select * from range(1,6) union all select * from range(1,4); +select * from range(1,6) intersect select * from range(1,4); +select * from range(1,6) intersect all select * from range(1,4); +select * from range(1,6) except select * from range(1,4); +select * from range(1,6) except all select * from range(1,4); + +-- check sorted implementation +-- set enable_hashagg = false; +-- set enable_sort = true; + +-- explain (costs off) +-- select from generate_series(1,5) union select from generate_series(1,3); +-- explain (costs off) +-- select from generate_series(1,5) intersect select from generate_series(1,3); + +select * from range(1,6) union select * from range(1,4); +select * from range(1,6) union all select * from range(1,4); +select * from range(1,6) intersect select * from range(1,4); +select * from range(1,6) intersect all select * from range(1,4); +select * from range(1,6) except select * from range(1,4); +select * from range(1,6) except all select * from range(1,4); + +-- reset enable_hashagg; +-- reset enable_sort; + +-- +-- Check handling of a case with unknown constants. We don't guarantee +-- an undecorated constant will work in all cases, but historically this +-- usage has worked, so test we don't break it. +-- + +-- SELECT a.f1 FROM (SELECT 'test' AS f1 FROM varchar_tbl) a +-- UNION +-- SELECT b.f1 FROM (SELECT f1 FROM varchar_tbl) b +-- ORDER BY 1; + +-- This should fail, but it should produce an error cursor +SELECT cast('3.4' as decimal(38, 18)) UNION SELECT 'foo'; + +-- Skip this test because it only test explain +-- +-- Test that expression-index constraints can be pushed down through +-- UNION or UNION ALL +-- + +-- CREATE TEMP TABLE t1 (a text, b text); +-- CREATE INDEX t1_ab_idx on t1 ((a || b)); +-- CREATE TEMP TABLE t2 (ab text primary key); +-- INSERT INTO t1 VALUES ('a', 'b'), ('x', 'y'); +-- INSERT INTO t2 VALUES ('ab'), ('xy'); + +-- set enable_seqscan = off; +-- set enable_indexscan = on; +-- set enable_bitmapscan = off; + +-- explain (costs off) +-- SELECT * FROM +-- (SELECT a || b AS ab FROM t1 +-- UNION ALL +-- SELECT * FROM t2) t +-- WHERE ab = 'ab'; + +-- explain (costs off) +-- SELECT * FROM +-- (SELECT a || b AS ab FROM t1 +-- UNION +-- SELECT * FROM t2) t +-- WHERE ab = 'ab'; + +-- Skip this test because we do not support inheritance +-- +-- Test that ORDER BY for UNION ALL can be pushed down to inheritance +-- children. +-- + +-- CREATE TEMP TABLE t1c (b text, a text); +-- ALTER TABLE t1c INHERIT t1; +-- CREATE TEMP TABLE t2c (primary key (ab)) INHERITS (t2); +-- INSERT INTO t1c VALUES ('v', 'w'), ('c', 'd'), ('m', 'n'), ('e', 'f'); +-- INSERT INTO t2c VALUES ('vw'), ('cd'), ('mn'), ('ef'); +-- CREATE INDEX t1c_ab_idx on t1c ((a || b)); + +-- set enable_seqscan = on; +-- set enable_indexonlyscan = off; + +-- explain (costs off) +-- SELECT * FROM +-- (SELECT a || b AS ab FROM t1 +-- UNION ALL +-- SELECT ab FROM t2) t +-- ORDER BY 1 LIMIT 8; + +-- SELECT * FROM +-- (SELECT a || b AS ab FROM t1 +-- UNION ALL +-- SELECT ab FROM t2) t +-- ORDER BY 1 LIMIT 8; + +-- reset enable_seqscan; +-- reset enable_indexscan; +-- reset enable_bitmapscan; + +-- This simpler variant of the above test has been observed to fail differently + +-- create table events (event_id int primary key); +-- create table other_events (event_id int primary key); +-- create table events_child () inherits (events); + +-- explain (costs off) +-- select event_id +-- from (select event_id from events +-- union all +-- select event_id from other_events) ss +-- order by event_id; + +-- drop table events_child, events, other_events; + +-- reset enable_indexonlyscan; + +-- Test constraint exclusion of UNION ALL subqueries +-- explain (costs off) +-- SELECT * FROM +-- (SELECT 1 AS t, * FROM tenk1 a +-- UNION ALL +-- SELECT 2 AS t, * FROM tenk1 b) c +-- WHERE t = 2; + +-- Test that we push quals into UNION sub-selects only when it's safe +-- explain (costs off) +-- SELECT * FROM +-- (SELECT 1 AS t, 2 AS x +-- UNION +-- SELECT 2 AS t, 4 AS x) ss +-- WHERE x < 4 +-- ORDER BY x; + +SELECT * FROM + (SELECT 1 AS t, 2 AS x + UNION + SELECT 2 AS t, 4 AS x) ss +WHERE x < 4 +ORDER BY x; + +-- explain (costs off) +-- SELECT * FROM +-- (SELECT 1 AS t, generate_series(1,10) AS x +-- UNION +-- SELECT 2 AS t, 4 AS x) ss +-- WHERE x < 4 +-- ORDER BY x; + +; + +SELECT * FROM + (SELECT 1 AS t, id as x from range(1,11) + UNION + SELECT 2 AS t, 4 AS x) ss +WHERE x < 4 +ORDER BY x; + +-- explain (costs off) +-- SELECT * FROM +-- (SELECT 1 AS t, (random()*3)::int AS x +-- UNION +-- SELECT 2 AS t, 4 AS x) ss +-- WHERE x > 3 +-- ORDER BY x; + +SELECT * FROM + (SELECT 1 AS t, int((random()*3)) AS x + UNION + SELECT 2 AS t, 4 AS x) ss +WHERE x > 3 +ORDER BY x; + +-- Test cases where the native ordering of a sub-select has more pathkeys +-- than the outer query cares about +-- explain (costs off) +-- select distinct q1 from +-- (select distinct * from int8_tbl i81 +-- union all +-- select distinct * from int8_tbl i82) ss +-- where q2 = q2; + +select distinct q1 from + (select distinct * from int8_tbl i81 + union all + select distinct * from int8_tbl i82) ss +where q2 = q2; + +-- explain (costs off) +-- select distinct q1 from +-- (select distinct * from int8_tbl i81 +-- union all +-- select distinct * from int8_tbl i82) ss +-- where -q1 = q2; + +select distinct q1 from + (select distinct * from int8_tbl i81 + union all + select distinct * from int8_tbl i82) ss +where -q1 = q2; + +-- Skip this test because it only test explain +-- Test proper handling of parameterized appendrel paths when the +-- potential join qual is expensive +-- create function expensivefunc(int) returns int +-- language plpgsql immutable strict cost 10000 +-- as $$begin return $1; end$$; + +-- create temp table t3 as select generate_series(-1000,1000) as x; +-- create index t3i on t3 (expensivefunc(x)); +-- analyze t3; + +-- explain (costs off) +-- select * from +-- (select * from t3 a union all select * from t3 b) ss +-- join int4_tbl on f1 = expensivefunc(x); +-- select * from +-- (select * from t3 a union all select * from t3 b) ss +-- join int4_tbl on f1 = expensivefunc(x); + +-- drop table t3; +-- drop function expensivefunc(int); + +-- Test handling of appendrel quals that const-simplify into an AND +-- explain (costs off) +-- select * from +-- (select *, 0 as x from int8_tbl a +-- union all +-- select *, 1 as x from int8_tbl b) ss +-- where (x = 0) or (q1 >= q2 and q1 <= q2); +select * from + (select *, 0 as x from int8_tbl a + union all + select *, 1 as x from int8_tbl b) ss +where (x = 0) or (q1 >= q2 and q1 <= q2); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part1.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part1.sql new file mode 100644 index 000000000000..d8f9f63e816d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part1.sql @@ -0,0 +1,361 @@ +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- Window Functions Testing +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L1-L319 + +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- CREATE TABLE empsalary ( +-- depname string, +-- empno integer, +-- salary int, +-- enroll_date date +-- ) USING parquet; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- INSERT INTO empsalary VALUES ('develop', 10, 5200, '2007-08-01'); +-- INSERT INTO empsalary VALUES ('sales', 1, 5000, '2006-10-01'); +-- INSERT INTO empsalary VALUES ('personnel', 5, 3500, '2007-12-10'); +-- INSERT INTO empsalary VALUES ('sales', 4, 4800, '2007-08-08'); +-- INSERT INTO empsalary VALUES ('personnel', 2, 3900, '2006-12-23'); +-- INSERT INTO empsalary VALUES ('develop', 7, 4200, '2008-01-01'); +-- INSERT INTO empsalary VALUES ('develop', 9, 4500, '2008-01-01'); +-- INSERT INTO empsalary VALUES ('sales', 3, 4800, '2007-08-01'); +-- INSERT INTO empsalary VALUES ('develop', 8, 6000, '2006-10-01'); +-- INSERT INTO empsalary VALUES ('develop', 11, 5200, '2007-08-15'); + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- SELECT depname, empno, salary, sum(salary) OVER (PARTITION BY depname) FROM empsalary ORDER BY depname, salary; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- SELECT depname, empno, salary, rank() OVER (PARTITION BY depname ORDER BY salary) FROM empsalary; + +-- with GROUP BY +SELECT four, ten, SUM(SUM(four)) OVER (PARTITION BY four), AVG(ten) FROM tenk1 +GROUP BY four, ten ORDER BY four, ten; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- SELECT depname, empno, salary, sum(salary) OVER w FROM empsalary WINDOW w AS (PARTITION BY depname); + +-- [SPARK-28064] Order by does not accept a call to rank() +-- SELECT depname, empno, salary, rank() OVER w FROM empsalary WINDOW w AS (PARTITION BY depname ORDER BY salary) ORDER BY rank() OVER w; + +-- empty window specification +SELECT COUNT(*) OVER () FROM tenk1 WHERE unique2 < 10; + +SELECT COUNT(*) OVER w FROM tenk1 WHERE unique2 < 10 WINDOW w AS (); + +-- no window operation +SELECT four FROM tenk1 WHERE FALSE WINDOW w AS (PARTITION BY ten); + +-- cumulative aggregate +SELECT sum(four) OVER (PARTITION BY ten ORDER BY unique2) AS sum_1, ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT row_number() OVER (ORDER BY unique2) FROM tenk1 WHERE unique2 < 10; + +SELECT rank() OVER (PARTITION BY four ORDER BY ten) AS rank_1, ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT dense_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT percent_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT cume_dist() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT ntile(3) OVER (ORDER BY ten, four), ten, four FROM tenk1 WHERE unique2 < 10; + +-- [SPARK-28065] ntile does not accept NULL as input +-- SELECT ntile(NULL) OVER (ORDER BY ten, four), ten, four FROM tenk1 LIMIT 2; + +SELECT lag(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +-- [SPARK-28068] `lag` second argument must be a literal in Spark +-- SELECT lag(ten, four) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +-- [SPARK-28068] `lag` second argument must be a literal in Spark +-- SELECT lag(ten, four, 0) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT lead(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT lead(ten * 2, 1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT lead(ten * 2, 1, -1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT first(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +-- last returns the last row of the frame, which is CURRENT ROW in ORDER BY window. +SELECT last(four) OVER (ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10; + +SELECT last(ten) OVER (PARTITION BY four), ten, four FROM +(SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s +ORDER BY four, ten; + +-- [SPARK-30707] Lead/Lag window function throws AnalysisException without ORDER BY clause +-- SELECT nth_value(ten, four + 1) OVER (PARTITION BY four), ten, four +-- FROM (SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s; + +SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER (PARTITION BY two ORDER BY ten) AS wsum +FROM tenk1 GROUP BY ten, two; + +SELECT count(*) OVER (PARTITION BY four), four FROM (SELECT * FROM tenk1 WHERE two = 1)s WHERE unique2 < 10; + +SELECT (count(*) OVER (PARTITION BY four ORDER BY ten) + + sum(hundred) OVER (PARTITION BY four ORDER BY ten)) AS cntsum + FROM tenk1 WHERE unique2 < 10; + +-- opexpr with different windows evaluation. +SELECT * FROM( + SELECT count(*) OVER (PARTITION BY four ORDER BY ten) + + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS total, + count(*) OVER (PARTITION BY four ORDER BY ten) AS fourcount, + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS twosum + FROM tenk1 +)sub WHERE total <> fourcount + twosum; + +SELECT avg(four) OVER (PARTITION BY four ORDER BY thousand / 100) FROM tenk1 WHERE unique2 < 10; + +SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER win AS wsum +FROM tenk1 GROUP BY ten, two WINDOW win AS (PARTITION BY two ORDER BY ten); + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- more than one window with GROUP BY +-- SELECT sum(salary), +-- row_number() OVER (ORDER BY depname), +-- sum(sum(salary)) OVER (ORDER BY depname DESC) +-- FROM empsalary GROUP BY depname; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- identical windows with different names +-- SELECT sum(salary) OVER w1, count(*) OVER w2 +-- FROM empsalary WINDOW w1 AS (ORDER BY salary), w2 AS (ORDER BY salary); + +-- subplan +-- Cannot specify window frame for lead function +-- SELECT lead(ten, (SELECT two FROM tenk1 WHERE s.unique2 = unique2)) OVER (PARTITION BY four ORDER BY ten) +-- FROM tenk1 s WHERE unique2 < 10; + +-- empty table +SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 WHERE FALSE)s; + +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- mixture of agg/wfunc in the same window +-- SELECT sum(salary) OVER w, rank() OVER w FROM empsalary WINDOW w AS (PARTITION BY depname ORDER BY salary DESC); + +-- Cannot safely cast 'enroll_date': string to date; +-- SELECT empno, depname, salary, bonus, depadj, MIN(bonus) OVER (ORDER BY empno), MAX(depadj) OVER () FROM( +-- SELECT *, +-- CASE WHEN enroll_date < '2008-01-01' THEN 2008 - extract(year FROM enroll_date) END * 500 AS bonus, +-- CASE WHEN +-- AVG(salary) OVER (PARTITION BY depname) < salary +-- THEN 200 END AS depadj FROM empsalary +-- )s; + +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1); + +-- window function over ungrouped agg over empty row set (bug before 9.1) +SELECT SUM(COUNT(f1)) OVER () FROM int4_tbl WHERE f1=42; + +-- window function with ORDER BY an expression involving aggregates (9.1 bug) +select ten, + sum(unique1) + sum(unique2) as res, + rank() over (order by sum(unique1) + sum(unique2)) as rank +from tenk1 +group by ten order by ten; + +-- window and aggregate with GROUP BY expression (9.2 bug) +-- explain +-- select first(max(x)) over (), y +-- from (select unique1 as x, ten+four as y from tenk1) ss +-- group by y; + +-- test non-default frame specifications +SELECT four, ten, +sum(ten) over (partition by four order by ten), +last(ten) over (partition by four order by ten) +FROM (select distinct ten, four from tenk1) ss; + +SELECT four, ten, +sum(ten) over (partition by four order by ten range between unbounded preceding and current row), +last(ten) over (partition by four order by ten range between unbounded preceding and current row) +FROM (select distinct ten, four from tenk1) ss; + +SELECT four, ten, +sum(ten) over (partition by four order by ten range between unbounded preceding and unbounded following), +last(ten) over (partition by four order by ten range between unbounded preceding and unbounded following) +FROM (select distinct ten, four from tenk1) ss; + +-- [SPARK-29451] Some queries with divisions in SQL windows are failling in Thrift +-- SELECT four, ten/4 as two, +-- sum(ten/4) over (partition by four order by ten/4 range between unbounded preceding and current row), +-- last(ten/4) over (partition by four order by ten/4 range between unbounded preceding and current row) +-- FROM (select distinct ten, four from tenk1) ss; + +-- [SPARK-29451] Some queries with divisions in SQL windows are failling in Thrift +-- SELECT four, ten/4 as two, +-- sum(ten/4) over (partition by four order by ten/4 rows between unbounded preceding and current row), +-- last(ten/4) over (partition by four order by ten/4 rows between unbounded preceding and current row) +-- FROM (select distinct ten, four from tenk1) ss; + +SELECT sum(unique1) over (order by four range between current row and unbounded following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (rows between current row and unbounded following), +unique1, four +FROM tenk1 WHERE unique1 < 10; +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +SELECT sum(unique1) over (rows between 2 preceding and 2 following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude no others), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (rows between 2 preceding and 2 following exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT first(unique1) over (ORDER BY four rows between current row and 2 following exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT first(unique1) over (ORDER BY four rows between current row and 2 following exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT first(unique1) over (ORDER BY four rows between current row and 2 following exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT last(unique1) over (ORDER BY four rows between current row and 2 following exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT last(unique1) over (ORDER BY four rows between current row and 2 following exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT last(unique1) over (ORDER BY four rows between current row and 2 following exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (rows between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10; +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +SELECT sum(unique1) over (rows between 1 following and 3 following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (rows between unbounded preceding and 1 following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (w range between current row and unbounded following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (w range between unbounded preceding and current row exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (w range between unbounded preceding and current row exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (w range between unbounded preceding and current row exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10 WINDOW w AS (order by four); + +-- [SPARK-30707] Lead/Lag window function throws AnalysisException without ORDER BY clause +-- SELECT first_value(unique1) over w, +-- nth_value(unique1, 2) over w AS nth_2, +-- last_value(unique1) over w, unique1, four +-- FROM tenk1 WHERE unique1 < 10 +-- WINDOW w AS (order by four range between current row and unbounded following); + +-- [SPARK-28501] Frame bound value must be a literal. +-- SELECT sum(unique1) over +-- (order by unique1 +-- rows (SELECT unique1 FROM tenk1 ORDER BY unique1 LIMIT 1) + 1 PRECEDING), +-- unique1 +-- FROM tenk1 WHERE unique1 < 10; + +CREATE TEMP VIEW v_window AS +SELECT i.id, sum(i.id) over (order by i.id rows between 1 preceding and 1 following) as sum_rows +FROM range(1, 11) i; + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; +SELECT * FROM v_window; + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following +-- exclude current row) as sum_rows FROM range(1, 10) i; + +-- SELECT * FROM v_window; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following +-- exclude group) as sum_rows FROM range(1, 10) i; +-- SELECT * FROM v_window; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following +-- exclude ties) as sum_rows FROM generate_series(1, 10) i; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i, sum(i) over (order by i rows between 1 preceding and 1 following +-- exclude no others) as sum_rows FROM generate_series(1, 10) i; +-- SELECT * FROM v_window; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- CREATE OR REPLACE TEMP VIEW v_window AS +-- SELECT i.id, sum(i.id) over (order by i.id groups between 1 preceding and 1 following) as sum_rows FROM range(1, 11) i; +-- SELECT * FROM v_window; + +DROP VIEW v_window; +-- [SPARK-29540] Thrift in some cases can't parse string to date +-- DROP TABLE empsalary; +DROP VIEW tenk2; +DROP VIEW int4_tbl; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part2.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part2.sql new file mode 100644 index 000000000000..50c0bc341031 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part2.sql @@ -0,0 +1,303 @@ +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- Window Functions Testing +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L320-562 + +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +CREATE TABLE empsalary ( + depname string, + empno integer, + salary int, + enroll_date date +) USING parquet; + +INSERT INTO empsalary VALUES + ('develop', 10, 5200, date '2007-08-01'), + ('sales', 1, 5000, date '2006-10-01'), + ('personnel', 5, 3500, date '2007-12-10'), + ('sales', 4, 4800, date '2007-08-08'), + ('personnel', 2, 3900, date '2006-12-23'), + ('develop', 7, 4200, date '2008-01-01'), + ('develop', 9, 4500, date '2008-01-01'), + ('sales', 3, 4800, date '2007-08-01'), + ('develop', 8, 6000, date '2006-10-01'), + ('develop', 11, 5200, date '2007-08-15'); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- CREATE TEMP VIEW v_window AS +-- SELECT i, min(i) over (order by i range between '1 day' preceding and '10 days' following) as min_i +-- FROM range(now(), now()+'100 days', '1 hour') i; + +-- RANGE offset PRECEDING/FOLLOWING tests + +SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (order by four desc range between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding exclude no others), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding exclude current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 6 following exclude ties), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 6 following exclude group), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +SELECT sum(unique1) over (partition by four order by unique1 range between 5 preceding and 6 following), +unique1, four +FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (partition by four order by unique1 range between 5 preceding and 6 following +-- exclude current row),unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select sum(salary) over (order by enroll_date range between '1 year' preceding and '1 year' following), +-- salary, enroll_date from empsalary; + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select sum(salary) over (order by enroll_date desc range between '1 year' preceding and '1 year' following), +-- salary, enroll_date from empsalary; + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select sum(salary) over (order by enroll_date desc range between '1 year' following and '1 year' following), +-- salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select sum(salary) over (order by enroll_date range between '1 year' preceding and '1 year' following +-- exclude current row), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select sum(salary) over (order by enroll_date range between '1 year' preceding and '1 year' following +-- exclude group), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select sum(salary) over (order by enroll_date range between '1 year' preceding and '1 year' following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28310] ANSI SQL grammar support: first_value/last_value(expression, [RESPECT NULLS | IGNORE NULLS]) +-- select first_value(salary) over(order by salary range between 1000 preceding and 1000 following), +-- lead(salary) over(order by salary range between 1000 preceding and 1000 following), +-- nth_value(salary, 1) over(order by salary range between 1000 preceding and 1000 following), +-- salary from empsalary; + +-- [SPARK-30734] AnalysisException that window RangeFrame not match RowFrame +-- select last(salary) over(order by salary range between 1000 preceding and 1000 following), +-- lag(salary) over(order by salary range between 1000 preceding and 1000 following), +-- salary from empsalary; + +-- [SPARK-28310] ANSI SQL grammar support: first_value/last_value(expression, [RESPECT NULLS | IGNORE NULLS]) +-- select first_value(salary) over(order by salary range between 1000 following and 3000 following +-- exclude current row), +-- lead(salary) over(order by salary range between 1000 following and 3000 following exclude ties), +-- nth_value(salary, 1) over(order by salary range between 1000 following and 3000 following +-- exclude ties), +-- salary from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select last(salary) over(order by salary range between 1000 following and 3000 following +-- exclude group), +-- lag(salary) over(order by salary range between 1000 following and 3000 following exclude group), +-- salary from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select first(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude ties), +-- last(salary) over(order by enroll_date range between unbounded preceding and '1 year' following), +-- salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select first(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude ties), +-- last(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude ties), +-- salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select first(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude group), +-- last(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude group), +-- salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select first(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude current row), +-- last(salary) over(order by enroll_date range between unbounded preceding and '1 year' following +-- exclude current row), +-- salary, enroll_date from empsalary; + +-- RANGE offset PRECEDING/FOLLOWING with null values +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id asc nulls first range between 2 preceding and 2 following); + +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id asc nulls last range between 2 preceding and 2 following); + +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id desc nulls first range between 2 preceding and 2 following); + +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id desc nulls last range between 2 preceding and 2 following); + +-- Check overflow behavior for various integer sizes + +select x.id, last(x.id) over (order by x.id range between current row and 2147450884 following) +from range(32764, 32767) x; + +select x.id, last(x.id) over (order by x.id desc range between current row and 2147450885 following) +from range(-32766, -32765) x; + +select x.id, last(x.id) over (order by x.id range between current row and 4 following) +from range(2147483644, 2147483647) x; + +select x.id, last(x.id) over (order by x.id desc range between current row and 5 following) +from range(-2147483646, -2147483645) x; + +select x.id, last(x.id) over (order by x.id range between current row and 4 following) +from range(9223372036854775804, 9223372036854775807) x; + +select x.id, last(x.id) over (order by x.id desc range between current row and 5 following) +from range(-9223372036854775806, -9223372036854775805) x; + +-- Test in_range for other numeric datatypes + +create table numerics ( + id int, + f_float4 float, + f_float8 float, + f_numeric int +) using parquet; + +insert into numerics values +(1, -3, -3, -3), +(2, -1, -1, -1), +(3, 0, 0, 0), +(4, 1.1, 1.1, 1.1), +(5, 1.12, 1.12, 1.12), +(6, 2, 2, 2), +(7, 100, 100, 100); +-- (8, 'infinity', 'infinity', '1000'), +-- (9, 'NaN', 'NaN', 'NaN'), +-- (0, '-infinity', '-infinity', '-1000'); -- numeric type lacks infinities + +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1 preceding and 1 following); + +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1 preceding and 1.1 following); + +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 'inf' preceding and 'inf' following); + +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1.1 preceding and 'NaN' following); -- error, NaN disallowed + +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1 preceding and 1 following); + +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1 preceding and 1.1 following); + +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 'inf' preceding and 'inf' following); + +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1.1 preceding and 'NaN' following); -- error, NaN disallowed + +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1 following); + +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1.1 following); + +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1.1 following); -- currently unsupported + +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1.1 preceding and 'NaN' following); -- error, NaN disallowed + +drop table empsalary; +drop table numerics; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part3.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part3.sql new file mode 100644 index 000000000000..6aad63beffcf --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part3.sql @@ -0,0 +1,459 @@ +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- Window Functions Testing +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L564-L911 + +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1; + +CREATE TABLE empsalary ( + depname string, + empno integer, + salary int, + enroll_date date +) USING parquet; + +INSERT INTO empsalary VALUES + ('develop', 10, 5200, date '2007-08-01'), + ('sales', 1, 5000, date '2006-10-01'), + ('personnel', 5, 3500, date '2007-12-10'), + ('sales', 4, 4800, date '2007-08-08'), + ('personnel', 2, 3900, date '2006-12-23'), + ('develop', 7, 4200, date '2008-01-01'), + ('develop', 9, 4500, date '2008-01-01'), + ('sales', 3, 4800, date '2007-08-01'), + ('develop', 8, 6000, date '2006-10-01'), + ('develop', 11, 5200, date '2007-08-15'); + +-- Test in_range for other datetime datatypes + +-- Spark only supports timestamp +-- [SPARK-29636] Spark can't parse '11:00 BST' or '2000-10-19 10:23:54+01' signatures to timestamp +create table datetimes ( + id int, + f_time timestamp, + f_timetz timestamp, + f_interval timestamp, + f_timestamptz timestamp, + f_timestamp timestamp +) using parquet; + +-- Spark cannot safely cast string to timestamp +-- [SPARK-29636] Spark can't parse '11:00 BST' or '2000-10-19 10:23:54+01' signatures to timestamp +insert into datetimes values +(1, timestamp '11:00', cast ('11:00 BST' as timestamp), cast ('1 year' as timestamp), cast ('2000-10-19 10:23:54+01' as timestamp), timestamp '2000-10-19 10:23:54'), +(2, timestamp '12:00', cast ('12:00 BST' as timestamp), cast ('2 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'), +(3, timestamp '13:00', cast ('13:00 BST' as timestamp), cast ('3 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'), +(4, timestamp '14:00', cast ('14:00 BST' as timestamp), cast ('4 years' as timestamp), cast ('2002-10-19 10:23:54+01' as timestamp), timestamp '2002-10-19 10:23:54'), +(5, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2003-10-19 10:23:54+01' as timestamp), timestamp '2003-10-19 10:23:54'), +(6, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2004-10-19 10:23:54+01' as timestamp), timestamp '2004-10-19 10:23:54'), +(7, timestamp '17:00', cast ('17:00 BST' as timestamp), cast ('7 years' as timestamp), cast ('2005-10-19 10:23:54+01' as timestamp), timestamp '2005-10-19 10:23:54'), +(8, timestamp '18:00', cast ('18:00 BST' as timestamp), cast ('8 years' as timestamp), cast ('2006-10-19 10:23:54+01' as timestamp), timestamp '2006-10-19 10:23:54'), +(9, timestamp '19:00', cast ('19:00 BST' as timestamp), cast ('9 years' as timestamp), cast ('2007-10-19 10:23:54+01' as timestamp), timestamp '2007-10-19 10:23:54'), +(10, timestamp '20:00', cast ('20:00 BST' as timestamp), cast ('10 years' as timestamp), cast ('2008-10-19 10:23:54+01' as timestamp), timestamp '2008-10-19 10:23:54'); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_time, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_time range between +-- '70 min' preceding and '2 hours' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_time, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_time desc range between +-- '70 min' preceding and '2 hours' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_timetz, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_timetz range between +-- '70 min' preceding and '2 hours' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_timetz, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_timetz desc range between +-- '70 min' preceding and '2 hours' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_interval, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_interval range between +-- '1 year' preceding and '1 year' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_interval, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_interval desc range between +-- '1 year' preceding and '1 year' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_timestamptz, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_timestamptz range between +-- '1 year' preceding and '1 year' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_timestamptz, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_timestamptz desc range between +-- '1 year' preceding and '1 year' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_timestamp, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_timestamp range between +-- '1 year' preceding and '1 year' following); + +-- [SPARK-28429] SQL Datetime util function being casted to double instead of timestamp +-- select id, f_timestamp, first(id) over w, last(id) over w +-- from datetimes +-- window w as (order by f_timestamp desc range between +-- '1 year' preceding and '1 year' following); + +-- RANGE offset PRECEDING/FOLLOWING error cases +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select sum(salary) over (order by enroll_date, salary range between '1 year' preceding and '2 years' following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select sum(salary) over (range between '1 year' preceding and '2 years' following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select sum(salary) over (order by depname range between '1 year' preceding and '2 years' following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select max(enroll_date) over (order by enroll_date range between 1 preceding and 2 following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select max(enroll_date) over (order by salary range between -1 preceding and 2 following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select max(enroll_date) over (order by salary range between 1 preceding and -2 following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select max(enroll_date) over (order by salary range between '1 year' preceding and '2 years' following +-- exclude ties), salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select max(enroll_date) over (order by enroll_date range between '1 year' preceding and '-2 years' following +-- exclude ties), salary, enroll_date from empsalary; + +-- GROUPS tests + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between unbounded preceding and current row), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between unbounded preceding and unbounded following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between current row and unbounded following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between 1 preceding and unbounded following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between 1 following and unbounded following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between unbounded preceding and 2 following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between 2 preceding and 1 preceding), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between 2 preceding and 1 following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (order by four groups between 0 preceding and 0 following), +-- unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four groups between 2 preceding and 1 following +-- exclude current row), unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 1 following +-- exclude group), unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- SELECT sum(unique1) over (order by four range between 2 preceding and 1 following +-- exclude ties), unique1, four +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (partition by ten +-- order by four groups between 0 preceding and 0 following),unique1, four, ten +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (partition by ten +-- order by four groups between 0 preceding and 0 following exclude current row), unique1, four, ten +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (partition by ten +-- order by four groups between 0 preceding and 0 following exclude group), unique1, four, ten +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- SELECT sum(unique1) over (partition by ten +-- order by four groups between 0 preceding and 0 following exclude ties), unique1, four, ten +-- FROM tenk1 WHERE unique1 < 10; + +-- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- select first_value(salary) over(order by enroll_date groups between 1 preceding and 1 following), +-- lead(salary) over(order by enroll_date groups between 1 preceding and 1 following), +-- nth_value(salary, 1) over(order by enroll_date groups between 1 preceding and 1 following), +-- salary, enroll_date from empsalary; + +-- [SPARK-28508] Support for range frame+row frame in the same query +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- select last(salary) over(order by enroll_date groups between 1 preceding and 1 following), +-- lag(salary) over(order by enroll_date groups between 1 preceding and 1 following), +-- salary, enroll_date from empsalary; + +-- [SPARK-27951] ANSI SQL: NTH_VALUE function +-- select first_value(salary) over(order by enroll_date groups between 1 following and 3 following +-- exclude current row), +-- lead(salary) over(order by enroll_date groups between 1 following and 3 following exclude ties), +-- nth_value(salary, 1) over(order by enroll_date groups between 1 following and 3 following +-- exclude ties), +-- salary, enroll_date from empsalary; + +-- [SPARK-28428] Spark `exclude` always expecting `()` +-- select last(salary) over(order by enroll_date groups between 1 following and 3 following +-- exclude group), +-- lag(salary) over(order by enroll_date groups between 1 following and 3 following exclude group), +-- salary, enroll_date from empsalary; + +-- Show differences in offset interpretation between ROWS, RANGE, and GROUPS +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; +WITH cte (x) AS ( + SELECT * FROM range(1, 36, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following); +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +WITH cte (x) AS ( + SELECT * FROM range(1, 36, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x range between 1 preceding and 1 following); + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- WITH cte (x) AS ( +-- SELECT * FROM range(1, 36, 2) +-- ) +-- SELECT x, (sum(x) over w) +-- FROM cte +-- WINDOW w AS (ORDER BY x groups between 1 preceding and 1 following); + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; +WITH cte (x) AS ( + select 1 union all select 1 union all select 1 union all + SELECT * FROM range(5, 50, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following); + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; +WITH cte (x) AS ( + select 1 union all select 1 union all select 1 union all + SELECT * FROM range(5, 50, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x range between 1 preceding and 1 following); + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- WITH cte (x) AS ( +-- select 1 union all select 1 union all select 1 union all +-- SELECT * FROM range(5, 50, 2) +-- ) +-- SELECT x, (sum(x) over w) +-- FROM cte +-- WINDOW w AS (ORDER BY x groups between 1 preceding and 1 following); + +-- with UNION +SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 UNION ALL SELECT * FROM tenk2)s LIMIT 0; + +-- check some degenerate cases +create table t1 (f1 int, f2 int) using parquet; +insert into t1 values (1,1),(1,2),(2,2); + +select f1, sum(f1) over (partition by f1 + range between 1 preceding and 1 following) +from t1 where f1 = f2; -- error, must have order by + +-- Since EXPLAIN clause rely on host physical location, it is commented out +-- explain +-- select f1, sum(f1) over (partition by f1 order by f2 +-- range between 1 preceding and 1 following) +-- from t1 where f1 = f2; + +select f1, sum(f1) over (partition by f1 order by f2 +range between 1 preceding and 1 following) +from t1 where f1 = f2; + +select f1, sum(f1) over (partition by f1, f1 order by f2 +range between 2 preceding and 1 preceding) +from t1 where f1 = f2; + +select f1, sum(f1) over (partition by f1, f2 order by f2 +range between 1 following and 2 following) +from t1 where f1 = f2; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- select f1, sum(f1) over (partition by f1, +-- groups between 1 preceding and 1 following) +-- from t1 where f1 = f2; + +-- Since EXPLAIN clause rely on host physical location, it is commented out +-- explain +-- select f1, sum(f1) over (partition by f1 order by f2 +-- range between 1 preceding and 1 following) +-- from t1 where f1 = f2; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- select f1, sum(f1) over (partition by f1 order by f2 +-- groups between 1 preceding and 1 following) +-- from t1 where f1 = f2; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- select f1, sum(f1) over (partition by f1, f1 order by f2 +-- groups between 2 preceding and 1 preceding) +-- from t1 where f1 = f2; + +-- [SPARK-28648] Adds support to `groups` unit type in window clauses +-- select f1, sum(f1) over (partition by f1, f2 order by f2 +-- groups between 1 following and 2 following) +-- from t1 where f1 = f2; + +-- ordering by a non-integer constant is allowed +SELECT rank() OVER (ORDER BY length('abc')); + +-- can't order by another window function +-- [SPARK-28566] window functions should not be allowed in window definitions +-- SELECT rank() OVER (ORDER BY rank() OVER (ORDER BY random())); + +-- some other errors +SELECT * FROM empsalary WHERE row_number() OVER (ORDER BY salary) < 10; + +SELECT * FROM empsalary INNER JOIN tenk1 ON row_number() OVER (ORDER BY salary) < 10; + +SELECT rank() OVER (ORDER BY 1), count(*) FROM empsalary GROUP BY 1; + +SELECT * FROM rank() OVER (ORDER BY random()); + +-- Original query: DELETE FROM empsalary WHERE (rank() OVER (ORDER BY random())) > 10; +SELECT * FROM empsalary WHERE (rank() OVER (ORDER BY random())) > 10; + +-- Original query: DELETE FROM empsalary RETURNING rank() OVER (ORDER BY random()); +SELECT * FROM empsalary WHERE rank() OVER (ORDER BY random()); + +-- [SPARK-28645] Throw an error on window redefinition +-- select count(*) OVER w FROM tenk1 WINDOW w AS (ORDER BY unique1), w AS (ORDER BY unique1); + +select rank() OVER (PARTITION BY four, ORDER BY ten) FROM tenk1; + +-- [SPARK-28646] Allow usage of `count` only for parameterless aggregate function +-- select count() OVER () FROM tenk1; + +-- The output is the expected one: `range` is not a window or aggregate function. +SELECT range(1, 100) OVER () FROM empsalary; + +SELECT ntile(0) OVER (ORDER BY ten), ten, four FROM tenk1; + +SELECT nth_value(four, 0) OVER (ORDER BY ten), ten, four FROM tenk1; + +-- filter + +-- [SPARK-30182] Support nested aggregates +-- SELECT sum(salary), row_number() OVER (ORDER BY depname), sum( +-- sum(salary) FILTER (WHERE enroll_date > '2007-01-01') +-- ) +-- FROM empsalary GROUP BY depname; + +-- Test pushdown of quals into a subquery containing window functions + +-- pushdown is safe because all PARTITION BY clauses include depname: +-- Since EXPLAIN clause rely on host physical location, it is commented out +-- EXPLAIN +-- SELECT * FROM +-- (SELECT depname, +-- sum(salary) OVER (PARTITION BY depname) depsalary, +-- min(salary) OVER (PARTITION BY depname || 'A', depname) depminsalary +-- FROM empsalary) emp +-- WHERE depname = 'sales'; + +-- pushdown is unsafe because there's a PARTITION BY clause without depname: +-- Since EXPLAIN clause rely on host physical location, it is commented out +-- EXPLAIN +-- SELECT * FROM +-- (SELECT depname, +-- sum(salary) OVER (PARTITION BY enroll_date) enroll_salary, +-- min(salary) OVER (PARTITION BY depname) depminsalary +-- FROM empsalary) emp +-- WHERE depname = 'sales'; + +-- Test Sort node collapsing +-- Since EXPLAIN clause rely on host physical location, it is commented out +-- EXPLAIN +-- SELECT * FROM +-- (SELECT depname, +-- sum(salary) OVER (PARTITION BY depname order by empno) depsalary, +-- min(salary) OVER (PARTITION BY depname, empno order by enroll_date) depminsalary +-- FROM empsalary) emp +-- WHERE depname = 'sales'; + +-- Test Sort node reordering +-- Since EXPLAIN clause rely on host physical location, it is commented out +-- EXPLAIN +-- SELECT +-- lead(1) OVER (PARTITION BY depname ORDER BY salary, enroll_date), +-- lag(1) OVER (PARTITION BY depname ORDER BY salary,enroll_date,empno) +-- FROM empsalary; + +-- cleanup +DROP TABLE empsalary; +DROP TABLE datetimes; +DROP TABLE t1; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part4.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part4.sql new file mode 100644 index 000000000000..484c7767ba8c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/window_part4.sql @@ -0,0 +1,406 @@ +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- Window Functions Testing +-- https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/sql/window.sql#L913-L1278 + +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +-- Spark doesn't handle UDFs in SQL +-- test user-defined window function with named args and default args +-- CREATE FUNCTION nth_value_def(val anyelement, n integer = 1) RETURNS anyelement +-- LANGUAGE internal WINDOW IMMUTABLE STRICT AS 'window_nth_value'; + +-- Spark doesn't handle UDFs in SQL +-- SELECT nth_value_def(n := 2, val := ten) OVER (PARTITION BY four), ten, four +-- FROM (SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten) s; + +-- Spark doesn't handle UDFs in SQL +-- SELECT nth_value_def(ten) OVER (PARTITION BY four), ten, four +-- FROM (SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten) s; + +-- +-- Test the basic moving-aggregate machinery +-- + +-- create aggregates that record the series of transform calls (these are +-- intentionally not true inverses) + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION logging_sfunc_nonstrict(text, anyelement) RETURNS text AS +-- $$ SELECT COALESCE($1, '') || '*' || quote_nullable($2) $$ +-- LANGUAGE SQL IMMUTABLE; + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION logging_msfunc_nonstrict(text, anyelement) RETURNS text AS +-- $$ SELECT COALESCE($1, '') || '+' || quote_nullable($2) $$ +-- LANGUAGE SQL IMMUTABLE; + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION logging_minvfunc_nonstrict(text, anyelement) RETURNS text AS +-- $$ SELECT $1 || '-' || quote_nullable($2) $$ +-- LANGUAGE SQL IMMUTABLE; + +-- Spark doesn't handle UDFs in SQL +-- CREATE AGGREGATE logging_agg_nonstrict (anyelement) +-- ( +-- stype = text, +-- sfunc = logging_sfunc_nonstrict, +-- mstype = text, +-- msfunc = logging_msfunc_nonstrict, +-- minvfunc = logging_minvfunc_nonstrict +-- ); + +-- Spark doesn't handle UDFs in SQL +-- CREATE AGGREGATE logging_agg_nonstrict_initcond (anyelement) +-- ( +-- stype = text, +-- sfunc = logging_sfunc_nonstrict, +-- mstype = text, +-- msfunc = logging_msfunc_nonstrict, +-- minvfunc = logging_minvfunc_nonstrict, +-- initcond = 'I', +-- minitcond = 'MI' +-- ); + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION logging_sfunc_strict(text, anyelement) RETURNS text AS +-- $$ SELECT $1 || '*' || quote_nullable($2) $$ +-- LANGUAGE SQL STRICT IMMUTABLE; + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION logging_msfunc_strict(text, anyelement) RETURNS text AS +-- $$ SELECT $1 || '+' || quote_nullable($2) $$ +-- LANGUAGE SQL STRICT IMMUTABLE; + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION logging_minvfunc_strict(text, anyelement) RETURNS text AS +-- $$ SELECT $1 || '-' || quote_nullable($2) $$ +-- LANGUAGE SQL STRICT IMMUTABLE; + +-- Spark doesn't handle UDFs in SQL +-- CREATE AGGREGATE logging_agg_strict (text) +-- ( +-- stype = text, +-- sfunc = logging_sfunc_strict, +-- mstype = text, +-- msfunc = logging_msfunc_strict, +-- minvfunc = logging_minvfunc_strict +-- ); + +-- Spark doesn't handle UDFs in SQL +-- CREATE AGGREGATE logging_agg_strict_initcond (anyelement) +-- ( +-- stype = text, +-- sfunc = logging_sfunc_strict, +-- mstype = text, +-- msfunc = logging_msfunc_strict, +-- minvfunc = logging_minvfunc_strict, +-- initcond = 'I', +-- minitcond = 'MI' +-- ); + +-- Spark doesn't handle UDFs in SQL +-- test strict and non-strict cases +-- SELECT +-- p::text || ',' || i::text || ':' || COALESCE(v::text, 'NULL') AS row, +-- logging_agg_nonstrict(v) over wnd as nstrict, +-- logging_agg_nonstrict_initcond(v) over wnd as nstrict_init, +-- logging_agg_strict(v::text) over wnd as strict, +-- logging_agg_strict_initcond(v) over wnd as strict_init +-- FROM (VALUES +-- (1, 1, NULL), +-- (1, 2, 'a'), +-- (1, 3, 'b'), +-- (1, 4, NULL), +-- (1, 5, NULL), +-- (1, 6, 'c'), +-- (2, 1, NULL), +-- (2, 2, 'x'), +-- (3, 1, 'z') +-- ) AS t(p, i, v) +-- WINDOW wnd AS (PARTITION BY P ORDER BY i ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +-- ORDER BY p, i; + +-- Spark doesn't handle UDFs in SQL +-- and again, but with filter +-- SELECT +-- p::text || ',' || i::text || ':' || +-- CASE WHEN f THEN COALESCE(v::text, 'NULL') ELSE '-' END as row, +-- logging_agg_nonstrict(v) filter(where f) over wnd as nstrict_filt, +-- logging_agg_nonstrict_initcond(v) filter(where f) over wnd as nstrict_init_filt, +-- logging_agg_strict(v::text) filter(where f) over wnd as strict_filt, +-- logging_agg_strict_initcond(v) filter(where f) over wnd as strict_init_filt +-- FROM (VALUES +-- (1, 1, true, NULL), +-- (1, 2, false, 'a'), +-- (1, 3, true, 'b'), +-- (1, 4, false, NULL), +-- (1, 5, false, NULL), +-- (1, 6, false, 'c'), +-- (2, 1, false, NULL), +-- (2, 2, true, 'x'), +-- (3, 1, true, 'z') +-- ) AS t(p, i, f, v) +-- WINDOW wnd AS (PARTITION BY p ORDER BY i ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +-- ORDER BY p, i; + +-- Spark doesn't handle UDFs in SQL +-- test that volatile arguments disable moving-aggregate mode +-- SELECT +-- i::text || ':' || COALESCE(v::text, 'NULL') as row, +-- logging_agg_strict(v::text) +-- over wnd as inverse, +-- logging_agg_strict(v::text || CASE WHEN random() < 0 then '?' ELSE '' END) +-- over wnd as noinverse +-- FROM (VALUES +-- (1, 'a'), +-- (2, 'b'), +-- (3, 'c') +-- ) AS t(i, v) +-- WINDOW wnd AS (ORDER BY i ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +-- ORDER BY i; + +-- Spark doesn't handle UDFs in SQL +-- SELECT +-- i::text || ':' || COALESCE(v::text, 'NULL') as row, +-- logging_agg_strict(v::text) filter(where true) +-- over wnd as inverse, +-- logging_agg_strict(v::text) filter(where random() >= 0) +-- over wnd as noinverse +-- FROM (VALUES +-- (1, 'a'), +-- (2, 'b'), +-- (3, 'c') +-- ) AS t(i, v) +-- WINDOW wnd AS (ORDER BY i ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +-- ORDER BY i; + +-- Spark doesn't handle UDFs in SQL +-- test that non-overlapping windows don't use inverse transitions +-- SELECT +-- logging_agg_strict(v::text) OVER wnd +-- FROM (VALUES +-- (1, 'a'), +-- (2, 'b'), +-- (3, 'c') +-- ) AS t(i, v) +-- WINDOW wnd AS (ORDER BY i ROWS BETWEEN CURRENT ROW AND CURRENT ROW) +-- ORDER BY i; + +-- Spark doesn't handle UDFs in SQL +-- test that returning NULL from the inverse transition functions +-- restarts the aggregation from scratch. The second aggregate is supposed +-- to test cases where only some aggregates restart, the third one checks +-- that one aggregate restarting doesn't cause others to restart. + +-- Spark doesn't handle UDFs in SQL +-- CREATE FUNCTION sum_int_randrestart_minvfunc(int4, int4) RETURNS int4 AS +-- $$ SELECT CASE WHEN random() < 0.2 THEN NULL ELSE $1 - $2 END $$ +-- LANGUAGE SQL STRICT; + +-- Spark doesn't handle UDFs in SQL +-- CREATE AGGREGATE sum_int_randomrestart (int4) +-- ( +-- stype = int4, +-- sfunc = int4pl, +-- mstype = int4, +-- msfunc = int4pl, +-- minvfunc = sum_int_randrestart_minvfunc +-- ); + +-- Spark doesn't handle UDFs in SQL +-- WITH +-- vs AS ( +-- SELECT i, (random() * 100)::int4 AS v +-- FROM generate_series(1, 100) AS i +-- ), +-- sum_following AS ( +-- SELECT i, SUM(v) OVER +-- (ORDER BY i DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS s +-- FROM vs +-- ) +-- SELECT DISTINCT +-- sum_following.s = sum_int_randomrestart(v) OVER fwd AS eq1, +-- -sum_following.s = sum_int_randomrestart(-v) OVER fwd AS eq2, +-- 100*3+(vs.i-1)*3 = length(logging_agg_nonstrict(''::text) OVER fwd) AS eq3 +-- FROM vs +-- JOIN sum_following ON sum_following.i = vs.i +-- WINDOW fwd AS ( +-- ORDER BY vs.i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING +-- ); + +-- +-- Test various built-in aggregates that have moving-aggregate support +-- + +-- test inverse transition functions handle NULLs properly +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.5),(2,2.5),(3,NULL),(4,NULL)) t(i,v); + +-- [SPARK-28602] Spark does not recognize 'interval' type as 'numeric' +-- SELECT i,AVG(v::interval) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +-- FROM (VALUES(1,'1 sec'),(2,'2 sec'),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +-- The cast syntax is present in PgSQL for legacy reasons and Spark will not recognize a money field +-- SELECT i,SUM(v::money) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +-- FROM (VALUES(1,'1.10'),(2,'2.20'),(3,NULL),(4,NULL)) t(i,v); + +-- [SPARK-28602] Spark does not recognize 'interval' type as 'numeric' +-- SELECT i,SUM(cast(v as interval)) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +-- FROM (VALUES(1,'1 sec'),(2,'2 sec'),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.1),(2,2.2),(3,NULL),(4,NULL)) t(i,v); + +SELECT SUM(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.01),(2,2),(3,3)) v(i,n); + +SELECT i,COUNT(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,COUNT(*) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +-- For the following queries Spark result differs from PgSQL: +-- Spark handles division by zero as 'NaN' instead of 'NULL', which is the PgSQL behaviour +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n); + +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n); + +-- test that inverse transition functions work with various frame options +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND CURRENT ROW) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v); +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,3),(4,4)) t(i,v); + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; +-- [SPARK-29638] Spark handles 'NaN' as 0 in sums +-- ensure aggregate over numeric properly recovers from NaN values +SELECT a, b, + SUM(b) OVER(ORDER BY A ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +FROM (VALUES(1,1),(2,2),(3,(cast('nan' as int))),(4,3),(5,4)) t(a,b); + +-- It might be tempting for someone to add an inverse trans function for +-- float and double precision. This should not be done as it can give incorrect +-- results. This test should fail if anyone ever does this without thinking too +-- hard about it. +-- [SPARK-28516] adds `to_char` +-- SELECT to_char(SUM(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING),'999999999999999999999D9') +-- FROM (VALUES(1,1e20),(2,1)) n(i,n); + +-- [SPARK-27880] Implement boolean aggregates(BOOL_AND, BOOL_OR and EVERY) +-- SELECT i, b, bool_and(b) OVER w, bool_or(b) OVER w +-- FROM (VALUES (1,true), (2,true), (3,false), (4,false), (5,true)) v(i,b) +-- WINDOW w AS (ORDER BY i ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING); + +-- Tests for problems with failure to walk or mutate expressions +-- within window frame clauses. + +-- [SPARK-37612] Support window frame ORDER BY i ROWS BETWEEN (('foo' < 'foobar')::integer) PRECEDING AND CURRENT ROW +-- test walker (fails with collation error if expressions are not walked) +-- SELECT array_agg(i) OVER w +-- FROM range(1,6) i +-- WINDOW w AS (ORDER BY i ROWS BETWEEN (('foo' < 'foobar')::integer) PRECEDING AND CURRENT ROW); + +-- Spark doesn't handle UDFs in SQL +-- test mutator (fails when inlined if expressions are not mutated) +-- CREATE FUNCTION pg_temp.f(group_size BIGINT) RETURNS SETOF integer[] +-- AS $$ +-- SELECT array_agg(s) OVER w +-- FROM generate_series(1,5) s +-- WINDOW w AS (ORDER BY s ROWS BETWEEN CURRENT ROW AND GROUP_SIZE FOLLOWING) +-- $$ LANGUAGE SQL STABLE; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/with.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/with.sql new file mode 100644 index 000000000000..a3e0b15b582f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/postgreSQL/with.sql @@ -0,0 +1,1208 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- WITH +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/with.sql +-- +-- This test uses the generate_series(...) function which is rewritten to EXPLODE(SEQUENCE(...)) as +-- it's feature tracking ticket SPARK-27767 is closed as Won't Do. + +-- +-- Tests for common table expressions (WITH query, ... SELECT ...) +-- + +-- Basic WITH +WITH q1(x,y) AS (SELECT 1,2) +SELECT * FROM q1, q1 AS q2; + +-- Multiple uses are evaluated only once +-- [SPARK-28299] Evaluation of multiple CTE uses +-- [ORIGINAL SQL] +--SELECT count(*) FROM ( +-- WITH q1(x) AS (SELECT random() FROM generate_series(1, 5)) +-- SELECT * FROM q1 +-- UNION +-- SELECT * FROM q1 +--) ss; +SELECT count(*) FROM ( + WITH q1(x) AS (SELECT rand() FROM (SELECT EXPLODE(SEQUENCE(1, 5)))) + SELECT * FROM q1 + UNION + SELECT * FROM q1 +) ss; + +-- WITH RECURSIVE + +-- sum of 1..100 +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM t WHERE n < 100 +--) +--SELECT sum(n) FROM t; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- SELECT (VALUES(1)) +--UNION ALL +-- SELECT n+1 FROM t WHERE n < 5 +--) +--SELECT * FROM t; + +-- recursive view +-- [SPARK-24497] Support recursive SQL query +--CREATE RECURSIVE VIEW nums (n) AS +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM nums WHERE n < 5; +-- +--SELECT * FROM nums; + +-- [SPARK-24497] Support recursive SQL query +--CREATE OR REPLACE RECURSIVE VIEW nums (n) AS +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM nums WHERE n < 6; +-- +--SELECT * FROM nums; + +-- This is an infinite loop with UNION ALL, but not with UNION +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- SELECT 1 +--UNION +-- SELECT 10-n FROM t) +--SELECT * FROM t; + +-- This'd be an infinite loop, but outside query reads only as much as needed +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM t) +--SELECT * FROM t LIMIT 10; + +-- UNION case should have same property +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- SELECT 1 +--UNION +-- SELECT n+1 FROM t) +--SELECT * FROM t LIMIT 10; + +-- Test behavior with an unknown-type literal in the WITH +-- [SPARK-28146] Support IS OF type predicate +--WITH q AS (SELECT 'foo' AS x) +--SELECT x, x IS OF (text) AS is_text FROM q; + +-- [SPARK-24497] Support recursive SQL query +-- [SPARK-28146] Support IS OF type predicate +--WITH RECURSIVE t(n) AS ( +-- SELECT 'foo' +--UNION ALL +-- SELECT n || ' bar' FROM t WHERE length(n) < 20 +--) +--SELECT n, n IS OF (text) AS is_text FROM t; + +-- In a perfect world, this would work and resolve the literal as int ... +-- but for now, we have to be content with resolving to text too soon. +-- [SPARK-24497] Support recursive SQL query +-- [SPARK-28146] Support IS OF type predicate +--WITH RECURSIVE t(n) AS ( +-- SELECT '7' +--UNION ALL +-- SELECT n+1 FROM t WHERE n < 10 +--) +--SELECT n, n IS OF (int) AS is_int FROM t; + +-- +-- Some examples with a tree +-- +-- department structure represented here is as follows: +-- +-- ROOT-+->A-+->B-+->C +-- | | +-- | +->D-+->F +-- +->E-+->G + + +-- [ORIGINAL SQL] +--CREATE TEMP TABLE department ( +-- id INTEGER PRIMARY KEY, -- department ID +-- parent_department INTEGER REFERENCES department, -- upper department ID +-- name string -- department name +--); +CREATE TABLE department ( + id INTEGER, -- department ID + parent_department INTEGER, -- upper department ID + name string -- department name +) USING parquet; + +INSERT INTO department VALUES (0, NULL, 'ROOT'); +INSERT INTO department VALUES (1, 0, 'A'); +INSERT INTO department VALUES (2, 1, 'B'); +INSERT INTO department VALUES (3, 2, 'C'); +INSERT INTO department VALUES (4, 2, 'D'); +INSERT INTO department VALUES (5, 0, 'E'); +INSERT INTO department VALUES (6, 4, 'F'); +INSERT INTO department VALUES (7, 5, 'G'); + + +-- extract all departments under 'A'. Result should be A, B, C, D and F +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE subdepartment AS +--( +-- -- non recursive term +-- SELECT name as root_name, * FROM department WHERE name = 'A' +-- +-- UNION ALL +-- +-- -- recursive term +-- SELECT sd.root_name, d.* FROM department AS d, subdepartment AS sd +-- WHERE d.parent_department = sd.id +--) +--SELECT * FROM subdepartment ORDER BY name; + +-- extract all departments under 'A' with "level" number +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE subdepartment(level, id, parent_department, name) AS +--( +-- -- non recursive term +-- SELECT 1, * FROM department WHERE name = 'A' +-- +-- UNION ALL +-- +-- -- recursive term +-- SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd +-- WHERE d.parent_department = sd.id +--) +--SELECT * FROM subdepartment ORDER BY name; + +-- extract all departments under 'A' with "level" number. +-- Only shows level 2 or more +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE subdepartment(level, id, parent_department, name) AS +--( +-- -- non recursive term +-- SELECT 1, * FROM department WHERE name = 'A' +-- +-- UNION ALL +-- +-- -- recursive term +-- SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd +-- WHERE d.parent_department = sd.id +--) +--SELECT * FROM subdepartment WHERE level >= 2 ORDER BY name; + +-- "RECURSIVE" is ignored if the query has no self-reference +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE subdepartment AS +--( +-- -- note lack of recursive UNION structure +-- SELECT * FROM department WHERE name = 'A' +--) +--SELECT * FROM subdepartment ORDER BY name; + +-- inside subqueries +-- [SPARK-24497] Support recursive SQL query +--SELECT count(*) FROM ( +-- WITH RECURSIVE t(n) AS ( +-- SELECT 1 UNION ALL SELECT n + 1 FROM t WHERE n < 500 +-- ) +-- SELECT * FROM t) AS t WHERE n < ( +-- SELECT count(*) FROM ( +-- WITH RECURSIVE t(n) AS ( +-- SELECT 1 UNION ALL SELECT n + 1 FROM t WHERE n < 100 +-- ) +-- SELECT * FROM t WHERE n < 50000 +-- ) AS t WHERE n < 100); + +-- use same CTE twice at different subquery levels +-- [SPARK-24497] Support recursive SQL query +--WITH q1(x,y) AS ( +-- SELECT hundred, sum(ten) FROM tenk1 GROUP BY hundred +-- ) +--SELECT count(*) FROM q1 WHERE y > (SELECT sum(y)/100 FROM q1 qsub); + +-- via a VIEW +-- [SPARK-24497] Support recursive SQL query +--CREATE TEMPORARY VIEW vsubdepartment AS +-- WITH RECURSIVE subdepartment AS +-- ( +-- -- non recursive term +-- SELECT * FROM department WHERE name = 'A' +-- UNION ALL +-- -- recursive term +-- SELECT d.* FROM department AS d, subdepartment AS sd +-- WHERE d.parent_department = sd.id +-- ) +-- SELECT * FROM subdepartment; +-- +--SELECT * FROM vsubdepartment ORDER BY name; +-- +---- Check reverse listing +--SELECT pg_get_viewdef('vsubdepartment'::regclass); +--SELECT pg_get_viewdef('vsubdepartment'::regclass, true); + +-- Another reverse-listing example +-- [SPARK-24497] Support recursive SQL query +--CREATE VIEW sums_1_100 AS +--WITH RECURSIVE t(n) AS ( +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM t WHERE n < 100 +--) +--SELECT sum(n) FROM t; +-- +--\d+ sums_1_100 + +-- corner case in which sub-WITH gets initialized first +-- [SPARK-24497] Support recursive SQL query +--with recursive q as ( +-- select * from department +-- union all +-- (with x as (select * from q) +-- select * from x) +-- ) +--select * from q limit 24; + +-- [SPARK-24497] Support recursive SQL query +--with recursive q as ( +-- select * from department +-- union all +-- (with recursive x as ( +-- select * from department +-- union all +-- (select * from q union all select * from x) +-- ) +-- select * from x) +-- ) +--select * from q limit 32; + +-- recursive term has sub-UNION +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(i,j) AS ( +-- VALUES (1,2) +-- UNION ALL +-- SELECT t2.i, t.j+1 FROM +-- (SELECT 2 AS i UNION ALL SELECT 3 AS i) AS t2 +-- JOIN t ON (t2.i = t.i+1)) +-- +-- SELECT * FROM t; + +-- +-- different tree example +-- +-- [ORIGINAL SQL] +--CREATE TEMPORARY TABLE tree( +-- id INTEGER PRIMARY KEY, +-- parent_id INTEGER REFERENCES tree(id) +--); +CREATE TABLE tree( + id INTEGER, + parent_id INTEGER +) USING parquet; + +INSERT INTO tree +VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3), + (9,4), (10,4), (11,7), (12,7), (13,7), (14, 9), (15,11), (16,11); + +-- +-- get all paths from "second level" nodes to leaf nodes +-- +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(id, path) AS ( +-- VALUES(1,ARRAY[]::integer[]) +--UNION ALL +-- SELECT tree.id, t.path || tree.id +-- FROM tree JOIN t ON (tree.parent_id = t.id) +--) +--SELECT t1.*, t2.* FROM t AS t1 JOIN t AS t2 ON +-- (t1.path[1] = t2.path[1] AND +-- array_upper(t1.path,1) = 1 AND +-- array_upper(t2.path,1) > 1) +-- ORDER BY t1.id, t2.id; + +-- just count 'em +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(id, path) AS ( +-- VALUES(1,ARRAY[]::integer[]) +--UNION ALL +-- SELECT tree.id, t.path || tree.id +-- FROM tree JOIN t ON (tree.parent_id = t.id) +--) +--SELECT t1.id, count(t2.*) FROM t AS t1 JOIN t AS t2 ON +-- (t1.path[1] = t2.path[1] AND +-- array_upper(t1.path,1) = 1 AND +-- array_upper(t2.path,1) > 1) +-- GROUP BY t1.id +-- ORDER BY t1.id; + +-- this variant tickled a whole-row-variable bug in 8.4devel +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(id, path) AS ( +-- VALUES(1,ARRAY[]::integer[]) +--UNION ALL +-- SELECT tree.id, t.path || tree.id +-- FROM tree JOIN t ON (tree.parent_id = t.id) +--) +--SELECT t1.id, t2.path, t2 FROM t AS t1 JOIN t AS t2 ON +--(t1.id=t2.id); + +-- +-- test cycle detection +-- +-- [ORIGINAL SQL] +--create temp table graph( f int, t int, label text ); +create table graph( f int, t int, label string ) USING parquet; + +insert into graph values + (1, 2, 'arc 1 -> 2'), + (1, 3, 'arc 1 -> 3'), + (2, 3, 'arc 2 -> 3'), + (1, 4, 'arc 1 -> 4'), + (4, 5, 'arc 4 -> 5'), + (5, 1, 'arc 5 -> 1'); + +-- [SPARK-24497] Support recursive SQL query +--with recursive search_graph(f, t, label, path, cycle) as ( +-- select *, array[row(g.f, g.t)], false from graph g +-- union all +-- select g.*, path || row(g.f, g.t), row(g.f, g.t) = any(path) +-- from graph g, search_graph sg +-- where g.f = sg.t and not cycle +--) +--select * from search_graph; + +-- ordering by the path column has same effect as SEARCH DEPTH FIRST +-- [SPARK-24497] Support recursive SQL query +--with recursive search_graph(f, t, label, path, cycle) as ( +-- select *, array[row(g.f, g.t)], false from graph g +-- union all +-- select g.*, path || row(g.f, g.t), row(g.f, g.t) = any(path) +-- from graph g, search_graph sg +-- where g.f = sg.t and not cycle +--) +--select * from search_graph order by path; + +-- +-- test multiple WITH queries +-- +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- y (id) AS (VALUES (1)), +-- x (id) AS (SELECT * FROM y UNION ALL SELECT id+1 FROM x WHERE id < 5) +--SELECT * FROM x; + +-- forward reference OK +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS (SELECT * FROM y UNION ALL SELECT id+1 FROM x WHERE id < 5), +-- y(id) AS (values (1)) +-- SELECT * FROM x; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS +-- (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 5), +-- y(id) AS +-- (VALUES (1) UNION ALL SELECT id+1 FROM y WHERE id < 10) +-- SELECT y.*, x.* FROM y LEFT JOIN x USING (id); + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS +-- (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 5), +-- y(id) AS +-- (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 10) +-- SELECT y.*, x.* FROM y LEFT JOIN x USING (id); + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS +-- (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 3 ), +-- y(id) AS +-- (SELECT * FROM x UNION ALL SELECT * FROM x), +-- z(id) AS +-- (SELECT * FROM x UNION ALL SELECT id+1 FROM z WHERE id < 10) +-- SELECT * FROM z; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS +-- (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 3 ), +-- y(id) AS +-- (SELECT * FROM x UNION ALL SELECT * FROM x), +-- z(id) AS +-- (SELECT * FROM y UNION ALL SELECT id+1 FROM z WHERE id < 10) +-- SELECT * FROM z; + +-- +-- Test WITH attached to a data-modifying statement +-- + +-- [ORIGINAL SQL] +--CREATE TEMPORARY TABLE y (a INTEGER); +CREATE TABLE y (a INTEGER) USING parquet; +-- [ORIGINAL SQL] +--INSERT INTO y SELECT generate_series(1, 10); +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)); + +-- [SPARK-28147] Support RETURNING clause +--WITH t AS ( +-- SELECT a FROM y +--) +--INSERT INTO y +--SELECT a+20 FROM t RETURNING *; +-- +--SELECT * FROM y; + +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH t AS ( +-- SELECT a FROM y +--) +--UPDATE y SET a = y.a-10 FROM t WHERE y.a > 20 AND t.a = y.a RETURNING y.a; +-- +--SELECT * FROM y; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH RECURSIVE t(a) AS ( +-- SELECT 11 +-- UNION ALL +-- SELECT a+1 FROM t WHERE a < 50 +--) +--DELETE FROM y USING t WHERE t.a = y.a RETURNING y.a; +-- +--SELECT * FROM y; + +DROP TABLE y; + +-- +-- error cases +-- + +-- INTERSECT +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT SELECT n+1 FROM x) +-- SELECT * FROM x; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT ALL SELECT n+1 FROM x) +-- SELECT * FROM x; + +-- EXCEPT +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT SELECT n+1 FROM x) +-- SELECT * FROM x; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT ALL SELECT n+1 FROM x) +-- SELECT * FROM x; + +-- no non-recursive term +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT n FROM x) +-- SELECT * FROM x; + +-- recursive term in the left hand side (strictly speaking, should allow this) +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT n FROM x UNION ALL SELECT 1) +-- SELECT * FROM x; + +-- [ORIGINAL SQL] +--CREATE TEMPORARY TABLE y (a INTEGER); +CREATE TABLE y (a INTEGER) USING parquet; +-- [ORIGINAL SQL] +--INSERT INTO y SELECT generate_series(1, 10); +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)); + +-- LEFT JOIN + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 +-- UNION ALL +-- SELECT x.n+1 FROM y LEFT JOIN x ON x.n = y.a WHERE n < 10) +--SELECT * FROM x; + +-- RIGHT JOIN +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 +-- UNION ALL +-- SELECT x.n+1 FROM x RIGHT JOIN y ON x.n = y.a WHERE n < 10) +--SELECT * FROM x; + +-- FULL JOIN +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 +-- UNION ALL +-- SELECT x.n+1 FROM x FULL JOIN y ON x.n = y.a WHERE n < 10) +--SELECT * FROM x; + +-- subquery +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x +-- WHERE n IN (SELECT * FROM x)) +-- SELECT * FROM x; + +-- aggregate functions +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT count(*) FROM x) +-- SELECT * FROM x; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT sum(n) FROM x) +-- SELECT * FROM x; + +-- ORDER BY +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x ORDER BY 1) +-- SELECT * FROM x; + +-- LIMIT/OFFSET +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x LIMIT 10 OFFSET 1) +-- SELECT * FROM x; + +-- FOR UPDATE +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x FOR UPDATE) +-- SELECT * FROM x; + +-- target list has a recursive query name +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(id) AS (values (1) +-- UNION ALL +-- SELECT (SELECT * FROM x) FROM x WHERE id < 5 +--) SELECT * FROM x; + +-- mutual recursive query (not implemented) +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM y WHERE id < 5), +-- y (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 5) +--SELECT * FROM x; + +-- non-linear recursion is not allowed +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (values (1) +-- UNION ALL +-- (SELECT i+1 FROM foo WHERE i < 10 +-- UNION ALL +-- SELECT i+1 FROM foo WHERE i < 5) +--) SELECT * FROM foo; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (values (1) +-- UNION ALL +-- SELECT * FROM +-- (SELECT i+1 FROM foo WHERE i < 10 +-- UNION ALL +-- SELECT i+1 FROM foo WHERE i < 5) AS t +--) SELECT * FROM foo; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (values (1) +-- UNION ALL +-- (SELECT i+1 FROM foo WHERE i < 10 +-- EXCEPT +-- SELECT i+1 FROM foo WHERE i < 5) +--) SELECT * FROM foo; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (values (1) +-- UNION ALL +-- (SELECT i+1 FROM foo WHERE i < 10 +-- INTERSECT +-- SELECT i+1 FROM foo WHERE i < 5) +--) SELECT * FROM foo; + +-- Wrong type induced from non-recursive term +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (SELECT i FROM (VALUES(1),(2)) t(i) +-- UNION ALL +-- SELECT (i+1)::numeric(10,0) FROM foo WHERE i < 10) +--SELECT * FROM foo; + +-- rejects different typmod, too (should we allow this?) +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (SELECT i::numeric(3,0) FROM (VALUES(1),(2)) t(i) +-- UNION ALL +-- SELECT (i+1)::numeric(10,0) FROM foo WHERE i < 10) +--SELECT * FROM foo; + +-- [NOTE] Spark SQL doesn't support RULEs +-- disallow OLD/NEW reference in CTE +--CREATE TABLE x (n integer) USING parquet; +--CREATE RULE r2 AS ON UPDATE TO x DO INSTEAD +-- WITH t AS (SELECT OLD.*) UPDATE y SET a = t.n FROM t; + +-- +-- test for bug #4902 +-- +-- [SPARK-28296] Improved VALUES support +--with cte(foo) as ( values(42) ) values((select foo from cte)); +with cte(foo) as ( select 42 ) select * from ((select foo from cte)) q; + +-- test CTE referencing an outer-level variable (to see that changed-parameter +-- signaling still works properly after fixing this bug) +-- [SPARK-28296] Improved VALUES support +-- [SPARK-28297] Handling outer links in CTE subquery expressions +--select ( with cte(foo) as ( values(f1) ) +-- select (select foo from cte) ) +--from int4_tbl; + +-- [SPARK-28296] Improved VALUES support +-- [SPARK-28297] Handling outer links in CTE subquery expressions +--select ( with cte(foo) as ( values(f1) ) +-- values((select foo from cte)) ) +--from int4_tbl; + +-- +-- test for nested-recursive-WITH bug +-- +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(j) AS ( +-- WITH RECURSIVE s(i) AS ( +-- VALUES (1) +-- UNION ALL +-- SELECT i+1 FROM s WHERE i < 10 +-- ) +-- SELECT i FROM s +-- UNION ALL +-- SELECT j+1 FROM t WHERE j < 10 +--) +--SELECT * FROM t; + +-- +-- test WITH attached to intermediate-level set operation +-- + +WITH outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM innermost + UNION SELECT 3) +) +SELECT * FROM outermost ORDER BY 1; + +WITH outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM outermost -- fail + UNION SELECT * FROM innermost) +) +SELECT * FROM outermost ORDER BY 1; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE outermost(x) AS ( +-- SELECT 1 +-- UNION (WITH innermost as (SELECT 2) +-- SELECT * FROM outermost +-- UNION SELECT * FROM innermost) +--) +--SELECT * FROM outermost ORDER BY 1; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE outermost(x) AS ( +-- WITH innermost as (SELECT 2 FROM outermost) -- fail +-- SELECT * FROM innermost +-- UNION SELECT * from outermost +--) +--SELECT * FROM outermost ORDER BY 1; + +-- +-- This test will fail with the old implementation of PARAM_EXEC parameter +-- assignment, because the "q1" Var passed down to A's targetlist subselect +-- looks exactly like the "A.id" Var passed down to C's subselect, causing +-- the old code to give them the same runtime PARAM_EXEC slot. But the +-- lifespans of the two parameters overlap, thanks to B also reading A. +-- + +-- [SPARK-27878] Support ARRAY(sub-SELECT) expressions +--with +--A as ( select q2 as id, (select q1) as x from int8_tbl ), +--B as ( select id, row_number() over (partition by id) as r from A ), +--C as ( select A.id, array(select B.id from B where B.id = A.id) from A ) +--select * from C; + +-- +-- Test CTEs read in non-initialization orders +-- + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- tab(id_key,link) AS (VALUES (1,17), (2,17), (3,17), (4,17), (6,17), (5,17)), +-- iter (id_key, row_type, link) AS ( +-- SELECT 0, 'base', 17 +-- UNION ALL ( +-- WITH remaining(id_key, row_type, link, min) AS ( +-- SELECT tab.id_key, 'true'::text, iter.link, MIN(tab.id_key) OVER () +-- FROM tab INNER JOIN iter USING (link) +-- WHERE tab.id_key > iter.id_key +-- ), +-- first_remaining AS ( +-- SELECT id_key, row_type, link +-- FROM remaining +-- WHERE id_key=min +-- ), +-- effect AS ( +-- SELECT tab.id_key, 'new'::text, tab.link +-- FROM first_remaining e INNER JOIN tab ON e.id_key=tab.id_key +-- WHERE e.row_type = 'false' +-- ) +-- SELECT * FROM first_remaining +-- UNION ALL SELECT * FROM effect +-- ) +-- ) +--SELECT * FROM iter; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- tab(id_key,link) AS (VALUES (1,17), (2,17), (3,17), (4,17), (6,17), (5,17)), +-- iter (id_key, row_type, link) AS ( +-- SELECT 0, 'base', 17 +-- UNION ( +-- WITH remaining(id_key, row_type, link, min) AS ( +-- SELECT tab.id_key, 'true'::text, iter.link, MIN(tab.id_key) OVER () +-- FROM tab INNER JOIN iter USING (link) +-- WHERE tab.id_key > iter.id_key +-- ), +-- first_remaining AS ( +-- SELECT id_key, row_type, link +-- FROM remaining +-- WHERE id_key=min +-- ), +-- effect AS ( +-- SELECT tab.id_key, 'new'::text, tab.link +-- FROM first_remaining e INNER JOIN tab ON e.id_key=tab.id_key +-- WHERE e.row_type = 'false' +-- ) +-- SELECT * FROM first_remaining +-- UNION ALL SELECT * FROM effect +-- ) +-- ) +--SELECT * FROM iter; + +-- +-- Data-modifying statements in WITH +-- + +-- INSERT ... RETURNING +-- [SPARK-28147] Support RETURNING clause +--WITH t AS ( +-- INSERT INTO y +-- VALUES +-- (11), +-- (12), +-- (13), +-- (14), +-- (15), +-- (16), +-- (17), +-- (18), +-- (19), +-- (20) +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; + +-- UPDATE ... RETURNING +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH t AS ( +-- UPDATE y +-- SET a=a+1 +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; + +-- DELETE ... RETURNING +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t AS ( +-- DELETE FROM y +-- WHERE a <= 10 +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; + +-- forward reference +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH RECURSIVE t AS ( +-- INSERT INTO y +-- SELECT a+5 FROM t2 WHERE a > 5 +-- RETURNING * +--), t2 AS ( +-- UPDATE y SET a=a-11 RETURNING * +--) +--SELECT * FROM t +--UNION ALL +--SELECT * FROM t2; +-- +--SELECT * FROM y; + +-- unconditional DO INSTEAD rule +-- [NOTE] Spark SQL doesn't support RULEs +--CREATE RULE y_rule AS ON DELETE TO y DO INSTEAD +-- INSERT INTO y VALUES(42) RETURNING *; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t AS ( +-- DELETE FROM y RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; + +--DROP RULE y_rule ON y; + +-- check merging of outer CTE with CTE in a rule action +--CREATE TEMP TABLE bug6051 AS +-- select i from generate_series(1,3) as t(i); + +--SELECT * FROM bug6051; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t1 AS ( DELETE FROM bug6051 RETURNING * ) +--INSERT INTO bug6051 SELECT * FROM t1; +-- +--SELECT * FROM bug6051; + +-- [NOTE] Spark SQL doesn't support RULEs +--CREATE TEMP TABLE bug6051_2 (i int); +-- +--CREATE RULE bug6051_ins AS ON INSERT TO bug6051 DO INSTEAD +-- INSERT INTO bug6051_2 +-- SELECT NEW.i; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t1 AS ( DELETE FROM bug6051 RETURNING * ) +--INSERT INTO bug6051 SELECT * FROM t1; +-- +--SELECT * FROM bug6051; +--SELECT * FROM bug6051_2; + +-- a truly recursive CTE in the same list +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(a) AS ( +-- SELECT 0 +-- UNION ALL +-- SELECT a+1 FROM t WHERE a+1 < 5 +--), t2 as ( +-- INSERT INTO y +-- SELECT * FROM t RETURNING * +--) +--SELECT * FROM t2 JOIN y USING (a) ORDER BY a; +-- +--SELECT * FROM y; + +-- data-modifying WITH in a modifying statement +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t AS ( +-- DELETE FROM y +-- WHERE a <= 10 +-- RETURNING * +--) +--INSERT INTO y SELECT -a FROM t RETURNING *; +-- +--SELECT * FROM y; + +-- check that WITH query is run to completion even if outer query isn't +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH t AS ( +-- UPDATE y SET a = a * 100 RETURNING * +--) +--SELECT * FROM t LIMIT 10; +-- +--SELECT * FROM y; + +-- data-modifying WITH containing INSERT...ON CONFLICT DO UPDATE +-- [ORIGINAL SQL] +--CREATE TABLE withz AS SELECT i AS k, (i || ' v')::text v FROM generate_series(1, 16, 3) i; +CREATE TABLE withz USING parquet AS SELECT i AS k, CAST(i AS string) || ' v' AS v FROM (SELECT EXPLODE(SEQUENCE(1, 16, 3)) i); +-- [NOTE] Spark SQL doesn't support UNIQUE constraints +--ALTER TABLE withz ADD UNIQUE (k); + +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH t AS ( +-- INSERT INTO withz SELECT i, 'insert' +-- FROM generate_series(0, 16) i +-- ON CONFLICT (k) DO UPDATE SET v = withz.v || ', now update' +-- RETURNING * +--) +--SELECT * FROM t JOIN y ON t.k = y.a ORDER BY a, k; + +-- Test EXCLUDED.* reference within CTE +-- [NOTE] Spark SQL doesn't support ON CONFLICT clause +--WITH aa AS ( +-- INSERT INTO withz VALUES(1, 5) ON CONFLICT (k) DO UPDATE SET v = EXCLUDED.v +-- WHERE withz.k != EXCLUDED.k +-- RETURNING * +--) +--SELECT * FROM aa; + +-- New query/snapshot demonstrates side-effects of previous query. +SELECT * FROM withz ORDER BY k; + +-- +-- Ensure subqueries within the update clause work, even if they +-- reference outside values +-- +-- [NOTE] Spark SQL doesn't support ON CONFLICT clause +--WITH aa AS (SELECT 1 a, 2 b) +--INSERT INTO withz VALUES(1, 'insert') +--ON CONFLICT (k) DO UPDATE SET v = (SELECT b || ' update' FROM aa WHERE a = 1 LIMIT 1); +--WITH aa AS (SELECT 1 a, 2 b) +--INSERT INTO withz VALUES(1, 'insert') +--ON CONFLICT (k) DO UPDATE SET v = ' update' WHERE withz.k = (SELECT a FROM aa); +--WITH aa AS (SELECT 1 a, 2 b) +--INSERT INTO withz VALUES(1, 'insert') +--ON CONFLICT (k) DO UPDATE SET v = (SELECT b || ' update' FROM aa WHERE a = 1 LIMIT 1); +--WITH aa AS (SELECT 'a' a, 'b' b UNION ALL SELECT 'a' a, 'b' b) +--INSERT INTO withz VALUES(1, 'insert') +--ON CONFLICT (k) DO UPDATE SET v = (SELECT b || ' update' FROM aa WHERE a = 'a' LIMIT 1); +--WITH aa AS (SELECT 1 a, 2 b) +--INSERT INTO withz VALUES(1, (SELECT b || ' insert' FROM aa WHERE a = 1 )) +--ON CONFLICT (k) DO UPDATE SET v = (SELECT b || ' update' FROM aa WHERE a = 1 LIMIT 1); + +-- Update a row more than once, in different parts of a wCTE. That is +-- an allowed, presumably very rare, edge case, but since it was +-- broken in the past, having a test seems worthwhile. +-- [NOTE] Spark SQL doesn't support ON CONFLICT clause +--WITH simpletup AS ( +-- SELECT 2 k, 'Green' v), +--upsert_cte AS ( +-- INSERT INTO withz VALUES(2, 'Blue') ON CONFLICT (k) DO +-- UPDATE SET (k, v) = (SELECT k, v FROM simpletup WHERE simpletup.k = withz.k) +-- RETURNING k, v) +--INSERT INTO withz VALUES(2, 'Red') ON CONFLICT (k) DO +--UPDATE SET (k, v) = (SELECT k, v FROM upsert_cte WHERE upsert_cte.k = withz.k) +--RETURNING k, v; + +DROP TABLE withz; + +-- check that run to completion happens in proper ordering + +TRUNCATE TABLE y; +-- [ORIGINAL SQL] +--INSERT INTO y SELECT generate_series(1, 3); +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 3)); +-- [ORIGINAL SQL] +--CREATE TEMPORARY TABLE yy (a INTEGER); +CREATE TABLE yy (a INTEGER) USING parquet; + +-- [SPARK-24497] Support recursive SQL query +-- [SPARK-28147] Support RETURNING clause +--WITH RECURSIVE t1 AS ( +-- INSERT INTO y SELECT * FROM y RETURNING * +--), t2 AS ( +-- INSERT INTO yy SELECT * FROM t1 RETURNING * +--) +--SELECT 1; + +SELECT * FROM y; +SELECT * FROM yy; + +-- [SPARK-24497] Support recursive SQL query +-- [SPARK-28147] Support RETURNING clause +--WITH RECURSIVE t1 AS ( +-- INSERT INTO yy SELECT * FROM t2 RETURNING * +--), t2 AS ( +-- INSERT INTO y SELECT * FROM y RETURNING * +--) +--SELECT 1; + +SELECT * FROM y; +SELECT * FROM yy; + +-- [NOTE] Spark SQL doesn't support TRIGGERs +-- triggers +-- +--TRUNCATE TABLE y; +--INSERT INTO y SELECT generate_series(1, 10); +-- +--CREATE FUNCTION y_trigger() RETURNS trigger AS $$ +--begin +-- raise notice 'y_trigger: a = %', new.a; +-- return new; +--end; +--$$ LANGUAGE plpgsql; +-- +-- +--CREATE TRIGGER y_trig BEFORE INSERT ON y FOR EACH ROW +-- EXECUTE PROCEDURE y_trigger(); +-- +--WITH t AS ( +-- INSERT INTO y +-- VALUES +-- (21), +-- (22), +-- (23) +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; +-- +--DROP TRIGGER y_trig ON y; +-- +--CREATE TRIGGER y_trig AFTER INSERT ON y FOR EACH ROW +-- EXECUTE PROCEDURE y_trigger(); +-- +--WITH t AS ( +-- INSERT INTO y +-- VALUES +-- (31), +-- (32), +-- (33) +-- RETURNING * +--) +--SELECT * FROM t LIMIT 1; +-- +--SELECT * FROM y; +-- +--DROP TRIGGER y_trig ON y; +-- +--CREATE OR REPLACE FUNCTION y_trigger() RETURNS trigger AS $$ +--begin +-- raise notice 'y_trigger'; +-- return null; +--end; +--$$ LANGUAGE plpgsql; +-- +--CREATE TRIGGER y_trig AFTER INSERT ON y FOR EACH STATEMENT +-- EXECUTE PROCEDURE y_trigger(); +-- +--WITH t AS ( +-- INSERT INTO y +-- VALUES +-- (41), +-- (42), +-- (43) +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; +-- +--DROP TRIGGER y_trig ON y; +--DROP FUNCTION y_trigger(); + +-- WITH attached to inherited UPDATE or DELETE + +-- [ORIGINAL SQL] +--CREATE TEMP TABLE parent ( id int, val text ); +CREATE TABLE parent ( id int, val string ) USING parquet; +-- [NOTE] Spark SQL doesn't support INHERITS clause +--CREATE TEMP TABLE child1 ( ) INHERITS ( parent ); +-- [NOTE] Spark SQL doesn't support INHERITS clause +--CREATE TEMP TABLE child2 ( ) INHERITS ( parent ); + +INSERT INTO parent VALUES ( 1, 'p1' ); +--INSERT INTO child1 VALUES ( 11, 'c11' ),( 12, 'c12' ); +--INSERT INTO child2 VALUES ( 23, 'c21' ),( 24, 'c22' ); + +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH rcte AS ( SELECT sum(id) AS totalid FROM parent ) +--UPDATE parent SET id = id + totalid FROM rcte; + +SELECT * FROM parent; + +-- [SPARK-28147] Support RETURNING clause +--WITH wcte AS ( INSERT INTO child1 VALUES ( 42, 'new' ) RETURNING id AS newid ) +--UPDATE parent SET id = id + newid FROM wcte; +-- +--SELECT * FROM parent; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH rcte AS ( SELECT max(id) AS maxid FROM parent ) +--DELETE FROM parent USING rcte WHERE id = maxid; + +SELECT * FROM parent; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH wcte AS ( INSERT INTO child2 VALUES ( 42, 'new2' ) RETURNING id AS newid ) +--DELETE FROM parent USING wcte WHERE id = newid; +-- +--SELECT * FROM parent; + +-- check EXPLAIN VERBOSE for a wCTE with RETURNING + +-- [NOTE] Spark SQL doesn't support DELETE statement +--EXPLAIN (VERBOSE, COSTS OFF) +--WITH wcte AS ( INSERT INTO int8_tbl VALUES ( 42, 47 ) RETURNING q2 ) +--DELETE FROM a USING wcte WHERE aa = q2; + +-- error cases + +-- data-modifying WITH tries to use its own output +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t AS ( +-- INSERT INTO y +-- SELECT * FROM t +--) +--VALUES(FALSE); + +-- no RETURNING in a referenced data-modifying WITH +-- [SPARK-24497] Support recursive SQL query +--WITH t AS ( +-- INSERT INTO y VALUES(0) +--) +--SELECT * FROM t; + +-- data-modifying WITH allowed only at the top level +-- [SPARK-28147] Support RETURNING clause +--SELECT * FROM ( +-- WITH t AS (UPDATE y SET a=a+1 RETURNING *) +-- SELECT * FROM t +--) ss; + +-- most variants of rules aren't allowed +-- [NOTE] Spark SQL doesn't support RULEs +--CREATE RULE y_rule AS ON INSERT TO y WHERE a=0 DO INSTEAD DELETE FROM y; +--WITH t AS ( +-- INSERT INTO y VALUES(0) +--) +--VALUES(FALSE); +--DROP RULE y_rule ON y; + +-- check that parser lookahead for WITH doesn't cause any odd behavior +create table foo (with baz); -- fail, WITH is a reserved word +create table foo (with ordinality); -- fail, WITH is a reserved word +with ordinality as (select 1 as x) select * from ordinality; + +-- check sane response to attempt to modify CTE relation +WITH test AS (SELECT 42) INSERT INTO test VALUES (1); + +-- check response to attempt to modify table with same name as a CTE (perhaps +-- surprisingly it works, because CTEs don't hide tables from data-modifying +-- statements) +-- [ORIGINAL SQL] +--create temp table test (i int); +create table test (i int) USING parquet; +with test as (select 42) insert into test select * from test; +select * from test; +drop table test; + +-- +-- Clean up +-- + +DROP TABLE department; +DROP TABLE tree; +DROP TABLE graph; +DROP TABLE y; +DROP TABLE yy; +DROP TABLE parent; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/pred-pushdown.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/pred-pushdown.sql new file mode 100644 index 000000000000..eff258a06635 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/pred-pushdown.sql @@ -0,0 +1,12 @@ +CREATE OR REPLACE TEMPORARY VIEW tbl_a AS VALUES (1, 1), (2, 1), (3, 6) AS T(c1, c2); +CREATE OR REPLACE TEMPORARY VIEW tbl_b AS VALUES 1 AS T(c1); + +-- SPARK-18597: Do not push down predicates to left hand side in an anti-join +SELECT * +FROM tbl_a + LEFT ANTI JOIN tbl_b ON ((tbl_a.c1 = tbl_a.c2) IS NULL OR tbl_a.c1 = tbl_a.c2); + +-- SPARK-18614: Do not push down predicates on left table below ExistenceJoin +SELECT l.c1, l.c2 +FROM tbl_a l +WHERE EXISTS (SELECT 1 FROM tbl_b r WHERE l.c1 = l.c2) OR l.c2 < 2; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/predicate-functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/predicate-functions.sql new file mode 100644 index 000000000000..195db17a3a1f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/predicate-functions.sql @@ -0,0 +1,120 @@ +-- NOT +select not true; +select ! true; +select not null::boolean; + +-- AND +select true and true; +select true and false; +select false and true; +select false and false; +select true and null::boolean; +select false and null::boolean; +select null::boolean and true; +select null::boolean and false; +select null::boolean and null::boolean; + +-- OR +select true or true; +select true or false; +select false or true; +select false or false; +select true or null::boolean; +select false or null::boolean; +select null::boolean or true; +select null::boolean or false; +select null::boolean or null::boolean; + +-- EqualTo +select 1 = 1; +select 1 = '1'; +select 1.0 = '1'; +select 1.5 = '1.51'; + +-- GreaterThan +select 1 > '1'; +select 2 > '1.0'; +select 2 > '2.0'; +select 2 > '2.2'; +select '1.5' > 0.5; +select to_date('2009-07-30 04:17:52') > to_date('2009-07-30 04:17:52'); +select to_date('2009-07-30 04:17:52') > '2009-07-30 04:17:52'; + +-- GreaterThanOrEqual +select 1 >= '1'; +select 2 >= '1.0'; +select 2 >= '2.0'; +select 2.0 >= '2.2'; +select '1.5' >= 0.5; +select to_date('2009-07-30 04:17:52') >= to_date('2009-07-30 04:17:52'); +select to_date('2009-07-30 04:17:52') >= '2009-07-30 04:17:52'; + +-- LessThan +select 1 < '1'; +select 2 < '1.0'; +select 2 < '2.0'; +select 2.0 < '2.2'; +select 0.5 < '1.5'; +select to_date('2009-07-30 04:17:52') < to_date('2009-07-30 04:17:52'); +select to_date('2009-07-30 04:17:52') < '2009-07-30 04:17:52'; + +-- LessThanOrEqual +select 1 <= '1'; +select 2 <= '1.0'; +select 2 <= '2.0'; +select 2.0 <= '2.2'; +select 0.5 <= '1.5'; +select to_date('2009-07-30 04:17:52') <= to_date('2009-07-30 04:17:52'); +select to_date('2009-07-30 04:17:52') <= '2009-07-30 04:17:52'; + +-- SPARK-23549: Cast to timestamp when comparing timestamp with date +select to_date('2017-03-01') = to_timestamp('2017-03-01 00:00:00'); +select to_timestamp('2017-03-01 00:00:01') > to_date('2017-03-01'); +select to_timestamp('2017-03-01 00:00:01') >= to_date('2017-03-01'); +select to_date('2017-03-01') < to_timestamp('2017-03-01 00:00:01'); +select to_date('2017-03-01') <= to_timestamp('2017-03-01 00:00:01'); + +-- In +select 1 in (1, 2, 3); +select 1 in (1, 2, 3, null); +select 1 in (1.0, 2.0, 3.0); +select 1 in (1.0, 2.0, 3.0, null); +select 1 in ('2', '3', '4'); +select 1 in ('2', '3', '4', null); +select null in (1, 2, 3); +select null in (1, 2, null); + +-- Not(In) +select 1 not in (1, 2, 3); +select 1 not in (1, 2, 3, null); +select 1 not in (1.0, 2.0, 3.0); +select 1 not in (1.0, 2.0, 3.0, null); +select 1 not in ('2', '3', '4'); +select 1 not in ('2', '3', '4', null); +select null not in (1, 2, 3); +select null not in (1, 2, null); + +-- Between +select 1 between 0 and 2; +select 0.5 between 0 and 1; +select 2.0 between '1.0' and '3.0'; +select 'b' between 'a' and 'c'; +select to_timestamp('2022-12-26 00:00:01') between to_date('2022-03-01') and to_date('2022-12-31'); +select rand(123) between 0.1 AND 0.2; + +-- Not(Between) +select 1 not between 0 and 2; +select 0.5 not between 0 and 1; +select 2.0 not between '1.0' and '3.0'; +select 'b' not between 'a' and 'c'; +select to_timestamp('2022-12-26 00:00:01') not between to_date('2022-03-01') and to_date('2022-12-31'); +select rand(123) not between 0.1 AND 0.2; + +-- Sanity test for legacy flag equating ! with NOT +set spark.sql.legacy.bangEqualsNot=true; +select 1 ! between 0 and 2; +select 1 ! in (3, 4); +select 'hello' ! like 'world'; +select 1 is ! null; +select false is ! true; +set spark.sql.legacy.bangEqualsNot=false; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/query_regex_column.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/query_regex_column.sql new file mode 100644 index 000000000000..ad96754826a4 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/query_regex_column.sql @@ -0,0 +1,52 @@ +set spark.sql.parser.quotedRegexColumnNames=false; + +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, "1", "11"), (2, "2", "22"), (3, "3", "33"), (4, "4", "44"), (5, "5", "55"), (6, "6", "66") +AS testData(key, value1, value2); + +CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES +(1, 1, 1, 2), (1, 2, 1, 2), (2, 1, 2, 3), (2, 2, 2, 3), (3, 1, 3, 4), (3, 2, 3, 4) +AS testData2(A, B, c, d); + +-- AnalysisException +SELECT `(a)?+.+` FROM testData2 WHERE a = 1; +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1; +SELECT `(a|b)` FROM testData2 WHERE a = 2; +SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2; +SELECT SUM(`(a|b)?+.+`) FROM testData2; +SELECT SUM(`(a)`) FROM testData2; + +set spark.sql.parser.quotedRegexColumnNames=true; + +-- Regex columns +SELECT `(a)?+.+` FROM testData2 WHERE a = 1; +SELECT `(A)?+.+` FROM testData2 WHERE a = 1; +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1; +SELECT t.`(A)?+.+` FROM testData2 t WHERE a = 1; +SELECT `(a|B)` FROM testData2 WHERE a = 2; +SELECT `(A|b)` FROM testData2 WHERE a = 2; +SELECT `(a|B)?+.+` FROM testData2 WHERE a = 2; +SELECT `(A|b)?+.+` FROM testData2 WHERE a = 2; +SELECT `(e|f)` FROM testData2; +SELECT t.`(e|f)` FROM testData2 t; +SELECT p.`(KEY)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3; +SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3; + +set spark.sql.caseSensitive=true; + +CREATE OR REPLACE TEMPORARY VIEW testdata3 AS SELECT * FROM VALUES +(0, 1), (1, 2), (2, 3), (3, 4) +AS testdata3(a, b); + +-- Regex columns +SELECT `(A)?+.+` FROM testdata3; +SELECT `(a)?+.+` FROM testdata3; +SELECT `(A)?+.+` FROM testdata3 WHERE a > 1; +SELECT `(a)?+.+` FROM testdata3 where `a` > 1; +SELECT SUM(`a`) FROM testdata3; +SELECT SUM(`(a)`) FROM testdata3; +SELECT SUM(`(a)?+.+`) FROM testdata3; +SELECT SUM(a) FROM testdata3 GROUP BY `a`; +-- AnalysisException +SELECT SUM(a) FROM testdata3 GROUP BY `(a)`; +SELECT SUM(a) FROM testdata3 GROUP BY `(a)?+.+`; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/random.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/random.sql new file mode 100644 index 000000000000..95be99595cc8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/random.sql @@ -0,0 +1,89 @@ +-- rand with the seed 0 +SELECT rand(0); +SELECT rand(cast(3 / 7 AS int)); +SELECT rand(NULL); +SELECT rand(cast(NULL AS int)); + +-- rand unsupported data type +SELECT rand(1.0); + +-- randn with the seed 0 +SELECT randn(0L); +SELECT randn(cast(3 / 7 AS long)); +SELECT randn(NULL); +SELECT randn(cast(NULL AS long)); + +-- randn unsupported data type +SELECT rand('1'); + +-- The uniform random number generation function supports generating random numbers within a +-- specified range. We use a seed of zero for these queries to keep tests deterministic. +SELECT uniform(0, 1, 0) AS result; +SELECT uniform(0, 10, 0) AS result; +SELECT uniform(0L, 10L, 0) AS result; +SELECT uniform(0, 10L, 0) AS result; +SELECT uniform(0, cast(10 as tinyint), 0) AS result; +SELECT uniform(0, cast(10 as smallint), 0) AS result; +SELECT uniform(0, 10S, 0) AS result; +SELECT uniform(10, 20, 0) AS result; +SELECT uniform(10.0F, 20.0F, 0) AS result; +SELECT uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(10, 3)), 0) AS result; +SELECT uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(11, 4)), 0) AS result; +SELECT uniform(10, cast(20 as decimal(10, 3)), 0) AS result; +SELECT uniform(cast(10 as decimal(10, 3)), 20, 0) AS result; +SELECT uniform(10.0D, 20.0D, CAST(3 / 7 AS LONG)) AS result; +SELECT uniform(10, 20.0F, 0) AS result; +SELECT uniform(10, 20, 0) AS result FROM VALUES (0), (1), (2) tab(col); +SELECT uniform(10, 20.0F) IS NOT NULL AS result; +SELECT uniform(-10L, 10L, 0) AS result; +SELECT uniform(-20L, -10L, 0) AS result; +SELECT uniform(-20L, -10L, -10) AS result; +SELECT uniform(NULL, 1, 0) AS result; +SELECT uniform(cast(NULL AS int), 1, 0) AS result; +SELECT uniform(cast(NULL AS float), 1, 0) AS result; +SELECT uniform(0, NULL, 0) AS result; +SELECT uniform(0, cast(NULL AS int), 0) AS result; +SELECT uniform(0, cast(NULL AS float), 0) AS result; +SELECT uniform(0, 1, NULL) AS result; +SELECT uniform(NULL, NULL, 0) AS result; +SELECT uniform(NULL, NULL, NULL) AS result; +-- Negative test cases for the uniform random number generator. +SELECT uniform(0, 1, cast(NULL as int)) AS result; +SELECT uniform(0, 1, cast(NULL as float)) AS result; +SELECT uniform(10, 20, col) AS result FROM VALUES (0), (1), (2) tab(col); +SELECT uniform(col, 10, 0) AS result FROM VALUES (0), (1), (2) tab(col); +SELECT uniform(10) AS result; +SELECT uniform(10, 20, 30, 40) AS result; +SELECT uniform(10.0F, 20.0F, 0.0F) AS result; +SELECT uniform(10.0F, 20.0F, 0.0D) AS result; +SELECT uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(10, 3)), cast(0 as decimal(10, 3))); +SELECT uniform('abc', 10, 0) AS result; +SELECT uniform(0, 'def', 0) AS result; +SELECT uniform(0, 10, 'ghi') AS result; + +-- The randstr random string generation function supports generating random strings within a +-- specified length. We use a seed of zero for most queries to keep tests deterministic. +SELECT randstr(1, 0) AS result; +SELECT randstr(5, 0) AS result; +SELECT randstr(10, 0) AS result; +SELECT randstr(10S, 0) AS result; +SELECT randstr(CAST(10 AS TINYINT), 0) AS result; +SELECT randstr(CAST(10 AS BIGINT), 0) AS result; +SELECT randstr(1.0F, 0) AS result; +SELECT randstr(1.0D, 0) AS result; +SELECT randstr(cast(1 AS DECIMAL(10, 2)), 0) AS result; +SELECT randstr(10, 0) AS result FROM VALUES (0), (1), (2) tab(col); +SELECT randstr(10) IS NOT NULL AS result; +SELECT randstr(1, -1) AS result; +-- Negative test cases for the randstr random number generator. +SELECT randstr(10L, 0) AS result; +SELECT randstr(10.0F, 0) AS result; +SELECT randstr(10.0D, 0) AS result; +SELECT randstr(NULL, 0) AS result; +SELECT randstr(0, NULL) AS result; +SELECT randstr(col, 0) AS result FROM VALUES (0), (1), (2) tab(col); +SELECT randstr(10, col) AS result FROM VALUES (0), (1), (2) tab(col); +SELECT randstr(10, 0, 1) AS result; +SELECT randstr(-1, 0) AS result; +SELECT randstr(10, "a") AS result FROM VALUES (0) tab(a); +SELECT randstr(10, 1.5) AS result FROM VALUES (0) tab(a); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/regexp-functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/regexp-functions.sql new file mode 100644 index 000000000000..b6d572434345 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/regexp-functions.sql @@ -0,0 +1,82 @@ +-- regexp_extract +SELECT regexp_extract('1a 2b 14m', '\\d+'); +SELECT regexp_extract('1a 2b 14m', '\\d+', 0); +SELECT regexp_extract('1a 2b 14m', '\\d+', 1); +SELECT regexp_extract('1a 2b 14m', '\\d+', 2); +SELECT regexp_extract('1a 2b 14m', '\\d+', -1); +SELECT regexp_extract('1a 2b 14m', '(\\d+)?', 1); +SELECT regexp_extract('a b m', '(\\d+)?', 1); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)'); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 0); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 1); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 2); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 3); +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', -1); +SELECT regexp_extract('1a 2b 14m', '(\\d+)?([a-z]+)', 1); +SELECT regexp_extract('a b m', '(\\d+)?([a-z]+)', 1); +SELECT regexp_extract('1a 2b 14m', '(?l)'); + +-- regexp_extract_all +SELECT regexp_extract_all('1a 2b 14m', '\\d+'); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 0); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 1); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 2); +SELECT regexp_extract_all('1a 2b 14m', '\\d+', -1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?', 1); +SELECT regexp_extract_all('a 2b 14m', '(\\d+)?', 1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)'); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 0); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 2); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 3); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', -1); +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?([a-z]+)', 1); +SELECT regexp_extract_all('a 2b 14m', '(\\d+)?([a-z]+)', 1); +SELECT regexp_extract_all('abc', col0, 1) FROM VALUES('], [') AS t(col0); + +-- regexp_replace +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something'); +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', -2); +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 0); +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 1); +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 2); +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 8); +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', 26); +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', 27); +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', 30); +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', null); + +-- regexp_like +SELECT regexp_like('1a 2b 14m', '\\d+b'); +SELECT regexp_like('1a 2b 14m', '[a-z]+b'); +SELECT regexp('1a 2b 14m', '\\d+b'); +SELECT regexp('1a 2b 14m', '[a-z]+b'); +SELECT rlike('1a 2b 14m', '\\d+b'); +SELECT rlike('1a 2b 14m', '[a-z]+b'); + +-- regexp_count +SELECT regexp_count('1a 2b 14m', '\\d+'); +SELECT regexp_count('1a 2b 14m', 'mmm'); +SELECT regexp_count('the fox', 'FOX'); +SELECT regexp_count('the fox', '(?i)FOX'); +SELECT regexp_count('passwd7 plain A1234 a1234', '(?=[^ ]*[a-z])(?=[^ ]*[0-9])[^ ]+'); +SELECT regexp_count(null, 'abc'); +SELECT regexp_count('abc', null); + +-- regexp_substr +SELECT regexp_substr('1a 2b 14m', '\\d+'); +SELECT regexp_substr('1a 2b 14m', '\\d+ '); +SELECT regexp_substr('1a 2b 14m', '\\d+(a|b|m)'); +SELECT regexp_substr('1a 2b 14m', '\\d{2}(a|b|m)'); +SELECT regexp_substr('1a 2b 14m', ''); +SELECT regexp_substr('Spark', null); +SELECT regexp_substr(null, '.*'); + +-- regexp_instr +SELECT regexp_instr('abc', 'b'); +SELECT regexp_instr('abc', 'x'); +SELECT regexp_instr('ABC', '(?-i)b'); +SELECT regexp_instr('1a 2b 14m', '\\d{2}(a|b|m)'); +SELECT regexp_instr('abc', null); +SELECT regexp_instr(null, 'b'); +SELECT regexp_instr('abc', col0, 1) FROM VALUES(') ?') AS t(col0); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/selectExcept.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/selectExcept.sql new file mode 100644 index 000000000000..dcac80d5faa7 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/selectExcept.sql @@ -0,0 +1,78 @@ +CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES + (10, "name1", named_struct("f1", 1, "s2", named_struct("f2", 101, "f3", "a"))), + (20, "name2", named_struct("f1", 2, "s2", named_struct("f2", 202, "f3", "b"))), + (30, "name3", named_struct("f1", 3, "s2", named_struct("f2", 303, "f3", "c"))), + (40, "name4", named_struct("f1", 4, "s2", named_struct("f2", 404, "f3", "d"))), + (50, "name5", named_struct("f1", 5, "s2", named_struct("f2", 505, "f3", "e"))), + (60, "name6", named_struct("f1", 6, "s2", named_struct("f2", 606, "f3", "f"))), + (70, "name7", named_struct("f1", 7, "s2", named_struct("f2", 707, "f3", "g"))) +AS tbl_view(id, name, data); + +CREATE TABLE ids (id INT) USING CSV; + +-- Happy path +-- EXCEPT basic scenario +SELECT * FROM tbl_view; +SELECT * EXCEPT (id) FROM tbl_view; +SELECT * EXCEPT (name) FROM tbl_view; +-- EXCEPT named structs +SELECT * EXCEPT (data) FROM tbl_view; +SELECT * EXCEPT (data.f1) FROM tbl_view; +SELECT * EXCEPT (data.s2) FROM tbl_view; +SELECT * EXCEPT (data.s2.f2) FROM tbl_view; +SELECT * EXCEPT (data.f1, data.s2) FROM tbl_view; +-- EXCEPT all columns +SELECT * EXCEPT (id, name, data) FROM tbl_view; +-- EXCEPT special character names +SELECT * EXCEPT (`a-b-c`) FROM (SELECT 1 a_b_c, 2 `a-b-c`); +-- EXCEPT qualified star +SELECT tbl_view.* EXCEPT (name) FROM tbl_view; +INSERT INTO ids +SELECT * EXCEPT (name, data) FROM tbl_view; +SELECT * FROM ids; +-- EXCEPT qualified columns +SELECT * EXCEPT (ids.id) FROM ids; +-- EXCEPT structs +SELECT data.* EXCEPT (s2) FROM tbl_view; +SELECT data.* EXCEPT (s2.f2) FROM tbl_view; +SELECT data.s2.* EXCEPT (f2) FROM tbl_view; + +-- Errors +-- EXCEPT missing brackets +SELECT * EXCEPT name FROM tbl_view; +-- EXCEPT no columns +SELECT * EXCEPT() name FROM tbl_view; +-- EXCEPT invalid column +SELECT * EXCEPT(invalid_column) FROM tbl_view; +-- EXCEPT find invalid column +SELECT * EXCEPT(id, invalid_column) FROM tbl_view; +-- EXCEPT duplicate column +SELECT * EXCEPT(id, id) FROM tbl_view; +-- EXCEPT overlapping columns +SELECT * EXCEPT(data.s2, data.s2.f2) FROM tbl_view; + +DROP VIEW tbl_view; + +CREATE TEMPORARY VIEW v1 AS VALUES (1, 2, NULL, 4, 5) AS T(c1, c2, c3, c4, c5); +-- star tests in select list +SELECT coalesce(*) FROM v1; +SELECT coalesce(* EXCEPT(c1, c2)) FROM v1; +SELECT array(*) FROM v1; +SELECT array(v1.*) FROM v1; +SELECT concat_ws(',', *) FROM v1; + +-- This is just SELECT * +SELECT (*) FROM v1; + +SELECT struct(*) FROM v1; +SELECT greatest(*) FROM v1; +SELECT 5 IN (*) FROM v1; +SELECT c1.* FROM VALUES(named_struct('a', 1, 'b', 2), 10, 20) as t(c1, c2, c3); + +-- star outside of select list +SELECT 1 FROM v1 WHERE coalesce(*) = 1; +SELECT 1 FROM v1 WHERE array(*) = array(1, 2, NULL, 4, 5); +SELECT 1 FROM v1 WHERE 4 IN (*); +SELECT T.* FROM v1, LATERAL (SELECT v1.*) AS T(c1, c2, c3, c4, c5); +SELECT T.* FROM v1, LATERAL (SELECT COALESCE(v1.*)) AS T(x); + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/show-create-table.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/show-create-table.sql new file mode 100644 index 000000000000..5192d2dc6b57 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/show-create-table.sql @@ -0,0 +1,114 @@ +-- simple +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet; + +SHOW CREATE TABLE tbl; +DROP TABLE tbl; + + +-- options +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +OPTIONS ('a' 1, 'password' = 'password'); + +SHOW CREATE TABLE tbl; +DROP TABLE tbl; + + +-- path option +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +OPTIONS ('path' '/path/to/table'); + +SHOW CREATE TABLE tbl; +DROP TABLE tbl; + + +-- location +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +LOCATION '/path/to/table'; + +SHOW CREATE TABLE tbl; +DROP TABLE tbl; + + +-- partition by +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +PARTITIONED BY (a); + +SHOW CREATE TABLE tbl; +DROP TABLE tbl; + + +-- clustered by +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +CLUSTERED BY (a) SORTED BY (b ASC) INTO 2 BUCKETS; + +SHOW CREATE TABLE tbl; +DROP TABLE tbl; + + +-- default column values +CREATE TABLE tbl (a INT DEFAULT 42, b STRING DEFAULT 'abc, def', c INT DEFAULT 42) USING parquet +COMMENT 'This is a comment'; + +SHOW CREATE TABLE tbl; +DROP TABLE tbl; + + +-- comment +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +COMMENT 'This is a comment'; + +SHOW CREATE TABLE tbl; +DROP TABLE tbl; + + +-- tblproperties +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +TBLPROPERTIES ('a' = '1', 'password' = 'password'); + +SHOW CREATE TABLE tbl; +DROP TABLE tbl; + +-- float alias real and decimal alias numeric +CREATE TABLE tbl (a REAL, b NUMERIC, c NUMERIC(10), d NUMERIC(10,1)) USING parquet; +SHOW CREATE TABLE tbl; +DROP TABLE tbl; + + +-- show create table for view +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet; + +-- simple +CREATE VIEW view_SPARK_30302 (aaa, bbb) +AS SELECT a, b FROM tbl; + +SHOW CREATE TABLE view_SPARK_30302 AS SERDE; + +SHOW CREATE TABLE view_SPARK_30302; + +DROP VIEW view_SPARK_30302; + + +-- comment +CREATE VIEW view_SPARK_30302 (aaa COMMENT 'comment with \'quoted text\' for aaa', bbb) +COMMENT 'This is a comment with \'quoted text\' for view' +AS SELECT a, b FROM tbl; + +SHOW CREATE TABLE view_SPARK_30302 AS SERDE; + +SHOW CREATE TABLE view_SPARK_30302; + +DROP VIEW view_SPARK_30302; + + +-- tblproperties +CREATE VIEW view_SPARK_30302 (aaa, bbb) +TBLPROPERTIES ('a' = '1', 'b' = '2') +AS SELECT a, b FROM tbl; + +SHOW CREATE TABLE view_SPARK_30302 AS SERDE; + +SHOW CREATE TABLE view_SPARK_30302; + +DROP VIEW view_SPARK_30302; + +DROP TABLE tbl; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/show-tables.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/show-tables.sql new file mode 100644 index 000000000000..8f46c93ee323 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/show-tables.sql @@ -0,0 +1,43 @@ +-- Test data. +CREATE DATABASE showdb; +USE showdb; +CREATE TABLE show_t1(a String, b Int, c String, d String) USING parquet PARTITIONED BY (c, d); +ALTER TABLE show_t1 ADD PARTITION (c='Us', d=1); +CREATE TABLE show_t2(b String, d Int) USING parquet; +CREATE TEMPORARY VIEW show_t3(e int) USING parquet; +CREATE GLOBAL TEMP VIEW show_t4 AS SELECT 1 as col1; + +-- SHOW TABLES +SHOW TABLES; +SHOW TABLES IN showdb; + +-- SHOW TABLES WITH wildcard match +SHOW TABLES 'show_t*'; +SHOW TABLES LIKE 'show_t1*|show_t2*'; +SHOW TABLES IN showdb 'show_t*'; +SHOW TABLES IN showdb LIKE 'show_t*'; + +-- SHOW TABLE EXTENDED +SHOW TABLE EXTENDED LIKE 'show_t*'; +SHOW TABLE EXTENDED; + +-- SHOW TABLE EXTENDED ... PARTITION +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us', d=1); +-- Throw a ParseException if table name is not specified. +SHOW TABLE EXTENDED PARTITION(c='Us', d=1); +-- Don't support regular expression for table name if a partition specification is present. +SHOW TABLE EXTENDED LIKE 'show_t*' PARTITION(c='Us', d=1); +-- Partition specification is not complete. +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us'); +-- Partition specification is invalid. +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(a='Us', d=1); +-- Partition specification doesn't exist. +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Ch', d=1); + +-- Clean Up +DROP TABLE show_t1; +DROP TABLE show_t2; +DROP VIEW show_t3; +DROP VIEW global_temp.show_t4; +USE default; +DROP DATABASE showdb; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/show-tblproperties.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/show-tblproperties.sql new file mode 100644 index 000000000000..05790f345f96 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/show-tblproperties.sql @@ -0,0 +1,26 @@ +-- create a table with properties +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +TBLPROPERTIES('p1'='v1', 'p2'='v2', password = 'password'); + +SHOW TBLPROPERTIES tbl; +SHOW TBLPROPERTIES tbl("p1"); +SHOW TBLPROPERTIES tbl("p3"); + +DROP TABLE tbl; + +-- create a view with properties +CREATE VIEW view TBLPROPERTIES('p1'='v1', 'p2'='v2') AS SELECT 1 AS c1; + +SHOW TBLPROPERTIES view; +SHOW TBLPROPERTIES view("p1"); +SHOW TBLPROPERTIES view("p3"); + +DROP VIEW view; + +-- create a temporary view +CREATE TEMPORARY VIEW tv AS SELECT 1 AS c1; + +-- Properties for a temporary view should be empty +SHOW TBLPROPERTIES tv; + +DROP VIEW tv; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/show-views.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/show-views.sql new file mode 100644 index 000000000000..bdf9ef4aa678 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/show-views.sql @@ -0,0 +1,28 @@ +-- Test data. +CREATE DATABASE showdb; +USE showdb; +CREATE TABLE tbl(a STRING, b INT, c STRING, d STRING) USING parquet; +CREATE VIEW view_1 AS SELECT * FROM tbl; +CREATE VIEW view_2 AS SELECT * FROM tbl WHERE c='a'; +CREATE GLOBAL TEMP VIEW view_3 AS SELECT 1 as col1; +CREATE TEMPORARY VIEW view_4(e INT) USING parquet; + +-- SHOW VIEWS +SHOW VIEWS; +SHOW VIEWS FROM showdb; +SHOW VIEWS IN showdb; +SHOW VIEWS IN global_temp; + +-- SHOW VIEWS WITH wildcard match +SHOW VIEWS 'view_*'; +SHOW VIEWS LIKE 'view_1*|view_2*'; +SHOW VIEWS IN showdb 'view_*'; +SHOW VIEWS IN showdb LIKE 'view_*'; +-- Error when database not exists +SHOW VIEWS IN wrongdb LIKE 'view_*'; + +-- Clean Up +DROP VIEW global_temp.view_3; +DROP VIEW view_4; +USE default; +DROP DATABASE showdb CASCADE; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/show_columns.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/show_columns.sql new file mode 100644 index 000000000000..521018e94e50 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/show_columns.sql @@ -0,0 +1,58 @@ +CREATE DATABASE showdb; + +USE showdb; + +CREATE TABLE showcolumn1 (col1 int, `col 2` int) USING json; +CREATE TABLE showcolumn2 (price int, qty int, year int, month int) USING parquet partitioned by (year, month); +CREATE TEMPORARY VIEW showColumn3 (col3 int, `col 4` int) USING json; +CREATE GLOBAL TEMP VIEW showColumn4 AS SELECT 1 as col1, 'abc' as `col 5`; + + +-- only table name +SHOW COLUMNS IN showcolumn1; + +-- qualified table name +SHOW COLUMNS IN showdb.showcolumn1; + +-- table name and database name +SHOW COLUMNS IN showcolumn1 FROM showdb; + +-- partitioned table +SHOW COLUMNS IN showcolumn2 IN showdb; + +-- Non-existent table. Raise an error in this case +SHOW COLUMNS IN badtable FROM showdb; + +-- database in table identifier and database name in different case +SHOW COLUMNS IN showdb.showcolumn1 from SHOWDB; + +-- different database name in table identifier and database name. +-- Raise an error in this case. +SHOW COLUMNS IN showdb.showcolumn1 FROM baddb; + +-- show column on temporary view +SHOW COLUMNS IN showcolumn3; + +-- error temp view can't be qualified with a database +SHOW COLUMNS IN showdb.showcolumn3; + +-- error temp view can't be qualified with a database +SHOW COLUMNS IN showcolumn3 FROM showdb; + +-- error global temp view needs to be qualified +SHOW COLUMNS IN showcolumn4; + +-- global temp view qualified with database +SHOW COLUMNS IN global_temp.showcolumn4; + +-- global temp view qualified with database +SHOW COLUMNS IN showcolumn4 FROM global_temp; + +DROP TABLE showcolumn1; +DROP TABLE showColumn2; +DROP VIEW showcolumn3; +DROP VIEW global_temp.showcolumn4; + +use default; + +DROP DATABASE showdb; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/sql-compatibility-functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/sql-compatibility-functions.sql new file mode 100644 index 000000000000..6c840154c618 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/sql-compatibility-functions.sql @@ -0,0 +1,27 @@ +-- A test suite for functions added for compatibility with other databases such as Oracle, MSSQL. +-- These functions are typically implemented using the trait RuntimeReplaceable. + +SELECT ifnull(null, 'x'), ifnull('y', 'x'), ifnull(null, null); +SELECT nullif('x', 'x'), nullif('x', 'y'); +SELECT nvl(null, 'x'), nvl('y', 'x'), nvl(null, null); +SELECT nvl2(null, 'x', 'y'), nvl2('n', 'x', 'y'), nvl2(null, null, null); + +-- type coercion +SELECT ifnull(1, 2.1d), ifnull(null, 2.1d); +SELECT nullif(1, 2.1d), nullif(1, 1.0d); +SELECT nvl(1, 2.1d), nvl(null, 2.1d); +SELECT nvl2(null, 1, 2.1d), nvl2('n', 1, 2.1d); + +-- SPARK-16730 cast alias functions for Hive compatibility +SELECT boolean(1), tinyint(1), smallint(1), int(1), bigint(1); +SELECT float(1), double(1), decimal(1); +SELECT date("2014-04-04"), timestamp(date("2014-04-04")); +-- error handling: only one argument +SELECT string(1, 2); + +-- SPARK-21555: RuntimeReplaceable used in group by +CREATE TEMPORARY VIEW tempView1 AS VALUES (1, NAMED_STRUCT('col1', 'gamma', 'col2', 'delta')) AS T(id, st); +SELECT nvl(st.col1, "value"), count(*) FROM from tempView1 GROUP BY nvl(st.col1, "value"); + +-- aggregate function inside NULLIF +SELECT nullif(SUM(id), 0) from range(5); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/sql-on-files.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/sql-on-files.sql new file mode 100644 index 000000000000..c3a16ca577ee --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/sql-on-files.sql @@ -0,0 +1,32 @@ +CREATE DATABASE IF NOT EXISTS sql_on_files; +-- Parquet +CREATE TABLE sql_on_files.test_parquet USING PARQUET AS SELECT 1; +SELECT * FROM parquet.``; +SELECT * FROM parquet.`/file/not/found`; +SELECT * FROM parquet.`${spark.sql.warehouse.dir}/sql_on_files.db/test_parquet`; +DROP TABLE sql_on_files.test_parquet; + +-- ORC +CREATE TABLE sql_on_files.test_orc USING ORC AS SELECT 1; +SELECT * FROM orc.``; +SELECT * FROM orc.`/file/not/found`; +SELECT * FROM orc.`${spark.sql.warehouse.dir}/sql_on_files.db/test_orc`; +DROP TABLE sql_on_files.test_orc; + +-- CSV +CREATE TABLE sql_on_files.test_csv USING CSV AS SELECT 1; +SELECT * FROM csv.``; +SELECT * FROM csv.`/file/not/found`; +SELECT * FROM csv.`${spark.sql.warehouse.dir}/sql_on_files.db/test_csv`; +DROP TABLE sql_on_files.test_csv; + +-- JSON +CREATE TABLE sql_on_files.test_json USING JSON AS SELECT 1; +SELECT * FROM json.``; +SELECT * FROM json.`/file/not/found`; +SELECT * FROM json.`${spark.sql.warehouse.dir}/sql_on_files.db/test_json`; +DROP TABLE sql_on_files.test_json; + +DROP DATABASE sql_on_files; + +SELECT * FROM json.`https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/src/main/resources/employees.json`; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/sql-session-variables.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/sql-session-variables.sql new file mode 100644 index 000000000000..d876be1bb6bc --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/sql-session-variables.sql @@ -0,0 +1,383 @@ +SET spark.sql.ansi.enabled = true; + +DECLARE title STRING; + +SET VARIABLE title = '-- Basic sanity --'; +DECLARE var1 INT = 5; +SELECT var1; +SET VARIABLE var1 = 6; +SELECT var1; +DROP TEMPORARY VARIABLE var1; + +SET VARIABLE title = 'Create Variable - Success Cases'; +DECLARE VARIABLE var1 INT; +SELECT 'Expect: INT, NULL', typeof(var1), var1; + +DECLARE OR REPLACE VARIABLE var1 DOUBLE; +SELECT 'Expect: DOUBLE, NULL', typeof(var1), var1; + +DROP TEMPORARY VARIABLE var1; +DECLARE OR REPLACE VARIABLE var1 TIMESTAMP; +SELECT 'Expect: TIMESTAMP, NULL', typeof(var1), var1; + +SET VARIABLE title = 'Create Variable - Failure Cases'; +-- No support for IF NOT EXISTS +DECLARE VARIABLE IF NOT EXISTS var1 INT; +DROP TEMPORARY VARIABLE IF EXISTS var1; + +SET VARIABLE title = 'Drop Variable'; +DECLARE VAR var1 INT; +SELECT var1; +DROP TEMPORARY VAR var1; + +-- Variable is gone +SELECT var1; +DROP TEMPORARY VARIABLE var1; + +-- Success +DROP TEMPORARY VARIABLE IF EXISTS var1; + +-- Fail: TEMPORARY is mandatory on DROP +DECLARE VARIABLE var1 INT; +DROP VARIABLE var1; +DROP VARIABLE system.session.var1; +DROP TEMPORARY VARIABLE var1; + +SET VARIABLE title = 'Test qualifiers - success'; +DECLARE VARIABLE var1 INT DEFAULT 1; +SELECT 1 as Expected, var1 as Unqualified, session.var1 AS SchemaQualified, system.session.var1 AS fullyQualified; +SET VARIABLE var1 = 2; +SELECT 2 as Expected, var1 as Unqualified, session.var1 AS SchemaQualified, system.session.var1 AS fullyQualified; + +DECLARE OR REPLACE VARIABLE session.var1 INT DEFAULT 1; +SELECT 1 as Expected, var1 as Unqualified, session.var1 AS SchemaQualified, system.session.var1 AS fullyQualified; +SET VARIABLE session.var1 = 2; +SELECT 2 as Expected, var1 as Unqualified, session.var1 AS SchemaQualified, system.session.var1 AS fullyQualified; + +DECLARE OR REPLACE VARIABLE system.session.var1 INT DEFAULT 1; +SELECT 1 as Expected, var1 as Unqualified, session.var1 AS SchemaQualified, system.session.var1 AS fullyQualified; +SET VARIABLE system.session.var1 = 2; +SELECT 2 as Expected, var1 as Unqualified, session.var1 AS SchemaQualified, system.session.var1 AS fullyQualified; + +DECLARE OR REPLACE VARIABLE sySteM.sEssIon.vAr1 INT DEFAULT 1; +SELECT 1 as Expected, var1 as Unqualified, sessIon.Var1 AS SchemaQualified, System.sessiOn.var1 AS fullyQualified; +SET VARIABLE sYstem.sesSiOn.vaR1 = 2; +SELECT 2 as Expected, VAR1 as Unqualified, SESSION.VAR1 AS SchemaQualified, SYSTEM.SESSION.VAR1 AS fullyQualified; + +DECLARE OR REPLACE VARIABLE var1 INT; +DROP TEMPORARY VARIABLE var1; +DROP TEMPORARY VARIABLE var1; + +DECLARE OR REPLACE VARIABLE var1 INT; +DROP TEMPORARY VARIABLE session.var1; +DROP TEMPORARY VARIABLE var1; + +DECLARE OR REPLACE VARIABLE var1 INT; +DROP TEMPORARY VARIABLE system.session.var1; +DROP TEMPORARY VARIABLE var1; + +DECLARE OR REPLACE VARIABLE var1 INT; +DROP TEMPORARY VARIABLE sysTem.sesSion.vAr1; +DROP TEMPORARY VARIABLE var1; + +SET VARIABLE title = 'Test variable in aggregate'; +SELECT (SELECT MAX(id) FROM RANGE(10) WHERE id < title) FROM VALUES 1, 2 AS t(title); + +SET VARIABLE title = 'Test qualifiers - fail'; +DECLARE OR REPLACE VARIABLE builtin.var1 INT; +DECLARE OR REPLACE VARIABLE system.sesion.var1 INT; +DECLARE OR REPLACE VARIABLE sys.session.var1 INT; + +DECLARE OR REPLACE VARIABLE var1 INT; +SELECT var; +SELECT ses.var1; +SELECT b.sesson.var1; +SELECT builtn.session.var1; + +SET VARIABLE ses.var1 = 1; +SET VARIABLE builtn.session.var1 = 1; + +SET VARIABLE title = 'Test DEFAULT on create - success'; +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 1; +SELECT 1 AS Expected, var1 AS result; + +DECLARE OR REPLACE VARIABLE var1 DOUBLE DEFAULT 1 + RAND(5); +SELECT true AS Expected, var1 >= 1 AS result; + +DECLARE OR REPLACE VARIABLE var1 = 'Hello'; +SELECT 'STRING, Hello' AS Expected, typeof(var1) AS type, var1 AS result; + +DECLARE OR REPLACE VARIABLE var1 DEFAULT NULL; +SELECT 'VOID, NULL' AS Expected, typeof(var1) AS type, var1 AS result; + +DECLARE OR REPLACE VARIABLE INT DEFAULT 5.0; +SELECT 'INT, 5' AS Expected, typeof(var1) AS type, var1 AS result; + +DECLARE OR REPLACE VARIABLE var1 MAP DEFAULT MAP('Hello', 5.1, 'World', -7.1E10); +SELECT 'MAP, [Hello -> 5.1, World -> -7E10]' AS Expected, typeof(var1) AS type, var1 AS result; + +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT NULL; +SELECT 'NULL' AS Expected, var1 AS result; + +DECLARE OR REPLACE VARIABLE var1 STRING DEFAULT CURRENT_DATABASE(); +SELECT 'true' AS Expected, length(var1) > 0 AS result; + +DROP TEMPORARY VARIABLE var1; + +-- No type and no default is not allowed +DECLARE var1; + +-- TBD: Store assignment cast test + +SET VARIABLE title = 'Test DEFAULT on create - failures'; + +-- No subqueries allowed in DEFAULT expression +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT (SELECT c1 FROM VALUES(1) AS T(c1)); + +-- Incompatible type +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 'hello'; + +-- Runtime error +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 1 / 0; + +-- Runtime overflow on assignment +DECLARE OR REPLACE VARIABLE var1 SMALLINT DEFAULT 100000; + +SET VARIABLE title = 'SET VARIABLE - single target'; + +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 5; + +SET VARIABLE var1 = 7; +SELECT var1; + +SET VAR var1 = 8; +SELECT var1; + +SET VARIABLE var1 = (SELECT c1 FROM VALUES(1) AS T(c1)); +SELECT var1; + +SET VARIABLE var1 = (SELECT c1 FROM VALUES(1) AS T(c1) WHERE 1=0); +SELECT var1 AS `null`; + +SET VARIABLE var1 = (SELECT c1 FROM VALUES(1.0) AS T(c1)); +SELECT var1; + +SET VARIABLE var1 = (SELECT c1 FROM VALUES(1.0E10) AS T(c1)); +SELECT var1; + +SET VARIABLE var1 = (SELECT c1 FROM VALUES(1), (2) AS T(c1)); + +SET VARIABLE var1 = (SELECT c1, c1 FROM VALUES(1), (2) AS T(c1)); + +SET VARIABLE var1 = (SELECT c1 FROM VALUES('hello') AS T(c1)); + +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 5; +SET VARIABLE var1 = var1 + 1; +SELECT var1; + +-- TBD store assignment cast test + +DROP TEMPORARY VARIABLE var1; + +SET VARIABLE title = 'SET VARIABLE - comma separated target'; + +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 5; +DECLARE OR REPLACE VARIABLE var2 STRING DEFAULT 'hello'; +DECLARE OR REPLACE VARIABLE var3 DOUBLE DEFAULT 2; + +SET VARIABLE var1 = 6, var2 = 'world', var3 = pi(); +SELECT var1 AS `6`, var2 AS `world` , var3 as `3.14...`; + +SET VAR var1 = 7, var2 = 'universe', var3 = -1; +SELECT var1 AS `7`, var2 AS `universe` , var3 as `-1`; + +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 5; +DECLARE OR REPLACE VARIABLE var2 STRING DEFAULT 'hello'; +DECLARE OR REPLACE VARIABLE var3 DOUBLE DEFAULT 2; + +SET VARIABLE var1 = var3, var2 = ascii(var1), var3 = var1; +SELECT var1 AS `2`, var2 AS `104`, var3 AS `5`; + +SET VARIABLE var1 = var3, var2 = INTERVAL'5' HOUR, var3 = var1; + +-- Duplicates check +SET VARIABLE var1 = 1, var2 = 0, vAr1 = 1; + +DROP TEMPORARY VARIABLE var1; +DROP TEMPORARY VARIABLE var2; +DROP TEMPORARY VARIABLE var3; + +SET VARIABLE title = 'SET VARIABLE - row assignment'; + +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 5; +DECLARE OR REPLACE VARIABLE var2 STRING DEFAULT 'hello'; +DECLARE OR REPLACE VARIABLE var3 DOUBLE DEFAULT 2; + +-- Must have at least one target +SET VARIABLE (var1) = (SELECT c1 FROM VALUES(1) AS T(c1)); +SELECT var1; + +SET VAR (var1) = (SELECT c1 FROM VALUES(2) AS T(c1)); +SELECT var1; + +SET VARIABLE (var1, var2) = (SELECT c1, c2 FROM VALUES(10, 11) AS T(c1, c2)); +SELECT var1 AS `10`, var2 AS `11`; + +SET VARIABLE (var1, var2, var3) = (SELECT c1, c2, c3 FROM VALUES(100, 110, 120) AS T(c1, c2, c3)); +SELECT var1 AS `100`, var2 AS `110`, var3 AS `120`; + +SET VARIABLE (var1, var2, var3) = (SELECT c1, c2, c3 FROM VALUES(100, 110, 120) AS T(c1, c2, c3) WHERE 1 = 0); +SELECT var1 AS `NULL`, var2 AS `NULL`, var3 AS `NULL`; + +-- Fail no target +SET VARIABLE () = (SELECT 1); + +-- Fail, more than one row +SET VARIABLE (var1, var2, var3) = (SELECT c1, c2, c3 FROM VALUES(100, 110, 120), (-100, -110, -120) AS T(c1, c2, c3)); + +-- Fail, not enough columns +SET VARIABLE (var1, var2, var3) = (SELECT c1, c2 FROM VALUES(100, 110, 120) AS T(c1, c2, c3)); + +-- Fail, too many columns +SET VARIABLE (var1, var2, var3) = (SELECT c1, c2, c3, c1 FROM VALUES(100, 110, 120) AS T(c1, c2, c3)); + +-- Fail, duplicated target +SET VARIABLE (var1, var2, var1) = (SELECT c1, c2, c3, c1 FROM VALUES(100, 110, 120) AS T(c1, c2, c3)); + +DROP TEMPORARY VARIABLE var1; +DROP TEMPORARY VARIABLE var2; +DROP TEMPORARY VARIABLE var3; + +SET VARIABLE title = 'DEFAULT expression usage'; + +DECLARE OR REPLACE VARIABLE var1 STRING DEFAULT 'default1'; +DECLARE OR REPLACE VARIABLE var2 STRING DEFAULT 'default2'; +DECLARE OR REPLACE VARIABLE var3 STRING DEFAULT 'default3'; + +SET VARIABLE var1 = 'hello'; + +SET VARIABLE var1 = DEFAULT; +SELECT var1 AS `default`; + +SET VARIABLE var1 = 'hello1'; +SET VARIABLE var1 = 'hello2'; +SET VARIABLE var1 = 'hello3'; +SET VARIABLE var1 = DEFAULT, var2 = DEFAULT, var3 = DEFAULT; +SELECT var1 AS `default1`, var2 AS `default2`, var3 AS `default3`; + +SET VARIABLE var1 = 'hello'; +SET VARIABLE (var1) = (SELECT DEFAULT FROM VALUES(1) AS T(c1)); +SELECT var1 AS `default`; + +SET VARIABLE var1 = 'hello'; +SET VARIABLE (var1) = (SELECT DEFAULT FROM VALUES('world') AS T(default)); +SELECT var1 AS `world`; + +SET VARIABLE var1 = 'hello'; +SET VARIABLE (var1) = (SELECT DEFAULT FROM VALUES(1) AS T(c1) LIMIT 1); +SELECT var1 AS `default`; + +SET VARIABLE var1 = 'hello'; +SET VARIABLE (var1) = (SELECT DEFAULT FROM VALUES(1),(2),(3) AS T(c1) LIMIT 1 OFFSET 1); +SELECT var1 AS `default`; + +SET VARIABLE var1 = 'hello'; +SET VARIABLE (var1) = (SELECT DEFAULT FROM VALUES(1),(2),(3) AS T(c1) OFFSET 1); +SELECT var1 AS `default`; + +SET VARIABLE var1 = 'hello'; +SET VARIABLE (var1) = (WITH v1(c1) AS (VALUES(1) AS T(c1)) SELECT DEFAULT FROM VALUES(1),(2),(3) AS T(c1)); +SELECT var1 AS `default`; + +-- Failure +SET VARIABLE var1 = 'Hello' || DEFAULT; + +SET VARIABLE (var1) = (VALUES(DEFAULT)); + +SET VARIABLE (var1) = (WITH v1(c1) AS (VALUES(1) AS T(c1)) SELECT DEFAULT + 1 FROM VALUES(1),(2),(3) AS T(c1)); + +SET VARIABLE var1 = session.default; + +DROP TEMPORARY VARIABLE var1; +DROP TEMPORARY VARIABLE var2; +DROP TEMPORARY VARIABLE var3; + +SET VARIABLE title = 'SET command'; + +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 1; + +-- Sanity: These are all configs +SET x.var1 = 5; +SET x = 5; +SET system.x.var = 5; +SET x.session.var1 = 5; + +-- These raise errors: UNSUPPORTED_FEATURE.SET_VARIABLE_IN_SET +SET var1 = 5; +SET session.var1 = 5; +SET system.session.var1 = 5; +SET vAr1 = 5; +SET seSSion.var1 = 5; +SET sYStem.session.var1 = 5; + +DROP TEMPORARY VARIABLE var1; + +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 1; + +SELECT var1 AS `2` FROM VALUES(2) AS T(var1); + +SELECT c1 AS `2` FROM VALUES(2) AS T(var1), LATERAL(SELECT var1) AS TT(c1); + +SELECT session.var1 AS `1` FROM VALUES(2) AS T(var1); + +SELECT c1 AS `1` FROM VALUES(2) AS T(var1), LATERAL(SELECT session.var1) AS TT(c1); + +DROP TEMPORARY VARIABLE var1; + +SET VARIABLE title = 'variable references -- visibility'; +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 1; + +VALUES (var1); + +SELECT var1; + +SELECT sum(var1) FROM VALUES(1); +SELECT var1 + SUM(0) FROM VALUES(1); +SELECT substr('12345', var1, 1); +SELECT 1 FROM VALUES(1, 2) AS T(c1, c2) GROUP BY c1 + var1; +SELECT c1, sum(c2) FROM VALUES(1, 2) AS T(c1, c2) GROUP BY c1 HAVING sum(c1) != var1; +SELECT 1 FROM VALUES(1) AS T(c1) WHERE c1 IN (var1); +SELECT sum(c1) FILTER (c1 != var1) FROM VALUES(1, 2), (2, 3) AS T(c1, c2); +SELECT array(1, 2, 4)[var1]; + +-- TBD usage in body of lambda function + +SELECT 1 FROM VALUES(1) AS T(c1) WHERE c1 = var1; + +WITH v1 AS (SELECT var1 AS c1) SELECT c1 AS `1` FROM v1; + +CREATE OR REPLACE TEMPORARY VIEW v AS SELECT var1 AS c1; +SELECT * FROM v; +DROP VIEW v; + +DROP TEMPORARY VARIABLE var1; + +SET VARIABLE title = 'variable references -- prohibited'; + +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 1; + +-- Known broken for parameters as well +--DROP TABLE IF EXISTS T; +--CREATE TABLE T(c1 INT DEFAULT (var1)); +--DROP TABLE IF EXISTS T; + +CREATE OR REPLACE VIEW v AS SELECT var1 AS c1; +DROP VIEW IF EXISTS V; + +DROP TEMPORARY VARIABLE var1; + +SET VARIABLE title = 'variable references -- test constant folding'; + +DECLARE OR REPLACE VARIABLE var1 STRING DEFAULT 'a INT'; +SELECT from_json('{"a": 1}', var1); +DROP TEMPORARY VARIABLE var1; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/sql-udf.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/sql-udf.sql new file mode 100644 index 000000000000..63c2a1945683 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/sql-udf.sql @@ -0,0 +1,989 @@ +-- test cases for SQL User Defined Functions + +-- 1. CREATE FUNCTION +-- 1.1 Parameter +-- 1.1.a A scalar function with various numbers of parameter +-- Expect success +CREATE FUNCTION foo1a0() RETURNS INT RETURN 1; +-- Expect: 1 +SELECT foo1a0(); +-- Expect failure +SELECT foo1a0(1); + +CREATE FUNCTION foo1a1(a INT) RETURNS INT RETURN 1; +-- Expect: 1 +SELECT foo1a1(1); +-- Expect failure +SELECT foo1a1(1, 2); + +CREATE FUNCTION foo1a2(a INT, b INT, c INT, d INT) RETURNS INT RETURN 1; +-- Expect: 1 +SELECT foo1a2(1, 2, 3, 4); + +-- 1.1.b A table function with various numbers of arguments +CREATE FUNCTION foo1b0() RETURNS TABLE (c1 INT) RETURN SELECT 1; +-- Expect (1) +SELECT * FROM foo1b0(); + +CREATE FUNCTION foo1b1(a INT) RETURNS TABLE (c1 INT) RETURN SELECT 1; +-- Expect (1) +SELECT * FROM foo1b1(1); + +CREATE FUNCTION foo1b2(a INT, b INT, c INT, d INT) RETURNS TABLE(c1 INT) RETURN SELECT 1; +-- Expect (1) +SELECT * FROM foo1b2(1, 2, 3, 4); + +-- 1.1.c Duplicate argument names +-- Expect failure +CREATE FUNCTION foo1c1(duplicate INT, DUPLICATE INT) RETURNS INT RETURN 1; + +-- Expect failure +CREATE FUNCTION foo1c2(a INT, b INT, thisisaduplicate INT, c INT, d INT, e INT, f INT, thisIsaDuplicate INT, g INT) + RETURNS TABLE (a INT) RETURN SELECT 1; + +-- 1.1.d DEFAULT parameters +-- A NULL default +CREATE OR REPLACE FUNCTION foo1d1(a INT DEFAULT NULL) RETURNS INT RETURN a; + +-- Expect 5, NULL +SELECT foo1d1(5), foo1d1(); + +-- A literal default +CREATE OR REPLACE FUNCTION foo1d1(a INT DEFAULT 10) RETURNS INT RETURN a; + +-- Expect 5, 10 +SELECT foo1d1(5), foo1d1(); + +-- A constant expression +CREATE OR REPLACE FUNCTION foo1d1(a INT DEFAULT length(substr(current_database(), 1, 1))) RETURNS INT RETURN a; + +-- Expect 5, 1 +SELECT foo1d1(5), foo1d1(); + +-- An expression that needs a cast +CREATE OR REPLACE FUNCTION foo1d1(a INT DEFAULT '5' || length(substr(current_database(), 1, 1))) + RETURNS INT RETURN a; + +-- Expect 5, 51 +SELECT foo1d1(5), foo1d1(); + +-- A non deterministic default +CREATE OR REPLACE FUNCTION foo1d1(a INT DEFAULT RAND()::INT) RETURNS INT RETURN a; + +-- Expect 5, 0 +SELECT foo1d1(5), foo1d1(); + +-- Cannot cast +-- Expect error +CREATE OR REPLACE FUNCTION foo1d1(a INT DEFAULT array(55, 17)) + RETURNS INT RETURN a; + +-- A subquery +CREATE OR REPLACE FUNCTION foo1d1(a INT DEFAULT (SELECT max(c1) FROM VALUES (1) AS T(c1))) + RETURNS INT RETURN a; + +-- Multiple parameters +CREATE OR REPLACE FUNCTION foo1d2(a INT, b INT DEFAULT 7, c INT DEFAULT 8, d INT DEFAULT 9 COMMENT 'test') + RETURNS STRING RETURN a || ' ' || b || ' ' || c || ' ' || d; + +-- Expect: (1 2 3 4), (1 2 3 9), (1 2 8 9), (1 7 8 9) +SELECT foo1d2(1, 2, 3, 4), foo1d2(1, 2, 3), foo1d2(1, 2), foo1d2(1); + +-- Expect error a has no default +SELECT foo1d2(); + +-- Expect error, too many parameters +SELECT foo1d2(1, 2, 3, 4, 5); + +-- Sparse default, expect error +CREATE OR REPLACE FUNCTION foo1d2(a INT DEFAULT 5, b INT , c INT DEFAULT 8, d INT DEFAULT 9 COMMENT 'test') + RETURNS STRING RETURN a || ' ' || b || ' ' || c || ' ' || d; + +CREATE OR REPLACE FUNCTION foo1d2(a INT, b INT DEFAULT 7, c INT DEFAULT 8, d INT COMMENT 'test') + RETURNS STRING RETURN a || ' ' || b || ' ' || c || ' ' || d; + +-- Temporary function +CREATE OR REPLACE TEMPORARY FUNCTION foo1d3(a INT DEFAULT 7 COMMENT 'hello') RETURNS INT RETURN a; + +-- Expect 5, 7 +SELECT foo1d3(5), foo1d3(); + +-- Dependent default +-- Expect error +CREATE OR REPLACE FUNCTION foo1d4(a INT, b INT DEFAULT a) RETURNS INT RETURN a + b; + +-- Defaults with SQL UDF +CREATE OR REPLACE FUNCTION foo1d4(a INT, b INT DEFAULT 3) RETURNS INT RETURN a + b; + +CREATE OR REPLACE FUNCTION foo1d5(a INT, b INT DEFAULT foo1d4(6)) RETURNS INT RETURN a + b; + +-- Expect 19, 12 +SELECT foo1d5(10), foo1d5(10, 2); + +-- Function invocation with default in SQL UDF +CREATE OR REPLACE FUNCTION foo1d5(a INT, b INT) RETURNS INT RETURN a + foo1d4(b); + +-- Expect 15 +SELECT foo1d5(10, 2); + +-- DEFAULT in table function +CREATE OR REPLACE FUNCTION foo1d6(a INT, b INT DEFAULT 7) RETURNS TABLE(a INT, b INT) RETURN SELECT a, b; + +-- Expect (5, 7) +SELECT * FROM foo1d6(5); + +-- Expect (5, 2) +SELECT * FROM foo1d6(5, 2); + +-- 1.1.e NOT NULL +-- Expect failure +CREATE FUNCTION foo1e1(x INT NOT NULL, y INT) RETURNS INT RETURN 1; +CREATE FUNCTION foo1e2(x INT, y INT NOT NULL) RETURNS TABLE (x INT) RETURN SELECT 1; +CREATE FUNCTION foo1e3(x INT, y INT) RETURNS TABLE (x INT NOT NULL) RETURN SELECT 1; + +-- 1.1.f GENERATED ALWAYS AS +-- Expect failure +CREATE FUNCTION foo1f1(x INT, y INT GENERATED ALWAYS AS (x + 10)) RETURNS INT RETURN y + 1; +CREATE FUNCTION foo1f2(id BIGINT GENERATED ALWAYS AS IDENTITY) RETURNS BIGINT RETURN id + 1; + +-- 1.2 Returns Columns +-- 1.2.a A table function with various numbers of returns columns +-- Expect error: Cannot have an empty RETURNS +CREATE FUNCTION foo2a0() RETURNS TABLE() RETURN SELECT 1; + +CREATE FUNCTION foo2a2() RETURNS TABLE(c1 INT, c2 INT) RETURN SELECT 1, 2; +-- Expect (1, 2) +SELECT * FROM foo2a2(); + +CREATE FUNCTION foo2a4() RETURNS TABLE(c1 INT, c2 INT, c3 INT, c4 INT) RETURN SELECT 1, 2, 3, 4; +-- Expect (1, 2, 3, 4) +SELECT * FROM foo2a2(); + +-- 1.2.b Duplicates in RETURNS clause +-- Expect failure +CREATE FUNCTION foo2b1() RETURNS TABLE(DuPLiCatE INT, duplicate INT) RETURN SELECT 1, 2; + +-- Expect failure +CREATE FUNCTION foo2b2() RETURNS TABLE(a INT, b INT, duplicate INT, c INT, d INT, e INT, DUPLICATE INT) +RETURN SELECT 1, 2, 3, 4, 5, 6, 7; + +-- 1.2.c No DEFAULT allowed in RETURNS +CREATE FUNCTION foo2c1() RETURNS TABLE(c1 INT DEFAULT 5) RETURN SELECT 1, 2; + +-- 1.3 Mismatched RETURN +-- Expect Failure +CREATE FUNCTION foo31() RETURNS INT RETURN (SELECT 1, 2); + +CREATE FUNCTION foo32() RETURNS TABLE(a INT) RETURN SELECT 1, 2; + +CREATE FUNCTION foo33() RETURNS TABLE(a INT, b INT) RETURN SELECT 1; + +-- 1.4 Table function returns expression and vice versa +CREATE FUNCTION foo41() RETURNS INT RETURN SELECT 1; +-- Expect failure +CREATE FUNCTION foo42() RETURNS TABLE(a INT) RETURN 1; + +-- 1.5 Scalar function returns subquery with more than one row or no rows + +-- 1.5.a More than one row +CREATE FUNCTION foo51() RETURNS INT RETURN (SELECT a FROM VALUES(1), (2) AS T(a)); +SELECT foo51(); + +-- 1.5.b No Rows +CREATE FUNCTION foo52() RETURNS INT RETURN (SELECT 1 FROM VALUES(1) WHERE 1 = 0); +-- Expect Success: NULL +SELECT foo52(); + +-- 1.6 Difficult identifiers +-- 1.6.a Space in the schema name +-- UNSUPPORTED BY CREATE SCHEMA +-- CREATE SCHEMA `a b`; + +-- CREATE FUNCTION `a b`.foo6a() RETURNS INT RETURN 1; +-- SELECT `a b`.foo6a(); + +-- DROP FUNCTION `a b`.foo6a; +-- DROP SCHEMA `a b`; + +-- 1.6.b Space in a function name +-- Default Hive configuration does not allow function name with space +-- CREATE FUNCTION `foo 6 b`() RETURNS INT RETURN 1; +-- SELECT `foo 6 b`(); +-- DROP FUNCTION `foo 6 b`; + +-- 1.6.c Spaces in parameter names +CREATE FUNCTION foo6c(` a` INT, a INT, `a b` INT) RETURNS INT RETURN 1; +SELECT foo6c(1, 2, 3); + +-- 1.6.d Spaces in RETURNS column list +CREATE FUNCTION foo6d() RETURNS TABLE(` a` INT, a INT, `a b` INT) RETURN SELECT 1, 2, 3; +SELECT * FROM foo6d(); + +-- 1.7 Parameter resolution +CREATE FUNCTION foo7a(a STRING, b STRING, c STRING) RETURNS STRING RETURN +SELECT 'Foo.a: ' || a || ' Foo.a: ' || foo7a.a + || ' T.b: ' || b || ' Foo.b: ' || foo7a.b + || ' T.c: ' || c || ' T.c: ' || t.c FROM VALUES('t.b', 't.c') AS T(b, c); + +SELECT foo7a('Foo.a', 'Foo.b', 'Foo.c'); + +CREATE FUNCTION foo7at(a STRING, b STRING, c STRING) RETURNS TABLE (a STRING, b STRING, c STRING, d STRING, e STRING) RETURN +SELECT CONCAT('Foo.a: ', a), CONCAT('Foo.b: ', foo7at.b), CONCAT('T.b: ', b), + CONCAT('Foo.c: ', foo7at.c), CONCAT('T.c: ', c) +FROM VALUES ('t.b', 't.c') AS T(b, c); +SELECT * FROM foo7at('Foo.a', 'Foo.b', 'Foo.c'); + +-- 1.8 Comments +-- Need to verify comments in non-sql tests + +-- 1.9 Test all data types +-- Boolean +CREATE FUNCTION foo9a(a BOOLEAN) RETURNS BOOLEAN RETURN NOT a; +SELECT foo9a(true); + +-- Expect error +SELECT foo9a(5); +SELECT foo9a('Nonsense'); + +-- Byte +CREATE FUNCTION foo9b(a BYTE) RETURNS BYTE RETURN CAST(a AS SHORT) + 1; +SELECT foo9b(126); +SELECT foo9b(127); +SELECT foo9b(128); + +-- Short +CREATE FUNCTION foo9c(a SHORT) RETURNS SHORT RETURN CAST(a AS INTEGER) + 1; +SELECT foo9c(32766); +SELECT foo9c(32767); +SELECT foo9c(32768); + +-- Integer +CREATE FUNCTION foo9d(a INTEGER) RETURNS INTEGER RETURN CAST(a AS BIGINT) + 1; +SELECT foo9d(2147483647 - 1); +SELECT foo9d(2147483647); +SELECT foo9d(2147483647 + 1); + +-- Bigint +CREATE FUNCTION foo9e(a BIGINT) RETURNS BIGINT RETURN CAST(a AS DECIMAL(20, 0)) + 1; +SELECT foo9e(9223372036854775807 - 1); +SELECT foo9e(9223372036854775807); +SELECT foo9e(9223372036854775807.0 + 1); + +-- DECIMAL +CREATE FUNCTION foo9f(a DECIMAL( 5, 2 )) RETURNS DECIMAL (5, 2) RETURN CAST(a AS DECIMAL(6, 2)) + 1; +SELECT foo9f(999 - 1); +SELECT foo9f(999); +SELECT foo9f(999 + 1); + +-- FLOAT +CREATE FUNCTION foo9g(a FLOAT, b String) RETURNS FLOAT RETURN b || CAST(a AS String); +SELECT foo9g(123.23, '7'); +SELECT foo9g('hello', '7'); +SELECT foo9g(123.23, 'q'); + +-- DOUBLE +CREATE FUNCTION foo9h(a DOUBLE, b String) RETURNS DOUBLE RETURN b || CAST(a AS String); +SELECT foo9h(123.23, '7'); +SELECT foo9h('hello', '7'); +SELECT foo9h(123.23, 'q'); + +-- VARCHAR +-- Expect failure: char/varchar type can only be used in the table schema. +CREATE FUNCTION foo9i(a VARCHAR(10), b VARCHAR(10)) RETURNS VARCHAR(12) RETURN a || b; +-- SELECT foo9i('1234567890', ''); +-- SELECT foo9i('12345678901', ''); +-- SELECT foo9i('1234567890', '1'); + +-- STRING +CREATE FUNCTION foo9j(a STRING, b STRING) RETURNS STRING RETURN a || b; +SELECT foo9j('1234567890', '12'); +SELECT foo9j(12345678901, '12'); + +-- DATE +CREATE FUNCTION foo9l(a DATE, b INTERVAL) RETURNS DATE RETURN a + b; +SELECT foo9l(DATE '2020-02-02', INTERVAL '1' YEAR); +SELECT foo9l('2020-02-02', INTERVAL '1' YEAR); +SELECT foo9l(DATE '-7', INTERVAL '1' YEAR); +SELECT foo9l(DATE '2020-02-02', INTERVAL '9999999' YEAR); + +-- TIMESTAMP +CREATE FUNCTION foo9m(a TIMESTAMP, b INTERVAL) RETURNS TIMESTAMP RETURN a + b; +SELECT foo9m(TIMESTAMP'2020-02-02 12:15:16.123', INTERVAL '1' YEAR); +SELECT foo9m('2020-02-02 12:15:16.123', INTERVAL '1' YEAR); +SELECT foo9m(TIMESTAMP'2020-02-02 12:15:16.123', INTERVAL '999999' YEAR); + +-- ARRAY +CREATE FUNCTION foo9n(a ARRAY) RETURNS ARRAY RETURN a; +SELECT foo9n(ARRAY(1, 2, 3)); +SELECT foo9n(from_json('[1, 2, 3]', 'array')); + +-- MAP +CREATE FUNCTION foo9o(a MAP) RETURNS MAP RETURN a; +SELECT foo9o(MAP('hello', 1, 'world', 2)); +SELECT foo9o(from_json('{"hello":1, "world":2}', 'map')); + +-- STRUCT +CREATE FUNCTION foo9p(a STRUCT) RETURNS STRUCT RETURN a; +SELECT foo9p(STRUCT(1, 'hello')); +SELECT foo9p(from_json('{1:"hello"}', 'struct')); + +-- ARRAY of STRUCT +CREATE FUNCTION foo9q(a ARRAY>) RETURNS ARRAY> RETURN a; +SELECT foo9q(ARRAY(STRUCT(1, 'hello'), STRUCT(2, 'world'))); +SELECT foo9q(ARRAY(NAMED_STRUCT('x', 1, 'y', 'hello'), NAMED_STRUCT('x', 2, 'y', 'world'))); +SELECT foo9q(from_json('[{1:"hello"}, {2:"world"}]', 'array>')); + +-- ARRAY of MAP +CREATE FUNCTION foo9r(a ARRAY>) RETURNS ARRAY> RETURN a; +SELECT foo9r(ARRAY(MAP('hello', 1), MAP('world', 2))); +SELECT foo9r(from_json('[{"hello":1}, {"world":2}]', 'array>')); + +-- 1.10 Proper name resolution when referencing another function +CREATE OR REPLACE FUNCTION foo1_10(a INT) RETURNS INT RETURN a + 2; +CREATE OR REPLACE FUNCTION bar1_10(b INT) RETURNS STRING RETURN foo1_10(TRY_CAST(b AS STRING)); +SELECT bar1_10(3); + +-- 1.11 Optional return types (type inference) +-- 1.11.a Scalar UDF without RETURNS clause - return type inferred from body +-- Simple literal return +CREATE OR REPLACE FUNCTION foo1_11a() RETURN 42; +-- Expect: 42 +SELECT foo1_11a(); + +-- String literal return +CREATE OR REPLACE FUNCTION foo1_11b() RETURN 'hello world'; +-- Expect: 'hello world' +SELECT foo1_11b(); + +-- Expression return - should infer INT +CREATE OR REPLACE FUNCTION foo1_11c(a INT, b INT) RETURN a + b; +-- Expect: 8 +SELECT foo1_11c(3, 5); + +-- Expression return - should infer DOUBLE +CREATE OR REPLACE FUNCTION foo1_11d(a DOUBLE, b INT) RETURN a * b + 1.5; +-- Expect: 16.5 +SELECT foo1_11d(3.0, 5); + +-- Boolean expression return +CREATE OR REPLACE FUNCTION foo1_11e(a INT) RETURN a > 10; +-- Expect: true, false +SELECT foo1_11e(15), foo1_11e(5); + +-- Date arithmetic return +CREATE OR REPLACE FUNCTION foo1_11f(d DATE) RETURN d + INTERVAL '1' DAY; +-- Expect: 2024-01-02 +SELECT foo1_11f(DATE '2024-01-01'); + +-- Array return +CREATE OR REPLACE FUNCTION foo1_11g(n INT) RETURN ARRAY(1, 2, n); +-- Expect: [1, 2, 5] +SELECT foo1_11g(5); + +-- Struct return +CREATE OR REPLACE FUNCTION foo1_11h(a INT, b STRING) RETURN STRUCT(a, b); +-- Expect: {1, 'test'} +SELECT foo1_11h(1, 'test'); + +-- Subquery return - scalar +CREATE OR REPLACE FUNCTION foo1_11i(x INT) RETURN (SELECT x * 2); +-- Expect: 10 +SELECT foo1_11i(5); + +-- Function call return +CREATE OR REPLACE FUNCTION foo1_11j(s STRING) RETURN UPPER(s); +-- Expect: 'HELLO' +SELECT foo1_11j('hello'); + +-- Complex expression with multiple types +CREATE OR REPLACE FUNCTION foo1_11k(a INT, b STRING) RETURN CONCAT(CAST(a AS STRING), '_', b); +-- Expect: '123_test' +SELECT foo1_11k(123, 'test'); + +-- 1.11.b Table UDF without TABLE schema - schema inferred from body +-- Simple SELECT with literals +CREATE OR REPLACE FUNCTION foo1_11l() RETURNS TABLE RETURN SELECT 1 as id, 'hello' as name; +-- Expect: (1, 'hello') +SELECT * FROM foo1_11l(); + +-- SELECT with expressions +CREATE OR REPLACE FUNCTION foo1_11m(a INT, b STRING) RETURNS TABLE RETURN SELECT a * 2 as doubled, UPPER(b) as upper_name; +-- Expect: (10, 'WORLD') +SELECT * FROM foo1_11m(5, 'world'); + +-- SELECT with complex data types +CREATE OR REPLACE FUNCTION foo1_11n(arr ARRAY) RETURNS TABLE RETURN SELECT size(arr) as array_size, arr[0] as first_element; +-- Expect: (3, 1) +SELECT * FROM foo1_11n(ARRAY(1, 2, 3)); + +-- SELECT with struct columns +CREATE OR REPLACE FUNCTION foo1_11o(id INT, name STRING) RETURNS TABLE RETURN SELECT STRUCT(id, name) as person_info, id + 100 as modified_id; +-- Expect: ({1, 'Alice'}, 101) +SELECT * FROM foo1_11o(1, 'Alice'); + +------------------------------- +-- 2. Scalar SQL UDF +-- 2.1 deterministic simple expressions +CREATE FUNCTION foo2_1a(a INT) RETURNS INT RETURN a; +SELECT foo2_1a(5); + +CREATE FUNCTION foo2_1b(a INT, b INT) RETURNS INT RETURN a + b; +SELECT foo2_1b(5, 6); + +CREATE FUNCTION foo2_1c(a INT, b INT) RETURNS INT RETURN 10 * (a + b) + 100 * (a -b); +SELECT foo2_1c(5, 6); + +CREATE FUNCTION foo2_1d(a INT, b INT) RETURNS INT RETURN ABS(a) - LENGTH(CAST(b AS VARCHAR(10))); +SELECT foo2_1d(-5, 6); + +-- 2.2 deterministic complex expression with subqueries +-- 2.2.1 Nested Scalar subqueries +CREATE FUNCTION foo2_2a(a INT) RETURNS INT RETURN SELECT a; +SELECT foo2_2a(5); + +CREATE FUNCTION foo2_2b(a INT) RETURNS INT RETURN 1 + (SELECT a); +SELECT foo2_2b(5); + +-- Expect error: deep correlation is not yet supported +CREATE FUNCTION foo2_2c(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT a)); +-- SELECT foo2_2c(5); + +-- Expect error: deep correlation is not yet supported +CREATE FUNCTION foo2_2d(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT (SELECT (SELECT a)))); +-- SELECT foo2_2d(5); + +-- 2.2.2 Set operations +-- Expect error: correlated scalar subquery must be aggregated. +CREATE FUNCTION foo2_2e(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) WHERE c1 = 2 +UNION ALL +SELECT a + 1 FROM (VALUES 1) AS V(c1); +-- SELECT foo2_2e(5); + +-- Expect error: correlated scalar subquery must be aggregated. +CREATE FUNCTION foo2_2f(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) +EXCEPT +SELECT a + 1 FROM (VALUES 1) AS V(a); +-- SELECT foo2_2f(5); + +-- Expect error: correlated scalar subquery must be aggregated. +CREATE FUNCTION foo2_2g(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) +INTERSECT +SELECT a FROM (VALUES 1) AS V(a); +-- SELECT foo2_2g(5); + +-- Prepare by dropping views or tables if they already exist. +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS ts; +DROP TABLE IF EXISTS tm; +DROP TABLE IF EXISTS ta; +DROP TABLE IF EXISTS V1; +DROP TABLE IF EXISTS V2; +DROP VIEW IF EXISTS t1; +DROP VIEW IF EXISTS t2; +DROP VIEW IF EXISTS ts; +DROP VIEW IF EXISTS tm; +DROP VIEW IF EXISTS ta; +DROP VIEW IF EXISTS V1; +DROP VIEW IF EXISTS V2; + +-- 2.3 Calling Scalar UDF from various places +CREATE FUNCTION foo2_3(a INT, b INT) RETURNS INT RETURN a + b; +CREATE VIEW V1(c1, c2) AS VALUES (1, 2), (3, 4), (5, 6); +CREATE VIEW V2(c1, c2) AS VALUES (-1, -2), (-3, -4), (-5, -6); + +-- 2.3.1 Multiple times in the select list +SELECT foo2_3(c1, c2), foo2_3(c2, 1), foo2_3(c1, c2) - foo2_3(c2, c1 - 1) FROM V1 ORDER BY 1, 2, 3; + +-- 2.3.2 In the WHERE clause +SELECT * FROM V1 WHERE foo2_3(c1, 0) = c1 AND foo2_3(c1, c2) < 8; + +-- 2.3.3 Different places around an aggregate +SELECT foo2_3(SUM(c1), SUM(c2)), SUM(c1) + SUM(c2), SUM(foo2_3(c1, c2) + foo2_3(c2, c1) - foo2_3(c2, c1)) +FROM V1; + +-- 2.4 Scalar UDF with complex one row relation subquery +-- 2.4.1 higher order functions +CREATE FUNCTION foo2_4a(a ARRAY) RETURNS STRING RETURN +SELECT array_sort(a, (i, j) -> rank[i] - rank[j])[0] FROM (SELECT MAP('a', 1, 'b', 2) rank); + +SELECT foo2_4a(ARRAY('a', 'b')); + +-- 2.4.2 built-in functions +CREATE FUNCTION foo2_4b(m MAP, k STRING) RETURNS STRING RETURN +SELECT v || ' ' || v FROM (SELECT upper(m[k]) AS v); + +SELECT foo2_4b(map('a', 'hello', 'b', 'world'), 'a'); + +-- Clean up +DROP VIEW V2; +DROP VIEW V1; + +-- 3. Misc +CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (0, 2), (1, 2); +CREATE VIEW t2(c1, c2) AS VALUES (0, 2), (0, 3); +CREATE VIEW ts(x) AS VALUES NAMED_STRUCT('a', 1, 'b', 2); +CREATE VIEW tm(x) AS VALUES MAP('a', 1, 'b', 2); +CREATE VIEW ta(x) AS VALUES ARRAY(1, 2, 3); + +-- 3.1 deterministic functions +CREATE FUNCTION foo3_1a(a DOUBLE, b DOUBLE) RETURNS DOUBLE RETURN a * b; +CREATE FUNCTION foo3_1b(x INT) RETURNS INT RETURN x; +CREATE FUNCTION foo3_1c(x INT) RETURNS INT RETURN SELECT x; +CREATE FUNCTION foo3_1d(x INT) RETURNS INT RETURN (SELECT SUM(c2) FROM t2 WHERE c1 = x); +CREATE FUNCTION foo3_1e() RETURNS INT RETURN foo3_1d(0); +-- Function body is a uncorrelated scalar subquery. +CREATE FUNCTION foo3_1f() RETURNS INT RETURN SELECT SUM(c2) FROM t2 WHERE c1 = 0; +CREATE FUNCTION foo3_1g(x INT) RETURNS INT RETURN SELECT (SELECT x); + +-- 3.1.1 scalar function in various operators +-- in project +SELECT a, b, foo3_1a(a + 1, b + 1) FROM t1 AS t(a, b); +SELECT x, foo3_1c(x) FROM t1 AS t(x, y); +SELECT c1, foo3_1d(c1) FROM t1; + +-- in project, with nested SQL functions +SELECT c1, foo3_1a(foo3_1b(c1), foo3_1b(c1)) FROM t1; +SELECT c1, foo3_1d(foo3_1c(foo3_1b(c1))) FROM t1; +SELECT c1, foo3_1a(foo3_1c(foo3_1b(c1)), foo3_1d(foo3_1b(c1))) FROM t1; +SELECT foo3_1c(foo3_1e()) FROM t1; + +-- in aggregate +SELECT foo3_1a(MAX(c1), MAX(c2)) FROM t1; +SELECT foo3_1a(MAX(c1), c2) FROM t1 GROUP BY c2; +SELECT foo3_1a(c1, c2) FROM t1 GROUP BY c1, c2; +SELECT MAX(foo3_1a(c1, c2)) FROM t1 GROUP BY c1, c2; +SELECT MAX(c1) + foo3_1b(MAX(c1)) FROM t1 GROUP BY c2; +SELECT c1, SUM(foo3_1c(c2)) FROM t1 GROUP BY c1; +SELECT c1, SUM(foo3_1d(c2)) FROM t1 GROUP BY c1; +SELECT foo3_1c(c1), foo3_1d(c1) FROM t1 GROUP BY c1; + +-- in aggregate, with non-deterministic input +SELECT foo3_1a(SUM(c1), rand(0) * 0) FROM t1; +SELECT foo3_1a(SUM(c1) + rand(0) * 0, SUM(c2)) FROM t1; +SELECT foo3_1b(SUM(c1) + rand(0) * 0) FROM t1; +SELECT foo3_1b(SUM(1) + rand(0) * 0) FROM t1 GROUP BY c2; +SELECT foo3_1c(SUM(c2) + rand(0) * 0) FROM t1 GROUP by c1; + +-- in aggregate, with nested SQL functions +SELECT foo3_1b(foo3_1b(MAX(c2))) FROM t1; +SELECT foo3_1b(MAX(foo3_1b(c2))) FROM t1; +SELECT foo3_1a(foo3_1b(c1), MAX(c2)) FROM t1 GROUP BY c1; + +-- in aggregate, with grouping expressions +SELECT c1, foo3_1b(c1) FROM t1 GROUP BY c1; +SELECT c1, foo3_1b(c1 + 1) FROM t1 GROUP BY c1; +SELECT c1, foo3_1b(c1 + rand(0) * 0) FROM t1 GROUP BY c1; +SELECT c1, foo3_1a(c1, MIN(c2)) FROM t1 GROUP BY c1; +SELECT c1, foo3_1a(c1 + 1, MIN(c2 + 1)) FROM t1 GROUP BY c1; +SELECT c1, c2, foo3_1a(c1, c2) FROM t1 GROUP BY c1, c2; +SELECT c1, c2, foo3_1a(1, 2) FROM t1 GROUP BY c1, c2; +SELECT c1 + c2, foo3_1b(c1 + c2 + 1) FROM t1 GROUP BY c1 + c2; +SELECT COUNT(*) + foo3_1b(c1) + foo3_1b(SUM(c2)) + SUM(foo3_1b(c2)) FROM t1 GROUP BY c1; + +-- in aggregate, with having expressions +SELECT c1, COUNT(*), foo3_1b(SUM(c2)) FROM t1 GROUP BY c1 HAVING COUNT(*) > 0; +SELECT c1, COUNT(*), foo3_1b(SUM(c2)) FROM t1 GROUP BY c1 HAVING foo3_1b(SUM(c2)) > 0; +-- Expect failure +SELECT c1, COUNT(*), foo3_1b(SUM(c2)) FROM t1 GROUP BY c1 HAVING SUM(foo3_1b(c2)) > 0; + +-- in aggregate, with sql function in group by columns +SELECT foo3_1b(c1), MIN(c2) FROM t1 GROUP BY 1; +SELECT foo3_1a(c1 + rand(0) * 0, c2) FROM t1 GROUP BY 1; +SELECT c1, c2, foo3_1a(c1, c2) FROM t1 GROUP BY c1, c2, 3; + +-- in aggregate, with scalar subquery +SELECT c1, (SELECT c1), (SELECT foo3_1b(c1)), SUM(c2) FROM t1 GROUP BY 1, 2, 3; +SELECT c1, SUM(c2) + foo3_1a(MIN(c2), MAX(c2)) + (SELECT SUM(c2)) FROM t1 GROUP BY c1; +SELECT foo3_1b(SUM(c1)) + (SELECT foo3_1b(SUM(c1))) FROM t1; + +-- in aggregate, with invalid aggregate expressions +SELECT SUM(foo3_1b(SUM(c1))) FROM t1; +SELECT foo3_1b(SUM(c1)) + (SELECT SUM(SUM(c1))) FROM t1; +SELECT foo3_1b(SUM(c1) + SUM(SUM(c1))) FROM t1; +SELECT foo3_1b(SUM(c1 + rand(0) * 0)) FROM t1; +SELECT SUM(foo3_1b(c1) + rand(0) * 0) FROM t1; + +-- in aggregate, with non-deterministic function input inside aggregate expression +SELECT SUM(foo3_1b(c1 + rand(0) * 0)) FROM t1; + +-- in aggregate, with nested SQL functions +SELECT foo3_1b(SUM(c1) + foo3_1b(SUM(c1))) FROM t1; +SELECT foo3_1b(SUM(c2) + foo3_1b(SUM(c1))) AS foo FROM t1 HAVING foo > 0; +SELECT c1, COUNT(*), foo3_1b(SUM(c2) + foo3_1b(SUM(c2))) FROM t1 GROUP BY c1 HAVING COUNT(*) > 0; + +-- in aggregate, with invalid group by +SELECT foo3_1a(c1, MAX(c2)) FROM t1 GROUP BY c1, 1; + +-- in CTE +WITH cte AS (SELECT foo3_1a(c1, c2) FROM t1) +SELECT * FROM cte; + +-- in GROUP BY +SELECT SUM(c2) FROM t1 GROUP BY foo3_1b(c1); +SELECT foo3_1b(c1), SUM(c2) FROM t1 GROUP BY 1; +SELECT foo3_1b(c1), c2, GROUPING(foo3_1b(c1)), SUM(c1) FROM t1 GROUP BY ROLLUP(foo3_1b(c1), c2); + +-- in HAVING +SELECT c1, SUM(c2) FROM t1 GROUP BY c1 HAVING foo3_1b(SUM(c2)) > 1; +SELECT c1, SUM(c2) FROM t1 GROUP BY CUBE(c1) HAVING foo3_1b(GROUPING(c1)) = 0; + +-- in join +SELECT * FROM t1 JOIN t2 ON foo3_1a(t1.c1, t2.c2) >= 2; +SELECT * FROM t1 JOIN t2 ON foo3_1b(t1.c2) = foo3_1b(t2.c2); +SELECT * FROM t1 JOIN t2 ON foo3_1b(t1.c1 + t2.c1 + 2) > 2; +SELECT * FROM t1 JOIN t2 ON foo3_1a(foo3_1b(t1.c1), t2.c2) >= 2; +-- in join with non-correlated scalar subquery +SELECT * FROM t1 JOIN t2 ON foo3_1f() > 0; +-- expect error: non-deterministic expressions cannot be used in Join +SELECT * FROM t1 JOIN t2 ON foo3_1b(t1.c1 + rand(0) * 0) > 1; +-- this works because the analyzer interprets the function body of 'SELECT x' as just 'x' now +SELECT * FROM t1 JOIN t2 ON foo3_1c(t1.c1) = 2; +-- expect error: correlated scalar subquery cannot be used in Join +SELECT * FROM t1 JOIN t2 ON foo3_1g(t1.c1) = 2; + +-- in sort: unsupported +SELECT * FROM t1 ORDER BY foo3_1b(c1); + +-- in limit: unsupported +SELECT * FROM t1 LIMIT foo3_1b(1); + +-- in generate: unsupported +SELECT * FROM ta LATERAL VIEW EXPLODE(ARRAY(foo3_1b(x[0]), foo3_1b(x[1]))) AS t; + +-- 3.1.2 scalar function with various function inputs +-- with non-deterministic expressions +SELECT CASE WHEN foo3_1b(rand(0) * 0 < 1 THEN 1 ELSE -1 END; + +-- with outer references +SELECT (SELECT SUM(c2) FROM t2 WHERE c1 = foo3_1b(t1.c1)) FROM t1; + +-- with uncorrelated scalar subquery +SELECT foo3_1b((SELECT SUM(c1) FROM t1)); +SELECT foo3_1a(c1, (SELECT MIN(c1) FROM t1)) FROM t1; + +-- with correlated scalar subquery +SELECT foo3_1b((SELECT SUM(c1))) FROM t1; +SELECT foo3_1b((SELECT SUM(c1) FROM t1 WHERE c2 = t2.c2)) FROM t2; +SELECT c2, AVG(foo3_1b((SELECT COUNT(*) FROM t1 WHERE c2 = t2.c2))) OVER (PARTITION BY c1) AS r FROM t2; + +-- 3.1.3 scalar function with complex data type +CREATE FUNCTION foo3_1x(x STRUCT) RETURNS INT RETURN x.a + x.b; +CREATE FUNCTION foo3_1y(x ARRAY) RETURNS INT RETURN aggregate(x, BIGINT(0), (x, y) -> x + y); + +-- with struct type +SELECT foo3_1a(x.a, x.b) FROM ts; +SELECT foo3_1x(x) FROM ts; + +-- with map type +SELECT foo3_1a(x['a'], x['b']) FROM tm; + +-- with array type +SELECT foo3_1a(x[0], x[1]) FROM ta; +SELECT foo3_1y(x) FROM ta; + +-- 3.2 Scalar function with complex function body +-- 3.2.a Non-deterministic expression +CREATE FUNCTION foo3_2a() RETURNS INT RETURN FLOOR(RAND() * 6) + 1; + +SELECT CASE WHEN foo3_2a() > 6 THEN FALSE ELSE TRUE END; +-- Expect error: non-deterministic expressions cannot be used in Join +SELECT * FROM t1 JOIN t2 ON foo3_2a() = 1; + +-- 3.2.b IN subqueries +CREATE FUNCTION foo3_2b1(x INT) RETURNS BOOLEAN RETURN x IN (SELECT 1); +SELECT * FROM t1 WHERE foo3_2b1(c1); + +CREATE FUNCTION foo3_2b2(x INT) RETURNS INT RETURN IF(x IN (SELECT 1), 1, 0); +SELECT * FROM t1 WHERE foo3_2b2(c1) = 0; +SELECT foo3_2b2(c1) FROM t1; + +CREATE FUNCTION foo3_2b3(x INT) RETURNS BOOLEAN RETURN x IN (SELECT c1 FROM t2); +SELECT * FROM t1 WHERE foo3_2b3(c1); + +CREATE FUNCTION foo3_2b4(x INT) RETURNS BOOLEAN RETURN x NOT IN (SELECT c2 FROM t2 WHERE x = c1); +SELECT * FROM t1 WHERE foo3_2b4(c1); + +-- Expect error +CREATE FUNCTION foo3_2b5(x INT) RETURNS BOOLEAN RETURN SUM(1) + IF(x IN (SELECT 1), 1, 0); +CREATE FUNCTION foo3_2b5(x INT) RETURNS BOOLEAN RETURN y IN (SELECT 1); +CREATE FUNCTION foo3_2b5(x INT) RETURNS BOOLEAN RETURN x IN (SELECT x WHERE x = 1); + +-- 3.2.c EXISTS subqueries +CREATE FUNCTION foo3_2c1(x INT) RETURNS BOOLEAN RETURN EXISTS(SELECT 1); +SELECT * FROM t1 WHERE foo3_2c1(c1); + +CREATE FUNCTION foo3_2c2(x INT) RETURNS BOOLEAN RETURN NOT EXISTS(SELECT * FROM t2 WHERE c1 = x); +SELECT * FROM t1 WHERE foo3_2c2(c1); + +-- 3.2.d with nested subquery: not supported +CREATE FUNCTION foo3_2d1(x INT) RETURNS INT RETURN SELECT (SELECT x); +CREATE FUNCTION foo3_2d2(x INT) RETURNS INT RETURN SELECT (SELECT 1 WHERE EXISTS (SELECT * FROM t2 WHERE c1 = x)); + +-- 3.2.e CTEs +CREATE FUNCTION foo3_2e1( + occurrences ARRAY>, + instance_start_time TIMESTAMP +) RETURNS STRING RETURN +WITH t AS ( + SELECT transform(occurrences, x -> named_struct( + 'diff', abs(unix_millis(x.start_time) - unix_millis(instance_start_time)), + 'id', x.occurrence_id + )) AS diffs +) +SELECT CASE WHEN occurrences IS NULL OR size(occurrences) = 0 + THEN NULL + ELSE sort_array(diffs)[0].id END AS id +FROM t; + +SELECT foo3_2e1( + ARRAY(STRUCT('2022-01-01 10:11:12', '1'), STRUCT('2022-01-01 10:11:15', '2')), + '2022-01-01'); + +-- 3.3 Create and invoke function with different SQL configurations +SET spark.sql.ansi.enabled=true; +CREATE FUNCTION foo3_3a(x INT) RETURNS DOUBLE RETURN 1 / x; +CREATE FUNCTION foo3_3at(x INT) RETURNS TABLE (a DOUBLE) RETURN SELECT 1 / x; +CREATE TEMPORARY FUNCTION foo3_3b(x INT) RETURNS DOUBLE RETURN 1 / x; +SET spark.sql.ansi.enabled=false; +-- Expect ArithmeticException +SELECT foo3_3a(0); +SELECT foo3_3b(0); +SELECT * FROM foo3_3at(0); +-- Replace the functions with different configs. +CREATE OR REPLACE FUNCTION foo3_3a(x INT) RETURNS DOUBLE RETURN 1 / x; +CREATE OR REPLACE FUNCTION foo3_3at(x INT) RETURNS TABLE (a DOUBLE) RETURN SELECT 1 / x; +CREATE OR REPLACE TEMPORARY FUNCTION foo3_3b(x INT) RETURNS DOUBLE RETURN 1 / x; +-- Expect null +SELECT foo3_3a(0); +SELECT foo3_3b(0); +SELECT * FROM foo3_3at(0); + +-- Cast inside the UDF should respect the captured SQL configurations +-- Explicit cast +CREATE FUNCTION foo3_3c() RETURNS INT RETURN CAST('a' AS INT); +CREATE FUNCTION foo3_3ct() RETURNS TABLE (a INT) RETURN SELECT CAST('a' AS INT); +-- Implicit cast +CREATE FUNCTION foo3_3d() RETURNS INT RETURN 'a' + 1; +CREATE FUNCTION foo3_3dt() RETURNS TABLE (a INT) RETURN SELECT 'a' + 1; +-- Expect null +SELECT foo3_3c(); +SELECT foo3_3d(); +SELECT * FROM foo3_3ct(); +SELECT * FROM foo3_3dt(); +SET spark.sql.ansi.enabled=true; +-- Expect null +SELECT foo3_3c(); +SELECT foo3_3d(); +SELECT * FROM foo3_3ct(); +SELECT * FROM foo3_3dt(); +RESET spark.sql.ansi.enabled; + +-- 3.4 Cyclic function reference +CREATE FUNCTION foo3_4a(x INT) RETURNS INT RETURN 1; +CREATE FUNCTION foo3_4b(x INT) RETURNS INT RETURN foo3_4a(x); +CREATE FUNCTION foo3_4c(x INT) RETURNS INT RETURN foo3_4b(x); +CREATE FUNCTION foo3_4d(x INT) RETURNS INT RETURN (SELECT foo3_4c(x)); +CREATE FUNCTION foo3_4e(x INT) RETURNS TABLE(a INT) RETURN SELECT foo3_4a(x); +CREATE FUNCTION foo3_4f(x INT) RETURNS TABLE(b INT) RETURN SELECT * FROM foo3_4e(x); +CREATE OR REPLACE TEMP FUNCTION foo3_4g(x INT) RETURN x + 1; +CREATE OR REPLACE TEMP FUNCTION foo3_4h(x INT) RETURN foo3_4g(x) + 1; +-- Expect error +CREATE OR REPLACE FUNCTION foo3_4a(x INT) RETURNS INT RETURN foo3_4b(x); +CREATE OR REPLACE FUNCTION foo3_4a(x INT) RETURNS INT RETURN foo3_4c(x); +CREATE OR REPLACE FUNCTION foo3_4a(x INT) RETURNS INT RETURN foo3_4d(x); +CREATE OR REPLACE FUNCTION foo3_4a(x INT) RETURN FoO3_4b(x); +CREATE OR REPLACE FUNCTION foo3_4a(x INT) RETURNS INT RETURN SELECT SUM(a) FROM foo3_4e(x); +CREATE OR REPLACE FUNCTION foo3_4e(x INT) RETURNS TABLE (c INT) RETURN SELECT * FROM foo3_4f(x); +CREATE OR REPLACE FUNCTION foo3_4e(x INT) RETURNS TABLE RETURN SELECT * FROM fOo3_4F(x); +CREATE OR REPLACE TEMP FUNCTION foo3_4g(x INT) RETURN foo3_4h(x) + 1; + +-- 3.5 Permanent functions with temp objects are not allowed +CREATE TEMPORARY VIEW t AS VALUES (0) t(a); +CREATE TEMPORARY FUNCTION foo3_5a(x INT) RETURNS INT RETURN x; +CREATE TEMPORARY FUNCTION foo3_5b(x INT) RETURNS INT RETURN (SELECT SUM(a) FROM t); +CREATE TEMPORARY FUNCTION foo3_5c(x INT) RETURNS TABLE (a INT) RETURN SELECT a FROM t; +-- Expect error: permanent function cannot reference temporary function +CREATE FUNCTION foo3_5d(x INT) RETURNS INT RETURN foo3_5a(x); +CREATE FUNCTION foo3_5d(x INT) RETURNS TABLE (a INT) RETURN SELECT foo3_5a(x); +-- Expect error: permanent function cannot reference temporary view +CREATE FUNCTION foo3_5d(x INT) RETURNS INT RETURN (SELECT SUM(a) FROM t); +CREATE FUNCTION foo3_5d(x INT) RETURNS TABLE (a INT) RETURN SELECT a FROM t; + +-- 3.14 Invalid usage of SQL scalar/table functions in query clauses. +CREATE FUNCTION foo3_14a() RETURNS INT RETURN 1; +CREATE FUNCTION foo3_14b() RETURNS TABLE (a INT) RETURN SELECT 1; +-- Expect error +SELECT * FROM foo3_14a(); +SELECT foo3_14b(); + +-- 4. SQL table functions +CREATE FUNCTION foo4_0() RETURNS TABLE (x INT) RETURN SELECT 1; +CREATE FUNCTION foo4_1(x INT) RETURNS TABLE (a INT) RETURN SELECT x; +CREATE FUNCTION foo4_2(x INT) RETURNS TABLE (a INT) RETURN SELECT c2 FROM t2 WHERE c1 = x; +CREATE FUNCTION foo4_3(x INT) RETURNS TABLE (a INT, cnt INT) RETURN SELECT c1, COUNT(*) FROM t2 WHERE c1 = x GROUP BY c1; + +-- 4.1 SQL table function with literals +SELECT * FROM foo4_0(); +SELECT * FROM foo4_1(1); +SELECT * FROM foo4_2(2); +SELECT * FROM foo4_3(0); +-- with non-deterministic inputs +SELECT * FROM foo4_1(rand(0) * 0); +-- named arguments +SELECT * FROM foo4_1(x => 1); + +-- 4.2 SQL table function with lateral references +SELECT * FROM t1, LATERAL foo4_1(c1); +SELECT * FROM t1, LATERAL foo4_2(c1); +SELECT * FROM t1 JOIN LATERAL foo4_2(c1) ON t1.c2 = foo4_2.a; +SELECT * FROM t1, LATERAL foo4_3(c1); +SELECT * FROM t1, LATERAL (SELECT cnt FROM foo4_3(c1)); +SELECT * FROM t1, LATERAL foo4_1(c1 + rand(0) * 0); + +-- 4.3 multiple SQL table functions +SELECT * FROM t1 JOIN foo4_1(1) AS foo4_1(x) ON t1.c1 = foo4_1.x; +SELECT * FROM t1, LATERAL foo4_1(c1), LATERAL foo4_2(foo4_1.a + c1); + +-- 4.4 table functions inside scalar subquery +SELECT (SELECT MAX(a) FROM foo4_1(c1)) FROM t1; +SELECT (SELECT MAX(a) FROM foo4_1(c1) WHERE a = c2) FROM t1; +SELECT (SELECT MAX(cnt) FROM foo4_3(c1)) FROM t1; + +-- Clean up +DROP VIEW t1; +DROP VIEW t2; + +DROP FUNCTION IF EXISTS foo1a0; +DROP FUNCTION IF EXISTS foo1a1; +DROP FUNCTION IF EXISTS foo1a2; +DROP FUNCTION IF EXISTS foo1b0; +DROP FUNCTION IF EXISTS foo1b1; +DROP FUNCTION IF EXISTS foo1b2; +DROP FUNCTION IF EXISTS foo1c1; +DROP FUNCTION IF EXISTS foo1c2; +DROP FUNCTION IF EXISTS foo1d1; +DROP FUNCTION IF EXISTS foo1d2; +DROP FUNCTION IF EXISTS foo1d4; +DROP FUNCTION IF EXISTS foo1d5; +DROP FUNCTION IF EXISTS foo1d6; +DROP FUNCTION IF EXISTS foo1e1; +DROP FUNCTION IF EXISTS foo1e2; +DROP FUNCTION IF EXISTS foo1e3; +DROP FUNCTION IF EXISTS foo1f1; +DROP FUNCTION IF EXISTS foo1f2; +DROP FUNCTION IF EXISTS foo1g1; +DROP FUNCTION IF EXISTS foo1g2; +DROP FUNCTION IF EXISTS foo2a0; +DROP FUNCTION IF EXISTS foo2a2; +DROP FUNCTION IF EXISTS foo2a4; +DROP FUNCTION IF EXISTS foo2b1; +DROP FUNCTION IF EXISTS foo2b2; +DROP FUNCTION IF EXISTS foo2c1; +DROP FUNCTION IF EXISTS foo31; +DROP FUNCTION IF EXISTS foo32; +DROP FUNCTION IF EXISTS foo33; +DROP FUNCTION IF EXISTS foo41; +DROP FUNCTION IF EXISTS foo42; +DROP FUNCTION IF EXISTS foo51; +DROP FUNCTION IF EXISTS foo52; +DROP FUNCTION IF EXISTS foo6c; +DROP FUNCTION IF EXISTS foo6d; +DROP FUNCTION IF EXISTS foo7a; +DROP FUNCTION IF EXISTS foo7at; +DROP FUNCTION IF EXISTS foo9a; +DROP FUNCTION IF EXISTS foo9b; +DROP FUNCTION IF EXISTS foo9c; +DROP FUNCTION IF EXISTS foo9d; +DROP FUNCTION IF EXISTS foo9e; +DROP FUNCTION IF EXISTS foo9f; +DROP FUNCTION IF EXISTS foo9g; +DROP FUNCTION IF EXISTS foo9h; +DROP FUNCTION IF EXISTS foo9i; +DROP FUNCTION IF EXISTS foo9j; +DROP FUNCTION IF EXISTS foo9l; +DROP FUNCTION IF EXISTS foo9m; +DROP FUNCTION IF EXISTS foo9n; +DROP FUNCTION IF EXISTS foo9o; +DROP FUNCTION IF EXISTS foo9p; +DROP FUNCTION IF EXISTS foo9q; +DROP FUNCTION IF EXISTS foo9r; +DROP FUNCTION IF EXISTS foo1_10; +DROP FUNCTION IF EXISTS bar1_10; +DROP FUNCTION IF EXISTS foo1_11a; +DROP FUNCTION IF EXISTS foo1_11b; +DROP FUNCTION IF EXISTS foo1_11c; +DROP FUNCTION IF EXISTS foo1_11d; +DROP FUNCTION IF EXISTS foo1_11e; +DROP FUNCTION IF EXISTS foo1_11f; +DROP FUNCTION IF EXISTS foo1_11g; +DROP FUNCTION IF EXISTS foo1_11h; +DROP FUNCTION IF EXISTS foo1_11i; +DROP FUNCTION IF EXISTS foo1_11j; +DROP FUNCTION IF EXISTS foo1_11k; +DROP FUNCTION IF EXISTS foo1_11l; +DROP FUNCTION IF EXISTS foo1_11m; +DROP FUNCTION IF EXISTS foo1_11n; +DROP FUNCTION IF EXISTS foo1_11o; +DROP FUNCTION IF EXISTS foo2_1a; +DROP FUNCTION IF EXISTS foo2_1b; +DROP FUNCTION IF EXISTS foo2_1c; +DROP FUNCTION IF EXISTS foo2_1d; +DROP FUNCTION IF EXISTS foo2_2a; +DROP FUNCTION IF EXISTS foo2_2b; +DROP FUNCTION IF EXISTS foo2_2c; +DROP FUNCTION IF EXISTS foo2_2d; +DROP FUNCTION IF EXISTS foo2_2e; +DROP FUNCTION IF EXISTS foo2_2f; +DROP FUNCTION IF EXISTS foo2_2g; +DROP FUNCTION IF EXISTS foo2_3; +DROP FUNCTION IF EXISTS foo2_4a; +DROP FUNCTION IF EXISTS foo2_4b; +DROP FUNCTION IF EXISTS foo3_1a; +DROP FUNCTION IF EXISTS foo3_1b; +DROP FUNCTION IF EXISTS foo3_1c; +DROP FUNCTION IF EXISTS foo3_1d; +DROP FUNCTION IF EXISTS foo3_1e; +DROP FUNCTION IF EXISTS foo3_1f; +DROP FUNCTION IF EXISTS foo3_1g; +DROP FUNCTION IF EXISTS foo3_1x; +DROP FUNCTION IF EXISTS foo3_1y; +DROP FUNCTION IF EXISTS foo3_2a; +DROP FUNCTION IF EXISTS foo3_2b1; +DROP FUNCTION IF EXISTS foo3_2b2; +DROP FUNCTION IF EXISTS foo3_2b3; +DROP FUNCTION IF EXISTS foo3_2b4; +DROP FUNCTION IF EXISTS foo3_2b5; +DROP FUNCTION IF EXISTS foo3_2c1; +DROP FUNCTION IF EXISTS foo3_2c2; +DROP FUNCTION IF EXISTS foo3_2d1; +DROP FUNCTION IF EXISTS foo3_2d2; +DROP FUNCTION IF EXISTS foo3_2e1; +DROP FUNCTION IF EXISTS foo3_3a; +DROP FUNCTION IF EXISTS foo3_3at; +DROP FUNCTION IF EXISTS foo3_14a; +DROP FUNCTION IF EXISTS foo3_14b; +DROP FUNCTION IF EXISTS foo3_3c; +DROP FUNCTION IF EXISTS foo3_3ct; +DROP FUNCTION IF EXISTS foo3_3d; +DROP FUNCTION IF EXISTS foo3_3dt; +DROP FUNCTION IF EXISTS foo3_4a; +DROP FUNCTION IF EXISTS foo3_4b; +DROP FUNCTION IF EXISTS foo3_4c; +DROP FUNCTION IF EXISTS foo3_4d; +DROP FUNCTION IF EXISTS foo3_4e; +DROP FUNCTION IF EXISTS foo3_4f; +DROP FUNCTION IF EXISTS foo4_0; +DROP FUNCTION IF EXISTS foo4_1; +DROP FUNCTION IF EXISTS foo4_2; +DROP FUNCTION IF EXISTS foo4_3; + +-- Drop temporary functions +DROP TEMPORARY FUNCTION IF EXISTS foo1d3; +DROP TEMPORARY FUNCTION IF EXISTS foo3_3b; +DROP TEMPORARY FUNCTION IF EXISTS foo3_5a; +DROP TEMPORARY FUNCTION IF EXISTS foo3_5b; +DROP TEMPORARY FUNCTION IF EXISTS foo3_5c; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/string-functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/string-functions.sql new file mode 100644 index 000000000000..c108f7c76f76 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/string-functions.sql @@ -0,0 +1,308 @@ +-- Argument number exception +select concat_ws(); +select format_string(); + +-- A pipe operator for string concatenation +select 'a' || 'b' || 'c'; + +-- replace function +select replace('abc', 'b', '123'); +select replace('abc', 'b'); + +-- uuid +select length(uuid()), (uuid() <> uuid()); + +-- position +select position('bar' in 'foobarbar'), position(null, 'foobarbar'), position('aaads', null); + +-- left && right +select left("abcd", 2), left("abcd", 5), left("abcd", '2'), left("abcd", null); +select left(null, -2); +select left("abcd", -2), left("abcd", 0), left("abcd", 'a'); +select right("abcd", 2), right("abcd", 5), right("abcd", '2'), right("abcd", null); +select right(null, -2); +select right("abcd", -2), right("abcd", 0), right("abcd", 'a'); + +-- split function +SELECT split('aa1cc2ee3', '[1-9]+'); +SELECT split('aa1cc2ee3', '[1-9]+', 2); +SELECT split('hello', ''); +SELECT split('', ''); +SELECT split('abc', null); +SELECT split(null, 'b'); + +-- split_part function +SELECT split_part('11.12.13', '.', 2); +SELECT split_part('11.12.13', '.', -1); +SELECT split_part('11.12.13', '.', -3); +SELECT split_part('11.12.13', '', 1); +SELECT split_part('11ab12ab13', 'ab', 1); +SELECT split_part('11.12.13', '.', 0); +SELECT split_part('11.12.13', '.', 4); +SELECT split_part('11.12.13', '.', 5); +SELECT split_part('11.12.13', '.', -5); +SELECT split_part(null, '.', 1); +SELECT split_part(str, delimiter, partNum) FROM VALUES ('11.12.13', '.', 3) AS v1(str, delimiter, partNum); + +-- substring function +SELECT substr('Spark SQL', 5); +SELECT substr('Spark SQL', -3); +SELECT substr('Spark SQL', 5, 1); +SELECT substr('Spark SQL' from 5); +SELECT substr('Spark SQL' from -3); +SELECT substr('Spark SQL' from 5 for 1); +SELECT substring('Spark SQL', 5); +SELECT substring('Spark SQL', -3); +SELECT substring('Spark SQL', 5, 1); +SELECT substring('Spark SQL' from 5); +SELECT substring('Spark SQL' from -3); +SELECT substring('Spark SQL' from 5 for 1); + +-- trim +SELECT trim(" xyz "), ltrim(" xyz "), rtrim(" xyz "); +SELECT trim(BOTH 'xyz' FROM 'yxTomxx'), trim('xyz' FROM 'yxTomxx'); +SELECT trim(BOTH 'x' FROM 'xxxbarxxx'), trim('x' FROM 'xxxbarxxx'); +SELECT trim(LEADING 'xyz' FROM 'zzzytest'); +SELECT trim(LEADING 'xyz' FROM 'zzzytestxyz'); +SELECT trim(LEADING 'xy' FROM 'xyxXxyLAST WORD'); +SELECT trim(TRAILING 'xyz' FROM 'testxxzx'); +SELECT trim(TRAILING 'xyz' FROM 'xyztestxxzx'); +SELECT trim(TRAILING 'xy' FROM 'TURNERyxXxy'); + +-- btrim +SELECT btrim('xyxtrimyyx', 'xy'); +SELECT btrim(encode(" xyz ", 'utf-8')); +SELECT btrim(encode('yxTomxx', 'utf-8'), encode('xyz', 'utf-8')); +SELECT btrim(encode('xxxbarxxx', 'utf-8'), encode('x', 'utf-8')); + +-- Check lpad/rpad with invalid length parameter +SELECT lpad('hi', 'invalid_length'); +SELECT rpad('hi', 'invalid_length'); + +-- lpad for BINARY inputs +SELECT hex(lpad(unhex(''), 5)); +SELECT hex(lpad(unhex('aabb'), 5)); +SELECT hex(lpad(unhex('aabbcc'), 2)); +SELECT hex(lpad(unhex('123'), 2)); +SELECT hex(lpad(unhex('12345'), 2)); +SELECT hex(lpad(unhex(''), 5, unhex('1f'))); +SELECT hex(lpad(unhex('aa'), 5, unhex('1f'))); +SELECT hex(lpad(unhex('aa'), 6, unhex('1f'))); +SELECT hex(lpad(unhex(''), 5, unhex('1f2e'))); +SELECT hex(lpad(unhex('aa'), 5, unhex('1f2e'))); +SELECT hex(lpad(unhex('aa'), 6, unhex('1f2e'))); +SELECT hex(lpad(unhex(''), 6, unhex(''))); +SELECT hex(lpad(unhex('aabbcc'), 6, unhex(''))); +SELECT hex(lpad(unhex('aabbcc'), 2, unhex('ff'))); + +-- rpad for BINARY inputs +SELECT hex(rpad(unhex(''), 5)); +SELECT hex(rpad(unhex('aabb'), 5)); +SELECT hex(rpad(unhex('aabbcc'), 2)); +SELECT hex(rpad(unhex('123'), 2)); +SELECT hex(rpad(unhex('12345'), 2)); +SELECT hex(rpad(unhex(''), 5, unhex('1f'))); +SELECT hex(rpad(unhex('aa'), 5, unhex('1f'))); +SELECT hex(rpad(unhex('aa'), 6, unhex('1f'))); +SELECT hex(rpad(unhex(''), 5, unhex('1f2e'))); +SELECT hex(rpad(unhex('aa'), 5, unhex('1f2e'))); +SELECT hex(rpad(unhex('aa'), 6, unhex('1f2e'))); +SELECT hex(rpad(unhex(''), 6, unhex(''))); +SELECT hex(rpad(unhex('aabbcc'), 6, unhex(''))); +SELECT hex(rpad(unhex('aabbcc'), 2, unhex('ff'))); + +-- lpad/rpad with mixed STRING and BINARY input +SELECT lpad('abc', 5, x'57'); +SELECT lpad(x'57', 5, 'abc'); +SELECT rpad('abc', 5, x'57'); +SELECT rpad(x'57', 5, 'abc'); + +-- encode +set spark.sql.legacy.javaCharsets=true; +select encode('hello', 'WINDOWS-1252'); +select encode(scol, ecol) from values('hello', 'WINDOWS-1252') as t(scol, ecol); +set spark.sql.legacy.javaCharsets=false; +select encode('hello', 'WINDOWS-1252'); +select encode(scol, ecol) from values('hello', 'WINDOWS-1252') as t(scol, ecol); +select encode('hello', 'Windows-xxx'); +select encode(scol, ecol) from values('hello', 'Windows-xxx') as t(scol, ecol); +set spark.sql.legacy.codingErrorAction=true; +select encode('渭城朝雨浥轻尘', 'US-ASCII'); +select encode(scol, ecol) from values('渭城朝雨浥轻尘', 'US-ASCII') as t(scol, ecol); +set spark.sql.legacy.codingErrorAction=false; +select encode('客舍青青柳色新', 'US-ASCII'); +select encode(scol, ecol) from values('客舍青青柳色新', 'US-ASCII') as t(scol, ecol); +select encode(decode(encode('白日依山尽,黄河入海流。欲穷千里目,更上一层楼。', 'UTF-16'), 'UTF-16'), 'UTF-8'); +select encode(decode(encode('南山經之首曰䧿山。其首曰招搖之山,臨於西海之上。', 'UTF-16'), 'UTF-16'), 'UTF-8'); +select encode(decode(encode('세계에서 가장 인기 있는 빅데이터 처리 프레임워크인 Spark', 'UTF-16'), 'UTF-16'), 'UTF-8'); +select encode(decode(encode('το Spark είναι το πιο δημοφιλές πλαίσιο επεξεργασίας μεγάλων δεδομένων παγκοσμίως', 'UTF-16'), 'UTF-16'), 'UTF-8'); +select encode(decode(encode('Sparkは世界で最も人気のあるビッグデータ処理フレームワークである。', 'UTF-16'), 'UTF-16'), 'UTF-8'); + +-- decode +select decode(); +select decode(encode('abc', 'utf-8')); +select decode(encode('abc', 'utf-8'), 'utf-8'); +select decode(encode('大千世界', 'utf-32'), 'utf-32'); +select decode(1, 1, 'Southlake'); +select decode(2, 1, 'Southlake'); +select decode(2, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle', 'Non domestic'); +select decode(6, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle', 'Non domestic'); +select decode(6, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle'); +select decode(null, 6, 'Spark', NULL, 'SQL', 4, 'rocks'); +select decode(null, 6, 'Spark', NULL, 'SQL', 4, 'rocks', NULL, '.'); +select decode(X'68656c6c6f', 'Windows-xxx'); +select decode(scol, ecol) from values(X'68656c6c6f', 'Windows-xxx') as t(scol, ecol); +set spark.sql.legacy.javaCharsets=true; +select decode(X'68656c6c6f', 'WINDOWS-1252'); +select decode(scol, ecol) from values(X'68656c6c6f', 'WINDOWS-1252') as t(scol, ecol); +set spark.sql.legacy.javaCharsets=false; +select decode(X'68656c6c6f', 'WINDOWS-1252'); +select decode(scol, ecol) from values(X'68656c6c6f', 'WINDOWS-1252') as t(scol, ecol); +set spark.sql.legacy.codingErrorAction=true; +select decode(X'E58A9DE5909BE69BB4E5B0BDE4B880E69DAFE98592', 'US-ASCII'); +select decode(scol, ecol) from values(X'E58A9DE5909BE69BB4E5B0BDE4B880E69DAFE98592', 'US-ASCII') as t(scol, ecol); +set spark.sql.legacy.codingErrorAction=false; +select decode(X'E8A5BFE587BAE998B3E585B3E697A0E69585E4BABA', 'US-ASCII'); +select decode(scol, ecol) from values(X'E8A5BFE587BAE998B3E585B3E697A0E69585E4BABA', 'US-ASCII') as t(scol, ecol); + +-- contains +SELECT CONTAINS(null, 'Spark'); +SELECT CONTAINS('Spark SQL', null); +SELECT CONTAINS(null, null); +SELECT CONTAINS('Spark SQL', 'Spark'); +SELECT CONTAINS('Spark SQL', 'SQL'); +SELECT CONTAINS('Spark SQL', 'SPARK'); + +SELECT startswith('Spark SQL', 'ark'); +SELECT startswith('Spark SQL', 'Spa'); +SELECT startswith(null, 'Spark'); +SELECT startswith('Spark', null); +SELECT startswith(null, null); + +SELECT endswith('Spark SQL', 'QL'); +SELECT endswith('Spark SQL', 'Spa'); +SELECT endswith(null, 'Spark'); +SELECT endswith('Spark', null); +SELECT endswith(null, null); + +SELECT contains(x'537061726b2053514c', x'537061726b'); +SELECT contains(x'', x''); +SELECT contains(x'537061726b2053514c', null); +SELECT contains(12, '1'); +SELECT contains(true, 'ru'); +SELECT contains(x'12', 12); +SELECT contains(true, false); + +SELECT startswith(x'537061726b2053514c', x'537061726b'); +SELECT startswith(x'537061726b2053514c', x''); +SELECT startswith(x'', x''); +SELECT startswith(x'537061726b2053514c', null); + +SELECT endswith(x'537061726b2053514c', x'53516c'); +SELECT endsWith(x'537061726b2053514c', x'537061726b'); +SELECT endsWith(x'537061726b2053514c', x''); +SELECT endsWith(x'', x''); +SELECT endsWith(x'537061726b2053514c', null); + +-- to_number +select to_number('454', '000'); +select to_number('454.2', '000.0'); +select to_number('12,454', '00,000'); +select to_number('$78.12', '$00.00'); +select to_number('+454', 'S000'); +select to_number('-454', 'S000'); +select to_number('12,454.8-', '00,000.9MI'); +select to_number('00,454.8-', '00,000.9MI'); +select to_number('<00,454.8>', '00,000.9PR'); + +-- to_binary +-- base64 valid +select to_binary('', 'base64'); +select to_binary(' ', 'base64'); +select to_binary(' ab cd ', 'base64'); +select to_binary(' ab c=', 'base64'); +select to_binary(' ab cdef= = ', 'base64'); +select to_binary( + concat(' b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0', + 'd2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl'), 'base64'); +-- base64 invalid +select to_binary('a', 'base64'); +select to_binary('a?', 'base64'); +select to_binary('abcde', 'base64'); +select to_binary('abcd=', 'base64'); +select to_binary('a===', 'base64'); +select to_binary('ab==f', 'base64'); +-- utf-8 +select to_binary( + '∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β)', 'utf-8'); +select to_binary('大千世界', 'utf8'); +select to_binary('', 'utf-8'); +select to_binary(' ', 'utf8'); +-- hex valid +select to_binary('737472696E67'); +select to_binary('737472696E67', 'hex'); +select to_binary(''); +select to_binary('1', 'hex'); +select to_binary('FF'); +select to_binary('123', 'hex'); +select to_binary('12345', 'hex'); +-- hex invalid +select to_binary('GG'); +select to_binary('01 AF', 'hex'); +-- 'format' parameter can be any foldable string value, not just literal. +select to_binary('abc', concat('utf', '-8')); +select to_binary(' ab cdef= = ', substr('base64whynot', 0, 6)); +select to_binary(' ab cdef= = ', replace('HEX0', '0')); +-- 'format' parameter is case insensitive. +select to_binary('abc', 'Hex'); +-- null inputs lead to null result. +select to_binary('abc', null); +select to_binary(null, 'utf-8'); +select to_binary(null, null); +select to_binary(null, cast(null as string)); +-- invalid format +select to_binary('abc', 1); +select to_binary('abc', 'invalidFormat'); +CREATE TEMPORARY VIEW fmtTable(fmtField) AS SELECT * FROM VALUES ('invalidFormat'); +SELECT to_binary('abc', fmtField) FROM fmtTable; +-- Clean up +DROP VIEW IF EXISTS fmtTable; +-- luhn_check +-- basic cases +select luhn_check('4111111111111111'); +select luhn_check('5500000000000004'); +select luhn_check('340000000000009'); +select luhn_check('6011000000000004'); +select luhn_check('6011000000000005'); +select luhn_check('378282246310006'); +select luhn_check('0'); +-- spaces in the beginning/middle/end +select luhn_check('4111111111111111 '); +select luhn_check('4111111 111111111'); +select luhn_check(' 4111111111111111'); +-- space +select luhn_check(''); +select luhn_check(' '); +-- non-digits +select luhn_check('510B105105105106'); +select luhn_check('ABCDED'); +-- null +select luhn_check(null); +-- non string (test implicit cast) +select luhn_check(6011111111111117); +select luhn_check(6011111111111118); +select luhn_check(123.456); + +--utf8 string validation +select is_valid_utf8(''); +select is_valid_utf8('abc'); +select is_valid_utf8(x'80'); +select make_valid_utf8(''); +select make_valid_utf8('abc'); +select make_valid_utf8(x'80'); +select validate_utf8(''); +select validate_utf8('abc'); +select validate_utf8(x'80'); +select try_validate_utf8(''); +select try_validate_utf8('abc'); +select try_validate_utf8(x'80'); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/struct.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/struct.sql new file mode 100644 index 000000000000..93a1238ab18c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/struct.sql @@ -0,0 +1,27 @@ +CREATE TEMPORARY VIEW tbl_x AS VALUES + (1, NAMED_STRUCT('C', 'gamma', 'D', 'delta')), + (2, NAMED_STRUCT('C', 'epsilon', 'D', 'eta')), + (3, NAMED_STRUCT('C', 'theta', 'D', 'iota')) + AS T(ID, ST); + +-- Create a struct +SELECT STRUCT('alpha', 'beta') ST; + +-- Create a struct with aliases +SELECT STRUCT('alpha' AS A, 'beta' AS B) ST; + +-- Star expansion in a struct. +SELECT ID, STRUCT(ST.*) NST FROM tbl_x; + +-- Append a column to a struct +SELECT ID, STRUCT(ST.*,CAST(ID AS STRING) AS E) NST FROM tbl_x; + +-- Prepend a column to a struct +SELECT ID, STRUCT(CAST(ID AS STRING) AS AA, ST.*) NST FROM tbl_x; + +-- Select a column from a struct +SELECT ID, STRUCT(ST.*).C NST FROM tbl_x; +SELECT ID, STRUCT(ST.C, ST.D).D NST FROM tbl_x; + +-- Select an alias from a struct +SELECT ID, STRUCT(ST.C as STC, ST.D as STD).STD FROM tbl_x; \ No newline at end of file diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subexp-elimination.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subexp-elimination.sql new file mode 100644 index 000000000000..9a594e0928dd --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subexp-elimination.sql @@ -0,0 +1,37 @@ +-- Test for subexpression elimination. + +--SET spark.sql.optimizer.enableJsonExpressionOptimization=false + +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false + +--CONFIG_DIM2 spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM2 spark.sql.codegen.factoryMode=NO_CODEGEN + +--CONFIG_DIM3 spark.sql.subexpressionElimination.enabled=true +--CONFIG_DIM3 spark.sql.subexpressionElimination.enabled=false + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +('{"a":1, "b":"2"}', '[{"a": 1, "b":2}, {"a":2, "b":2}]'), ('{"a":1, "b":"2"}', null), ('{"a":2, "b":"3"}', '[{"a": 3, "b":4}, {"a":4, "b":5}]'), ('{"a":5, "b":"6"}', '[{"a": 6, "b":7}, {"a":8, "b":9}]'), (null, '[{"a": 1, "b":2}, {"a":2, "b":2}]') +AS testData(a, b); + +SELECT from_json(a, 'struct').a, from_json(a, 'struct').b, from_json(b, 'array>')[0].a, from_json(b, 'array>')[0].b FROM testData; + +SELECT if(from_json(a, 'struct').a > 1, from_json(b, 'array>')[0].a, from_json(b, 'array>')[0].a + 1) FROM testData; + +SELECT if(isnull(from_json(a, 'struct').a), from_json(b, 'array>')[0].b + 1, from_json(b, 'array>')[0].b) FROM testData; + +SELECT case when from_json(a, 'struct').a > 5 then from_json(a, 'struct').b when from_json(a, 'struct').a > 4 then from_json(a, 'struct').b + 1 else from_json(a, 'struct').b + 2 end FROM testData; + +SELECT case when from_json(a, 'struct').a > 5 then from_json(b, 'array>')[0].b when from_json(a, 'struct').a > 4 then from_json(b, 'array>')[0].b + 1 else from_json(b, 'array>')[0].b + 2 end FROM testData; + +-- With non-deterministic expressions. +SELECT from_json(a, 'struct').a + random() > 2, from_json(a, 'struct').b, from_json(b, 'array>')[0].a, from_json(b, 'array>')[0].b + + random() > 2 FROM testData; + +SELECT if(from_json(a, 'struct').a + random() > 5, from_json(b, 'array>')[0].a, from_json(b, 'array>')[0].a + 1) FROM testData; + +SELECT case when from_json(a, 'struct').a > 5 then from_json(a, 'struct').b + random() > 5 when from_json(a, 'struct').a > 4 then from_json(a, 'struct').b + 1 + random() > 2 else from_json(a, 'struct').b + 2 + random() > 5 end FROM testData; + +-- Clean up +DROP VIEW IF EXISTS testData; \ No newline at end of file diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-aggregate.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-aggregate.sql new file mode 100644 index 000000000000..9dc4ed30fa00 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-aggregate.sql @@ -0,0 +1,174 @@ +-- Tests aggregate expressions in outer query and EXISTS subquery. + +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN +--ONLY_IF spark + +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id); + +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt); + +-- Aggregate in outer query block. +-- TC.01.01 +SELECT emp.dept_id, + avg(salary), + sum(salary) +FROM emp +WHERE EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id) +GROUP BY dept_id; + +-- Aggregate in inner/subquery block +-- TC.01.02 +SELECT emp_name +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id); + +-- Aggregate expression in both outer and inner query block. +-- TC.01.03 +SELECT count(*) +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id); + +-- Nested exists with aggregate expression in inner most query block. +-- TC.01.04 +SELECT * +FROM bonus +WHERE EXISTS (SELECT 1 + FROM emp + WHERE emp.emp_name = bonus.emp_name + AND EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE emp.dept_id = dept.dept_id + GROUP BY dept.dept_id)); + +-- Not exists with Aggregate expression in outer +-- TC.01.05 +SELECT emp.dept_id, + Avg(salary), + Sum(salary) +FROM emp +WHERE NOT EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id) +GROUP BY dept_id; + +-- Not exists with Aggregate expression in subquery block +-- TC.01.06 +SELECT emp_name +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id); + +-- Not exists with Aggregate expression in outer and subquery block +-- TC.01.07 +SELECT count(*) +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id); + +-- Nested not exists and exists with aggregate expression in inner most query block. +-- TC.01.08 +SELECT * +FROM bonus +WHERE NOT EXISTS (SELECT 1 + FROM emp + WHERE emp.emp_name = bonus.emp_name + AND EXISTS (SELECT Max(dept.dept_id) + FROM dept + WHERE emp.dept_id = dept.dept_id + GROUP BY dept.dept_id)); + +-- Window functions are not supported in EXISTS subqueries yet +SELECT * +FROM BONUS +WHERE EXISTS(SELECT RANK() OVER (PARTITION BY hiredate ORDER BY salary) AS s + FROM EMP, DEPT where EMP.dept_id = DEPT.dept_id + AND DEPT.dept_name < BONUS.emp_name); + +-- SPARK-46468: Aggregate always returns 1 row, so EXISTS is always true. +SELECT tt1.emp_name +FROM EMP as tt1 +WHERE EXISTS ( + select max(tt2.id) + from EMP as tt2 + where tt1.emp_name is null +); + +-- Plain exists subquery with a top-level aggregation +SELECT + emp.dept_id, + EXISTS (SELECT dept.dept_id FROM dept) +FROM emp +GROUP BY emp.dept_id ORDER BY emp.dept_id; + +-- Correlated exists subquery with a top-level aggregation +SELECT + emp.dept_id, + EXISTS (SELECT dept.dept_id FROM dept) +FROM emp +GROUP BY emp.dept_id ORDER BY emp.dept_id; + +-- Correlated exists subquery with a top-level aggregation +SELECT + emp.dept_id, + NOT EXISTS (SELECT dept.dept_id FROM dept) +FROM emp +GROUP BY emp.dept_id ORDER BY emp.dept_id; + +-- Correlated exists subquery with a top-level aggregation +SELECT + emp.dept_id, + SUM( + CASE WHEN EXISTS (SELECT dept.dept_id FROM dept WHERE dept.dept_id = emp.dept_id) THEN 1 + ELSE 0 END) +FROM emp +GROUP BY emp.dept_id ORDER BY emp.dept_id; + +---- Grouping expression contains a subquery +SELECT + cast(EXISTS (SELECT id FROM dept where dept.dept_id = emp.dept_id) AS int) +FROM emp +GROUP BY + cast(EXISTS (SELECT id FROM dept where dept.dept_id = emp.dept_id) AS int) diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-basic.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-basic.sql new file mode 100644 index 000000000000..4055f798ad85 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-basic.sql @@ -0,0 +1,124 @@ +-- Tests EXISTS subquery support. Tests basic form +-- of EXISTS subquery (both EXISTS and NOT EXISTS) +--ONLY_IF spark + +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id); + +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt); + +-- uncorrelated exist query +-- TC.01.01 +SELECT * +FROM emp +WHERE EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30); + +-- simple correlated predicate in exist subquery +-- TC.01.02 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id); + +-- correlated outer isnull predicate +-- TC.01.03 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id + OR emp.dept_id IS NULL); + +-- Simple correlation with a local predicate in outer query +-- TC.01.04 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id) + AND emp.id > 200; + +-- Outer references (emp.id) should not be pruned from outer plan +-- TC.01.05 +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id) + AND emp.id > 200; + +-- not exists with correlated predicate +-- TC.01.06 +SELECT * +FROM dept +WHERE NOT EXISTS (SELECT emp_name + FROM emp + WHERE emp.dept_id = dept.dept_id); + +-- not exists with correlated predicate + local predicate +-- TC.01.07 +SELECT * +FROM dept +WHERE NOT EXISTS (SELECT emp_name + FROM emp + WHERE emp.dept_id = dept.dept_id + OR state = 'NJ'); + +-- not exist both equal and greaterthan predicate +-- TC.01.08 +SELECT * +FROM bonus +WHERE NOT EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary); + +-- select employees who have not received any bonus +-- TC 01.09 +SELECT emp.* +FROM emp +WHERE NOT EXISTS (SELECT NULL + FROM bonus + WHERE bonus.emp_name = emp.emp_name); + +-- Nested exists +-- TC.01.10 +SELECT * +FROM bonus +WHERE EXISTS (SELECT emp_name + FROM emp + WHERE bonus.emp_name = emp.emp_name + AND EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id)); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-count-bug.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-count-bug.sql new file mode 100644 index 000000000000..2c2799ce432b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-count-bug.sql @@ -0,0 +1,43 @@ +--ONLY_IF spark +create temporary view t1(c1, c2) as values (0, 1), (1, 2); +create temporary view t2(c1, c2) as values (0, 2), (0, 3); +create temporary view t3(c1, c2) as values (0, 3), (1, 4), (2, 5); + +select * from t1 where exists (select count(*) from t2 where t2.c1 = t1.c1); + +select * from t1 where not exists (select count(*) from t2 where t2.c1 = t1.c1); + +select *, exists (select count(*) from t2 where t2.c1 = t1.c1) from t1; + +select *, not exists (select count(*) from t2 where t2.c1 = t1.c1) from t1; + +select * from t1 where + exists(select count(*) + 1 from t2 where t2.c1 = t1.c1) OR + not exists (select count(*) - 1 from t2 where t2.c1 = t1.c1); + + +select * from t1 where + (exists(select count(*) + 1 from t2 where t2.c1 = t1.c1) OR + not exists(select count(*) - 1 from t2 where t2.c1 = t1.c1)) AND + exists(select count(*) from t2 where t2.c1 = t1.c2); + +select * from t1 where exists (select count(*) from t2 where t1.c1 = 100); + + +-- With legacy behavior flag set, some answers are not correct. +set spark.sql.optimizer.decorrelateExistsSubqueryLegacyIncorrectCountHandling.enabled = true; +select * from t1 where exists (select count(*) from t2 where t2.c1 = t1.c1); + +select * from t1 where not exists (select count(*) from t2 where t2.c1 = t1.c1); + +select *, exists (select count(*) from t2 where t2.c1 = t1.c1) from t1; + +select *, not exists (select count(*) from t2 where t2.c1 = t1.c1) from t1; + +select * from t1 where + exists(select count(*) + 1 from t2 where t2.c1 = t1.c1) OR + not exists (select count(*) - 1 from t2 where t2.c1 = t1.c1); + +select * from t1 where exists (select count(*) from t2 where t1.c1 = 100); + +set spark.sql.optimizer.decorrelateExistsSubqueryLegacyIncorrectCountHandling.enabled = false; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-cte.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-cte.sql new file mode 100644 index 000000000000..ea8c0fc36ccf --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-cte.sql @@ -0,0 +1,143 @@ +-- Tests EXISTS subquery used along with +-- Common Table Expressions(CTE) +--ONLY_IF spark + +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id); + +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt); + +-- CTE used inside subquery with correlated condition +-- TC.01.01 +WITH bonus_cte + AS (SELECT * + FROM bonus + WHERE EXISTS (SELECT dept.dept_id, + emp.emp_name, + Max(salary), + Count(*) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY dept.dept_id, + emp.emp_name + ORDER BY emp.emp_name)) +SELECT * +FROM bonus a +WHERE a.bonus_amt > 30 + AND EXISTS (SELECT 1 + FROM bonus_cte b + WHERE a.emp_name = b.emp_name); + +-- Inner join between two CTEs with correlated condition +-- TC.01.02 +WITH emp_cte + AS (SELECT * + FROM emp + WHERE id >= 100 + AND id <= 300), + dept_cte + AS (SELECT * + FROM dept + WHERE dept_id = 10) +SELECT * +FROM bonus +WHERE EXISTS (SELECT * + FROM emp_cte a + JOIN dept_cte b + ON a.dept_id = b.dept_id + WHERE bonus.emp_name = a.emp_name); + +-- Left outer join between two CTEs with correlated condition +-- TC.01.03 +WITH emp_cte + AS (SELECT * + FROM emp + WHERE id >= 100 + AND id <= 300), + dept_cte + AS (SELECT * + FROM dept + WHERE dept_id = 10) +SELECT DISTINCT b.emp_name, + b.bonus_amt +FROM bonus b, + emp_cte e, + dept d +WHERE e.dept_id = d.dept_id + AND e.emp_name = b.emp_name + AND EXISTS (SELECT * + FROM emp_cte a + LEFT JOIN dept_cte b + ON a.dept_id = b.dept_id + WHERE e.emp_name = a.emp_name); + +-- Joins inside cte and aggregation on cte referenced subquery with correlated condition +-- TC.01.04 +WITH empdept + AS (SELECT id, + salary, + emp_name, + dept.dept_id + FROM emp + LEFT JOIN dept + ON emp.dept_id = dept.dept_id + WHERE emp.id IN ( 100, 200 )) +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE EXISTS (SELECT dept_id, + max(salary) + FROM empdept + GROUP BY dept_id + HAVING count(*) > 1) +GROUP BY emp_name; + +-- Using not exists +-- TC.01.05 +WITH empdept + AS (SELECT id, + salary, + emp_name, + dept.dept_id + FROM emp + LEFT JOIN dept + ON emp.dept_id = dept.dept_id + WHERE emp.id IN ( 100, 200 )) +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE NOT EXISTS (SELECT dept_id, + Max(salary) + FROM empdept + GROUP BY dept_id + HAVING count(*) < 1) +GROUP BY emp_name; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-having.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-having.sql new file mode 100644 index 000000000000..d172220fbc41 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-having.sql @@ -0,0 +1,91 @@ +-- Tests HAVING clause in subquery. + +CREATE TEMPORARY VIEW EMP(id, emp_name, hiredate, salary, dept_id) AS VALUES + (100, 'emp 1', date '2005-01-01', double(100.00), 10), + (100, 'emp 1', date '2005-01-01', double(100.00), 10), + (200, 'emp 2', date '2003-01-01', double(200.00), 10), + (300, 'emp 3', date '2002-01-01', double(300.00), 20), + (400, 'emp 4', date '2005-01-01', double(400.00), 30), + (500, 'emp 5', date '2001-01-01', double(400.00), NULL), + (600, 'emp 6 - no dept', date '2001-01-01', double(400.00), 100), + (700, 'emp 7', date '2010-01-01', double(400.00), 100), + (800, 'emp 8', date '2016-01-01', double(150.00), 70); + +CREATE TEMPORARY VIEW DEPT(dept_id, dept_name, state) AS VALUES + (10, 'dept 1', 'CA'), + (20, 'dept 2', 'NY'), + (30, 'dept 3', 'TX'), + (40, 'dept 4 - unassigned', 'OR'), + (50, 'dept 5 - unassigned', 'NJ'), + (70, 'dept 7', 'FL'); + +CREATE TEMPORARY VIEW BONUS(emp_name, bonus_amt) AS VALUES + ('emp 1', double(10.00)), + ('emp 1', double(20.00)), + ('emp 2', double(300.00)), + ('emp 2', double(100.00)), + ('emp 3', double(300.00)), + ('emp 4', double(100.00)), + ('emp 5', double(1000.00)), + ('emp 6 - no dept', double(500.00)); + +-- simple having in subquery. +-- TC.01.01 +SELECT dept_id, count(*) +FROM emp +GROUP BY dept_id +HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt < min(emp.salary)); + +-- nested having in subquery +-- TC.01.02 +SELECT * +FROM dept +WHERE EXISTS (SELECT dept_id, + Count(*) + FROM emp + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt < Min(emp.salary))); + +-- aggregation in outer and inner query block with having +-- TC.01.03 +SELECT dept_id, + Max(salary) +FROM emp gp +WHERE EXISTS (SELECT dept_id, + Count(*) + FROM emp p + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt < Min(p.salary))) +GROUP BY gp.dept_id; + +-- more aggregate expressions in projection list of subquery +-- TC.01.04 +SELECT * +FROM dept +WHERE EXISTS (SELECT dept_id, + Count(*) + FROM emp + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt > Min(emp.salary))); + +-- multiple aggregations in nested subquery +-- TC.01.05 +SELECT * +FROM dept +WHERE EXISTS (SELECT dept_id, + count(emp.dept_id) + FROM emp + WHERE dept.dept_id = dept_id + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE ( bonus_amt > min(emp.salary) + AND count(emp.dept_id) > 1 ))); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-in-join-condition.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-in-join-condition.sql new file mode 100644 index 000000000000..bc732cc3d320 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-in-join-condition.sql @@ -0,0 +1,95 @@ +-- Test that correlated EXISTS subqueries in join conditions are supported. + +-- Permutations of the test: +-- 1. Exists / Not Exists +-- 2. Reference left / right child +-- 3. Join type: inner / left outer / right outer / full outer / left semi / left anti +-- 4. AND or OR for the join condition + +--ONLY_IF spark +CREATE TEMP VIEW x(x1, x2) AS VALUES + (2, 1), + (1, 1), + (3, 4); + +CREATE TEMP VIEW y(y1, y2) AS VALUES + (0, 2), + (1, 4), + (4, 11); + +CREATE TEMP VIEW z(z1, z2) AS VALUES + (4, 2), + (3, 3), + (8, 1); + +-- Correlated EXISTS, REFERENCE LEFT, INNER JOIN +select * from x inner join y on x1 = y1 and exists (select * from z where z2 = x2) order by x1, x2, y1, y2; + +-- Correlated NOT EXISTS, REFERENCE LEFT, INNER JOIN +select * from x inner join y on x1 = y1 and not exists (select * from z where z2 = x2) order by x1, x2, y1, y2; + +-- Correlated EXISTS, REFERENCE RIGHT, INNER JOIN +select * from x inner join y on x1 = y1 and exists (select * from z where z2 = y2) order by x1, x2, y1, y2; + +-- Correlated NOT EXISTS, REFERENCE RIGHT, INNER JOIN +select * from x inner join y on x1 = y1 and not exists (select * from z where z2 = y2) order by x1, x2, y1, y2; + +-- Same as above, but for left outer join +select * from x left join y on x1 = y1 and exists (select * from z where z2 = x2) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 and not exists (select * from z where z2 = x2) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 and exists (select * from z where z2 = y2) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 and not exists (select * from z where z2 = y2) order by x1, x2, y1, y2; + +-- Same as above, but for right outer join +select * from x right join y on x1 = y1 and exists (select * from z where z2 = x2) order by x1, x2, y1, y2; +select * from x right join y on x1 = y1 and not exists (select * from z where z2 = x2) order by x1, x2, y1, y2; +select * from x right join y on x1 = y1 and exists (select * from z where z2 = y2) order by x1, x2, y1, y2; +select * from x right join y on x1 = y1 and not exists (select * from z where z2 = y2) order by x1, x2, y1, y2; + +-- Same as above, but for full outer join +select * from x right join y on x1 = y1 and exists (select * from z where z2 = x2) order by x1, x2, y1, y2; +select * from x right join y on x1 = y1 and not exists (select * from z where z2 = x2) order by x1, x2, y1, y2; +select * from x right join y on x1 = y1 and exists (select * from z where z2 = y2) order by x1, x2, y1, y2; +select * from x right join y on x1 = y1 and not exists (select * from z where z2 = y2) order by x1, x2, y1, y2; + +-- Same as above, but for left semi join +select * from x left semi join y on x1 = y1 and exists (select * from z where z2 = x2) order by x1, x2; +select * from x left semi join y on x1 = y1 and not exists (select * from z where z2 = x2) order by x1, x2; +select * from x left semi join y on x1 = y1 and exists (select * from z where z2 = y2) order by x1, x2; +select * from x left semi join y on x1 = y1 and not exists (select * from z where z2 = y2) order by x1, x2; + +-- Same as above, but for left anti join +select * from x left anti join y on x1 = y1 and exists (select * from z where z2 = x2) order by x1, x2; +select * from x left anti join y on x1 = y1 and not exists (select * from z where z2 = x2) order by x1, x2; +select * from x left anti join y on x1 = y1 and exists (select * from z where z2 = y2) order by x1, x2; +select * from x left anti join y on x1 = y1 and not exists (select * from z where z2 = y2) order by x1, x2; + +-- Same as above, but for full outer join +select * from x full outer join y on x1 = y1 and exists (select * from z where z2 = x2) order by x1, x2, y1, y2; +select * from x full outer join y on x1 = y1 and not exists (select * from z where z2 = x2) order by x1, x2, y1, y2; +select * from x full outer join y on x1 = y1 and exists (select * from z where z2 = y2) order by x1, x2, y1, y2; +select * from x full outer join y on x1 = y1 and not exists (select * from z where z2 = y2) order by x1, x2, y1, y2; + +-- OR instead of AND in the join condition +select * from x inner join y on x1 = y1 or exists (select * from z where z1 = x1) order by x1, x2, y1, y2; +select * from x inner join y on x1 = y1 or not exists (select * from z where z1 = x1) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 or exists (select * from z where z1 = x1) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 or not exists (select * from z where z1 = x1) order by x1, x2, y1, y2; +select * from x inner join y on x1 = y1 or exists (select * from z where z1 = y1) order by x1, x2, y1, y2; +select * from x inner join y on x1 = y1 or not exists (select * from z where z1 = y1) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 or exists (select * from z where z1 = y1) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 or not exists (select * from z where z1 = y1) order by x1, x2, y1, y2; + +-- Transitive predicates to test if inferring filters can cause issues. +select * from x inner join y on x1 = y1 and exists (select * from z where z1 = x1) order by x1, x2, y1, y2; +select * from x inner join y on x1 = y1 and not exists (select * from z where z1 = x1) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 and exists (select * from z where z1 = x1) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 and not exists (select * from z where z1 = x1) order by x1, x2, y1, y2; +select * from x inner join y on x1 = y1 and exists (select * from z where z1 = y1) order by x1, x2, y1, y2; +select * from x inner join y on x1 = y1 and not exists (select * from z where z1 = y1) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 and exists (select * from z where z1 = y1) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 and not exists (select * from z where z1 = y1) order by x1, x2, y1, y2; + +-- Correlated subquery references both left and right children, errors +select * from x join y on x1 = y1 and exists (select * from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2; +select * from x join y on x1 = y1 and not exists (select * from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-joins-and-set-ops.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-joins-and-set-ops.sql new file mode 100644 index 000000000000..1542fa5149aa --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-joins-and-set-ops.sql @@ -0,0 +1,321 @@ +-- Tests EXISTS subquery support. Tests Exists subquery +-- used in Joins (Both when joins occurs in outer and suquery blocks) + +-- There are 2 dimensions we want to test +-- 1. run with broadcast hash join, sort merge join or shuffle hash join. +-- 2. run with whole-stage-codegen, operator codegen or no codegen. + +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=10485760 +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.forceApplyShuffledHashJoin=true + +--CONFIG_DIM2 spark.sql.codegen.wholeStage=true +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +--ONLY_IF spark +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id); + +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt); + +-- Join in outer query block +-- TC.01.01 +SELECT * +FROM emp, + dept +WHERE emp.dept_id = dept.dept_id + AND EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name); + +-- Join in outer query block with ON condition +-- TC.01.02 +SELECT * +FROM emp + JOIN dept + ON emp.dept_id = dept.dept_id +WHERE EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name); + +-- Left join in outer query block with ON condition +-- TC.01.03 +SELECT * +FROM emp + LEFT JOIN dept + ON emp.dept_id = dept.dept_id +WHERE EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name); + +-- Join in outer query block + NOT EXISTS +-- TC.01.04 +SELECT * +FROM emp, + dept +WHERE emp.dept_id = dept.dept_id + AND NOT EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name); + + +-- inner join in subquery. +-- TC.01.05 +SELECT * +FROM bonus +WHERE EXISTS (SELECT * + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name); + +-- right join in subquery +-- TC.01.06 +SELECT * +FROM bonus +WHERE EXISTS (SELECT * + FROM emp + RIGHT JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name); + + +-- Aggregation and join in subquery +-- TC.01.07 +SELECT * +FROM bonus +WHERE EXISTS (SELECT dept.dept_id, + emp.emp_name, + Max(salary), + Count(*) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY dept.dept_id, + emp.emp_name + ORDER BY emp.emp_name); + +-- Aggregations in outer and subquery + join in subquery +-- TC.01.08 +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE EXISTS (SELECT emp_name, + Max(salary) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY emp_name + HAVING Count(*) > 1 + ORDER BY emp_name) +GROUP BY emp_name; + +-- TC.01.09 +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE NOT EXISTS (SELECT emp_name, + Max(salary) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY emp_name + HAVING Count(*) > 1 + ORDER BY emp_name) +GROUP BY emp_name; + +-- Set operations along with EXISTS subquery +-- union +-- TC.02.01 +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + UNION + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50); + +-- intersect +-- TC.02.02 +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50); + +-- intersect + not exists +-- TC.02.03 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50); + +-- Union all in outer query and except,intersect in subqueries. +-- TC.02.04 +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + EXCEPT + SELECT * + FROM dept + WHERE dept_id > 50) +UNION ALL +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50); + +-- Union in outer query and except,intersect in subqueries. +-- TC.02.05 +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + EXCEPT + SELECT * + FROM dept + WHERE dept_id > 50) +UNION +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50); + +-- Correlated predicates under set ops - unsupported +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + UNION + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX"); + +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + UNION + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX"); + +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + INTERSECT ALL + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX"); + +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + INTERSECT DISTINCT + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX"); + +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + EXCEPT ALL + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX"); + +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + EXCEPT DISTINCT + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX"); + +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + INTERSECT ALL + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX"); + +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + EXCEPT DISTINCT + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX"); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql new file mode 100644 index 000000000000..9ff3409b21a3 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql @@ -0,0 +1,289 @@ +-- Tests EXISTS subquery support with ORDER BY and LIMIT clauses. + +-- Test sort operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +--ONLY_IF spark +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id); + +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt); + +-- order by in both outer and/or inner query block +-- TC.01.01 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state) +ORDER BY hiredate; + +-- TC.01.02 +SELECT id, + hiredate +FROM emp +WHERE EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state) +ORDER BY hiredate DESC; + +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state + LIMIT 1) +ORDER BY hiredate; + +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state + LIMIT 0) +ORDER BY hiredate; + +-- order by with not exists +-- TC.01.03 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state) +ORDER BY hiredate; + +-- group by + order by with not exists +-- TC.01.04 +SELECT emp_name +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY state + ORDER BY state); +-- TC.01.05 +SELECT count(*) +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept_id + ORDER BY dept_id); + +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state + LIMIT 1) +ORDER BY hiredate; + +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state + LIMIT 0) +ORDER BY hiredate; + +-- limit in the exists subquery block. +-- TC.02.01 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 10 + LIMIT 1); + +-- limit in the exists subquery block with aggregate. +-- TC.02.02 +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + GROUP BY state + LIMIT 1); + +-- limit in the not exists subquery block. +-- TC.02.03 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 100 + LIMIT 1); + +-- limit in the not exists subquery block with aggregates. +-- TC.02.04 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE dept.dept_id > 100 + GROUP BY state + LIMIT 1); + +SELECT emp_name +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY state + ORDER BY state + LIMIT 2 + OFFSET 1); + +-- limit and offset in the exists subquery block. +-- TC.03.01 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 10 + LIMIT 1 + OFFSET 2); + +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > emp.dept_id + LIMIT 1); + +-- limit and offset in the exists subquery block with aggregate. +-- TC.03.02 +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + GROUP BY state + LIMIT 1 + OFFSET 2); + +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE dept.dept_id <> emp.dept_id + GROUP BY state + LIMIT 1); + +-- SPARK-46526: LIMIT over correlated predicate that references only the outer table. +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE emp.salary > 200 + LIMIT 1); + +-- SPARK-46526: LIMIT over correlated predicate that references only the outer table, +-- and a group by. +SELECT * +FROM emp +WHERE EXISTS (SELECT state, max(dept.dept_name) + FROM dept + WHERE emp.salary > 200 + GROUP BY state + LIMIT 1); + +-- limit and offset in the not exists subquery block. +-- TC.03.03 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 100 + LIMIT 1 + OFFSET 2); + +-- limit and offset in the not exists subquery block with aggregates. +-- TC.03.04 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE dept.dept_id > 100 + GROUP BY state + LIMIT 1 + OFFSET 2); + +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id <> emp.dept_id + LIMIT 1 + OFFSET 2); + +-- offset in the exists subquery block. +-- TC.04.01 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 10 + OFFSET 2); + +-- offset in the exists subquery block with aggregate. +-- TC.04.02 +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + GROUP BY state + OFFSET 2); + +-- limit in the not exists subquery block. +-- TC.04.03 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 100 + OFFSET 2); + +-- limit in the not exists subquery block with aggregates. +-- TC.04.04 +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE dept.dept_id > 100 + GROUP BY state + OFFSET 2); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-outside-filter.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-outside-filter.sql new file mode 100644 index 000000000000..585de247b11c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-outside-filter.sql @@ -0,0 +1,157 @@ +-- Tests EXISTS subquery support where the subquery is used outside the WHERE clause. + +--ONLY_IF spark + +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id); + +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt); + +-- uncorrelated select exist +-- TC.01.01 +SELECT + emp_name, + EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30) +FROM emp; + +-- correlated select exist +-- TC.01.02 +SELECT + emp_name, + EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id) +FROM emp; + +-- uncorrelated exist in aggregate filter +-- TC.01.03 +SELECT + sum(salary), + sum(salary) FILTER (WHERE EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30)) +FROM emp; + +-- correlated exist in aggregate filter +-- TC.01.04 +SELECT + sum(salary), + sum(salary) FILTER (WHERE EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id)) +FROM emp; + +-- Multiple correlated exist in aggregate filter +-- TC.01.05 +SELECT + sum(salary), + sum(salary) FILTER (WHERE EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id) + OR EXISTS (SELECT 1 + FROM bonus + WHERE emp.emp_name = bonus.emp_name)) +FROM emp; + +-- correlated exist in DISTINCT aggregate filter +-- TC.01.06 +SELECT + sum(DISTINCT salary), + count(DISTINCT hiredate) FILTER (WHERE EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id)) +FROM emp; + +-- correlated exist in group by of an aggregate +-- TC.01.07 +SELECT + count(hiredate), + sum(salary) +FROM emp +GROUP BY EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id); + +-- correlated exist in group by of a distinct aggregate +-- TC.01.08 +SELECT + count(DISTINCT hiredate), + sum(DISTINCT salary) +FROM emp +GROUP BY EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id); + +-- uncorrelated exist in aggregate function +-- TC.01.09 +SELECT + count(CASE WHEN EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30) THEN 1 END), + sum(CASE WHEN EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30) THEN salary END) +FROM emp; + +-- correlated exist in aggregate function +-- TC.01.10 +SELECT + count(CASE WHEN EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id) THEN 1 END), + sum(CASE WHEN EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id) THEN salary END) +FROM emp; + +-- uncorrelated exist in window +-- TC.01.11 +SELECT + emp_name, + sum(salary) OVER (PARTITION BY EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30)) +FROM emp; + +-- correlated exist in window +-- TC.01.12 +SELECT + emp_name, + sum(salary) OVER (PARTITION BY EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id)) +FROM emp; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-within-and-or.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-within-and-or.sql new file mode 100644 index 000000000000..5920b61dade5 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/exists-subquery/exists-within-and-or.sql @@ -0,0 +1,97 @@ +-- Tests EXISTS subquery support. Tests EXISTS +-- subquery within a AND or OR expression. + +--ONLY_IF spark +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id); + +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt); + + +-- Or used in conjunction with exists - ExistenceJoin +-- TC.02.01 +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id) + OR emp.id > 200; + +-- all records from emp including the null dept_id +-- TC.02.02 +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id) + OR emp.dept_id IS NULL; + +-- EXISTS subquery in both LHS and RHS of OR. +-- TC.02.03 +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id + AND dept.dept_id = 20) + OR EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id + AND dept.dept_id = 30); +; + +-- not exists and exists predicate within OR +-- TC.02.04 +SELECT * +FROM bonus +WHERE ( NOT EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary) + OR EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + OR bonus_amt < emp.salary) ); + +-- not exists and in predicate within AND +-- TC.02.05 +SELECT * FROM bonus WHERE NOT EXISTS +( + SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary) +AND +emp_name IN +( + SELECT emp_name + FROM emp + WHERE bonus_amt < emp.salary); + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-basic.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-basic.sql new file mode 100644 index 000000000000..fc243422bd7c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-basic.sql @@ -0,0 +1,18 @@ +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false +--ONLY_IF spark + +create temporary view tab_a as select * from values (1, 1) as tab_a(a1, b1); +create temporary view tab_b as select * from values (1, 1) as tab_b(a2, b2); +create temporary view struct_tab as select struct(col1 as a, col2 as b) as record from + values (1, 1), (1, 2), (2, 1), (2, 2); + +select 1 from tab_a where (a1, b1) not in (select a2, b2 from tab_b); +-- Invalid query, see SPARK-24341 +select 1 from tab_a where (a1, b1) not in (select (a2, b2) from tab_b); + +-- Aliasing is needed as a workaround for SPARK-24443 +select count(*) from struct_tab where record in + (select (a2 as a, b2 as b) from tab_b); +select count(*) from struct_tab where record not in + (select (a2 as a, b2 as b) from tab_b); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-count-bug.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-count-bug.sql new file mode 100644 index 000000000000..f07ead3e868b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-count-bug.sql @@ -0,0 +1,51 @@ +--ONLY_IF spark +create temporary view t1(c1, c2) as values (0, 1), (1, 2); +create temporary view t2(c1, c2) as values (0, 2), (0, 3); +create temporary view t3(c1, c2) as values (0, 3), (1, 4), (2, 5); + +select * from t1 where c1 in (select count(*) + 1 from t2 where t2.c1 = t1.c1); + +select *, c1 in (select count(*) + 1 from t2 where t2.c1 = t1.c1) +from t1; + +select *, c1 not in (select count(*) + 1 from t2 where t2.c1 = t1.c1) +from t1; + +select * from t1 where + c1 in (select count(*) + 1 from t2 where t2.c1 = t1.c1) OR + c2 in (select count(*) - 1 from t2 where t2.c1 = t1.c1); + +select * from t1 where + (c1 in (select count(*) + 1 from t2 where t2.c1 = t1.c1) OR + c2 in (select count(*) - 1 from t2 where t2.c1 = t1.c1)) AND + c1 NOT in (select count(*) from t2 where t2.c1 = t1.c2); + +select * from t1 where c1 in (select 1 from t2 where t2.c1 = t1.c1 having count(*) = 0); + +select * from t1 where c1 not in (select 1 from t2 where t2.c1 = t1.c1 having count(*) = 0); + + +select * from t1 where c1 in (select count(*) from t1 join t3 using (c1) where t3.c1 = t1.c2); + +select * from t1 where c1 not in (select count(*) + 1 from t1 join t3 using (c1) where t3.c1 = t1.c2); + + +-- With legacy behavior flag set, some answers are not correct. +set spark.sql.optimizer.decorrelateExistsSubqueryLegacyIncorrectCountHandling.enabled = true; +select * from t1 where c1 in (select count(*) + 1 from t2 where t2.c1 = t1.c1); + +select *, c1 in (select count(*) + 1 from t2 where t2.c1 = t1.c1) +from t1; + +select *, c1 not in (select count(*) + 1 from t2 where t2.c1 = t1.c1) +from t1; + +select * from t1 where c1 in (select 1 from t2 where t2.c1 = t1.c1 having count(*) = 0); + +select * from t1 where c1 not in (select 1 from t2 where t2.c1 = t1.c1 having count(*) = 0); + + +select * from t1 where c1 in (select count(*) from t1 join t3 using (c1) where t3.c1 = t1.c2); + +select * from t1 where c1 not in (select count(*) + 1 from t1 join t3 using (c1) where t3.c1 = t1.c2); +set spark.sql.optimizer.decorrelateExistsSubqueryLegacyIncorrectCountHandling.enabled = false; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-group-by.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-group-by.sql new file mode 100644 index 000000000000..3161ac148e22 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-group-by.sql @@ -0,0 +1,319 @@ +-- A test suite for GROUP BY in parent side, subquery, and both predicate subquery +-- It includes correlated cases. + +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +--ONLY_IF spark +create temporary view t1 as select * from values + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("t1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("t1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("t2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("t1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("t2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("t1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("t1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("t3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("t3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("t3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("t3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("t1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("t3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- GROUP BY in parent side +-- TC 01.01 +SELECT t1a, + Avg(t1b) +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2) +GROUP BY t1a; + +-- TC 01.02 +SELECT t1a, + Max(t1b) +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a) +GROUP BY t1a, + t1d; + +-- TC 01.03 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) +GROUP BY t1a, + t1b; + +-- TC 01.04 +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) + OR t1c IN (SELECT t3c + FROM t3 + WHERE t1a = t3a) +GROUP BY t1a, + t1c; + +-- TC 01.05 +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) + AND t1c IN (SELECT t3c + FROM t3 + WHERE t1a = t3a) +GROUP BY t1a, + t1c; + +-- TC 01.06 +SELECT t1a, + Count(DISTINCT( t1b )) +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) +GROUP BY t1a, + t1c +HAVING t1a = "t1b"; + +-- GROUP BY in subquery +-- TC 01.07 +SELECT * +FROM t1 +WHERE t1b IN (SELECT Max(t2b) + FROM t2 + GROUP BY t2a); + +-- TC 01.08 +SELECT * +FROM (SELECT t2a, + t2b + FROM t2 + WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b) + GROUP BY t2a, + t2b) t2; + +-- TC 01.09 +SELECT Count(DISTINCT * ) +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + AND t1c = t2c + GROUP BY t2a); + +-- TC 01.10 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT Max(t2c) + FROM t2 + WHERE t1a = t2a + GROUP BY t2a, + t2c + HAVING t2c > 8); + +-- TC 01.11 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2a IN (SELECT Min(t3a) + FROM t3 + WHERE t3a = t2a + GROUP BY t3b) + GROUP BY t2c); + +-- GROUP BY in both +-- TC 01.12 +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a) +GROUP BY t1a; + +-- TC 01.13 +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b IN (SELECT Min(t3b) + FROM t3 + WHERE t2a = t3a + GROUP BY t3a) + GROUP BY t2c) +GROUP BY t1a, + t1d; + +-- TC 01.14 +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a) + AND t1d IN (SELECT t3d + FROM t3 + WHERE t1c = t3c + GROUP BY t3d) +GROUP BY t1a; + +-- TC 01.15 +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a) + OR t1d IN (SELECT t3d + FROM t3 + WHERE t1c = t3c + GROUP BY t3d) +GROUP BY t1a; + +-- TC 01.16 +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a + HAVING t2a > t1a) + OR t1d IN (SELECT t3d + FROM t3 + WHERE t1c = t3c + GROUP BY t3d + HAVING t3d = t1d) +GROUP BY t1a +HAVING Min(t1b) IS NOT NULL; + +-- Window functions are not supported in IN subqueries yet +select t1a +from t1 +where t1f IN (SELECT RANK() OVER (partition by t3c order by t2b) as s + FROM t2, t3 where t2.t2c = t3.t3c and t2.t2a < t1.t1a); + +-- Plain in-subquery with a top-level aggregation +SELECT + t1.t1a, + t1.t1a IN (SELECT t2a FROM t2) as v1 +FROM t1 +GROUP BY t1.t1a ORDER BY t1.t1a; + +-- Aggregate function over expression with subquery, without explicit GROUP BY, with NOT IN +SELECT + count(cast(t1.t1a IN (SELECT t2a FROM t2) as INT)), + sum(cast(t1.t1b NOT IN (SELECT t2b FROM t2) as INT)) +FROM t1; + +-- Derived table from subquery +SELECT + agg_results.t1a, + COUNT(*) + FROM (SELECT t1.t1a FROM t1 WHERE t1.t1a IN (SELECT t2a FROM t2)) AS agg_results +GROUP BY agg_results.t1a ORDER BY agg_results.t1a; + +-- CASE statement with an in-subquery and aggregation +SELECT + t1.t1a, + CASE + WHEN t1.t1a IN (SELECT t2a FROM t2) THEN 10 + ELSE -10 + END AS v1 +FROM t1 +GROUP BY t1.t1a +ORDER BY t1.t1a; + +-- CASE statement with an in-subquery inside an agg function +SELECT + t1.t1c, + -- sums over t1.t1c + SUM(CASE + WHEN t1.t1c IN (SELECT t2c FROM t2) THEN 10 + ELSE -10 + END) AS v1, + -- sums over t1.t1d + SUM(CASE + WHEN t1.t1d IN (SELECT t2c FROM t2) THEN 10 + ELSE -10 + END) AS v2, + -- no agg function, uses t1.t1c + t1.t1c + 10 IN (SELECT t2c + 2 FROM t2) AS v3, + count(t1.t1c) as ct, + count(t1.t1d) +FROM t1 +GROUP BY t1.t1c +ORDER BY t1.t1c; + +-- CASE statement with an in-subquery inside an agg function, without group-by +SELECT + SUM(CASE + WHEN t1.t1c IN (SELECT t2c FROM t2) THEN 10 + ELSE -10 + END) AS v1, + count(t1.t1c) as ct +FROM t1; + +-- Group-by statement contains an in-subquery, and there's an additional exists in select clause. +SELECT + cast(t1a in (select t2a from t2) as int) + 1 as groupExpr, + sum(cast(t1a in (select t2a from t2) as int) + 1) as aggExpr, + cast(t1a in (select t2a from t2) as int) + 1 + cast(exists (select t2a from t2) as int) + as complexExpr +FROM t1 +GROUP BY + cast(t1a in (select t2a from t2) as int) + 1; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-having.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-having.sql new file mode 100644 index 000000000000..4d11ea7005b3 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-having.sql @@ -0,0 +1,156 @@ +-- A test suite for IN HAVING in parent side, subquery, and both predicate subquery +-- It includes correlated cases. + +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + +--ONLY_IF spark +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- HAVING in the subquery +-- TC 01.01 +SELECT t1a, + t1b, + t1h +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + GROUP BY t2b + HAVING t2b < 10); + +-- TC 01.02 +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2b + HAVING t2b > 1); + +-- HAVING in the parent +-- TC 01.03 +SELECT t1a, t1b, t1c +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1c < t2c) +GROUP BY t1a, t1b, t1c +HAVING t1b < 10; + +-- TC 01.04 +SELECT t1a, t1b, t1c +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1c = t2c) +GROUP BY t1a, t1b, t1c +HAVING COUNT (DISTINCT t1b) < 10; + +-- BOTH +-- TC 01.05 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a + GROUP BY t2c + HAVING t2c > 10) +GROUP BY t1b +HAVING t1b >= 8; + +-- TC 01.06 +SELECT t1a, + Max(t1b) +FROM t1 +WHERE t1b > 0 +GROUP BY t1a +HAVING t1a IN (SELECT t2a + FROM t2 + WHERE t2b IN (SELECT t3b + FROM t3 + WHERE t2c = t3c) + ); + +-- HAVING clause with NOT IN +-- TC 01.07 +SELECT t1a, + t1c, + Min(t1d) +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + GROUP BY t2a + HAVING t2a > 'val2a') +GROUP BY t1a, t1c +HAVING Min(t1d) > t1c; + +-- TC 01.08 +SELECT t1a, + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + WHERE t1a = t2a + GROUP BY t2c, t2d + HAVING t2c > 8) +GROUP BY t1a, t1b +HAVING t1b < 10; + +-- TC 01.09 +SELECT t1a, + Max(t1b) +FROM t1 +WHERE t1b > 0 +GROUP BY t1a +HAVING t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b > 3); + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-joins.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-joins.sql new file mode 100644 index 000000000000..d12144ae7e49 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-joins.sql @@ -0,0 +1,396 @@ +-- A test suite for IN JOINS in parent side, subquery, and both predicate subquery +-- It includes correlated cases. + +-- There are 2 dimensions we want to test +-- 1. run with broadcast hash join, sort merge join or shuffle hash join. +-- 2. run with whole-stage-codegen, operator codegen or no codegen. + +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=10485760 +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--CONFIG_DIM1 spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.forceApplyShuffledHashJoin=true + +--CONFIG_DIM2 spark.sql.codegen.wholeStage=true +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM2 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +--CONFIG_DIM3 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM3 spark.sql.optimizeNullAwareAntiJoin=false + +--ONLY_IF spark +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +create temporary view s1 as select * from values + (1), (3), (5), (7), (9) + as s1(id); + +create temporary view s2 as select * from values + (1), (3), (4), (6), (9) + as s2(id); + +create temporary view s3 as select * from values + (3), (4), (6), (9) + as s3(id); + +-- correlated IN subquery +-- different JOIN in parent side +-- TC 01.01 +SELECT t1a, t1b, t1c, t3a, t3b, t3c +FROM t1 natural JOIN t3 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1a = t2a) + AND t1b = t3b + AND t1a = t3a +ORDER BY t1a, + t1b, + t1c DESC nulls first; + +-- TC 01.02 +SELECT Count(DISTINCT(t1a)), + t1b, + t3a, + t3b, + t3c +FROM t1 natural left JOIN t3 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1d = t2d) +AND t1b > t3b +GROUP BY t1a, + t1b, + t3a, + t3b, + t3c +ORDER BY t1a DESC, t3b DESC, t3c ASC; + +-- TC 01.03 +SELECT Count(DISTINCT(t1a)) +FROM t1 natural right JOIN t3 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1b = t2b) +AND t1d IN + ( + SELECT t2d + FROM t2 + WHERE t1c > t2c) +AND t1a = t3a +GROUP BY t1a +ORDER BY t1a; + +-- TC 01.04 +SELECT t1a, + t1b, + t1c, + t3a, + t3b, + t3c +FROM t1 FULL OUTER JOIN t3 +where t1a IN + ( + SELECT t2a + FROM t2 + WHERE t2c IS NOT NULL) +AND t1b != t3b +AND t1a = 'val1b' +ORDER BY t1a; + +-- TC 01.05 +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 RIGHT JOIN t3 +where t1a IN + ( + SELECT t2a + FROM t2 + WHERE t2h > t3h) +AND t3a IN + ( + SELECT t2a + FROM t2 + WHERE t2c > t3c) +AND t1h >= t3h +GROUP BY t1a, + t1b +HAVING t1b > 8 +ORDER BY t1a; + +-- TC 01.06 +SELECT Count(DISTINCT(t1a)) +FROM t1 LEFT OUTER +JOIN t3 +ON t1a = t3a +WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1h < t2h ) +GROUP BY t1a +ORDER BY t1a; + +-- TC 01.07 +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 INNER JOIN t2 +ON t1a > t2a +WHERE t1b IN + ( + SELECT t2b + FROM t2 + WHERE t2h > t1h) +OR t1a IN + ( + SELECT t2a + FROM t2 + WHERE t2h < t1h) +GROUP BY t1b +HAVING t1b > 6; + +-- different JOIN in the subquery +-- TC 01.08 +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + JOIN t1 + WHERE t2b <> t1b) +AND t1h IN + ( + SELECT t2h + FROM t2 + RIGHT JOIN t3 + where t2b = t3b) +GROUP BY t1b +HAVING t1b > 8; + +-- TC 01.09 +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + JOIN t1 + WHERE t2b <> t1b) +AND t1h IN + ( + SELECT t2h + FROM t2 + RIGHT JOIN t3 + where t2b = t3b) +AND t1b IN + ( + SELECT t2b + FROM t2 + FULL OUTER JOIN t3 + where t2b = t3b) + +GROUP BY t1b +HAVING t1b > 8; + +-- JOIN in the parent and subquery +-- TC 01.10 +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 +INNER JOIN t2 on t1b = t2b +RIGHT JOIN t3 ON t1a = t3a +where t1a IN + ( + SELECT t2a + FROM t2 + FULL OUTER JOIN t3 + WHERE t2b > t3b) +AND t1c IN + ( + SELECT t3c + FROM t3 + LEFT OUTER JOIN t2 + ON t3a = t2a ) +AND t1b IN + ( + SELECT t3b + FROM t3 LEFT OUTER + JOIN t1 + WHERE t3c = t1c) + +AND t1a = t2a +GROUP BY t1b +ORDER BY t1b DESC; + +-- TC 01.11 +SELECT t1a, + t1b, + t1c, + count(distinct(t2a)), + t2b, + t2c +FROM t1 +FULL JOIN t2 on t1a = t2a +RIGHT JOIN t3 on t1a = t3a +where t1a IN + ( + SELECT t2a + FROM t2 INNER + JOIN t3 + ON t2b < t3b + WHERE t2c IN + ( + SELECT t1c + FROM t1 + WHERE t1a = t2a)) +and t1a = t2a +Group By t1a, t1b, t1c, t2a, t2b, t2c +HAVING t2c IS NOT NULL +ORDER By t2b DESC nulls last; + + +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id IN (SELECT 9); + + +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id NOT IN (SELECT 9); + + +-- IN with Subquery ON INNER JOIN +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3); + + +-- IN with Subquery ON LEFT SEMI JOIN +SELECT s1.id AS id2 FROM s1 +LEFT SEMI JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3); + + +-- IN with Subquery ON LEFT ANTI JOIN +SELECT s1.id as id2 FROM s1 +LEFT ANTI JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3); + + +-- IN with Subquery ON LEFT OUTER JOIN +SELECT s1.id, s2.id as id2 FROM s1 +LEFT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3); + + +-- IN with Subquery ON RIGHT OUTER JOIN +SELECT s1.id, s2.id as id2 FROM s1 +RIGHT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3); + + +-- IN with Subquery ON FULL OUTER JOIN +SELECT s1.id, s2.id AS id2 FROM s1 +FULL OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3); + + +-- NOT IN with Subquery ON INNER JOIN +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3); + + +-- NOT IN with Subquery ON LEFT SEMI JOIN +SELECT s1.id AS id2 FROM s1 +LEFT SEMI JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3); + + +-- NOT IN with Subquery ON LEFT ANTI JOIN +SELECT s1.id AS id2 FROM s1 +LEFT ANTI JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3); + + +-- NOT IN with Subquery ON LEFT OUTER JOIN +SELECT s1.id, s2.id AS id2 FROM s1 +LEFT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3); + + +-- NOT IN with Subquery ON RIGHT OUTER JOIN +SELECT s1.id, s2.id AS id2 FROM s1 +RIGHT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3); + + +-- NOT IN with Subquery ON FULL OUTER JOIN +SELECT s1.id, s2.id AS id2 FROM s1 +FULL OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3); + + +DROP VIEW s1; + +DROP VIEW s2; + +DROP VIEW s3; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-limit.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-limit.sql new file mode 100644 index 000000000000..7c816d8a4167 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-limit.sql @@ -0,0 +1,384 @@ +-- A test suite for IN LIMIT in parent side, subquery, and both predicate subquery +-- It includes correlated cases. + +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + +--ONLY_IF spark +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- LIMIT in parent side +-- TC 01.01 +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +LIMIT 2; + +-- correlated IN subquery +-- LIMIT on both parent and subquery sides +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d + LIMIT 10) +LIMIT 2; + +-- correlated IN subquery +-- LIMIT with OFFSET in parent side +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +LIMIT 2 +OFFSET 1; + +-- correlated IN subquery +-- LIMIT with OFFSET on both parent and subquery sides +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d + LIMIT 10 + OFFSET 2) +LIMIT 2 +OFFSET 1; + +-- correlated IN subquery +-- OFFSET in parent side +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +OFFSET 1; + +-- correlated IN subquery +-- OFFSET on both parent and subquery sides +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d + OFFSET 2) +OFFSET 1; + +-- TC 01.02 +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b >= 8 + LIMIT 2) +LIMIT 4; + +-- TC 01.03 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + ORDER BY t2c, t2d + LIMIT 2) +GROUP BY t1b +ORDER BY t1b DESC NULLS FIRST +LIMIT 1; + +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b <= t1d + LIMIT 2) +LIMIT 4; + +-- LIMIT with NOT IN +-- TC 01.04 +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b > 6 + LIMIT 2); + +-- TC 01.05 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + ORDER BY t2b DESC nulls first, t2d + LIMIT 1) +GROUP BY t1b +ORDER BY t1b NULLS last +LIMIT 1; + +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + WHERE t2b <= t1d + ORDER BY t2c, t2d + LIMIT 2) +GROUP BY t1b +ORDER BY t1b DESC NULLS FIRST +LIMIT 1; + +-- LIMIT and OFFSET in parent side +-- TC 02.01 +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +LIMIT 2 +OFFSET 2; + +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d > t2d + ORDER BY t2a DESC + LIMIT 3) +LIMIT 2 +OFFSET 2; + +-- TC 02.02 +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b >= 8 + LIMIT 2 + OFFSET 2) +LIMIT 4 +OFFSET 2; + +-- TC 02.03 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + ORDER BY t2c, t2d + LIMIT 2) +GROUP BY t1b +ORDER BY t1b DESC NULLS FIRST +LIMIT 1 +OFFSET 1; + +-- LIMIT with NOT IN +-- TC 02.04 +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b > 6 + LIMIT 2 + OFFSET 2); + +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b <= t1d + LIMIT 2); + +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b = t1b + LIMIT 2 + OFFSET 2); + +-- TC 02.05 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + ORDER BY t2b DESC nulls first, t2d + LIMIT 1 + OFFSET 1) +GROUP BY t1b +ORDER BY t1b NULLS last +LIMIT 1 +OFFSET 1; + +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + WHERE t2b > t1b + ORDER BY t2b DESC nulls first, t2d + LIMIT 1 + OFFSET 1) +GROUP BY t1b +ORDER BY t1b NULLS last +LIMIT 1 +OFFSET 1; + +-- OFFSET in parent side +-- TC 03.01 +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +OFFSET 2; + +-- OFFSET in correlated subquery +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d + ORDER BY t2a + OFFSET 2) +OFFSET 2; + +-- TC 03.02 +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b >= 8 + OFFSET 2) +OFFSET 4; + +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b < t1b + ORDER BY t2c + OFFSET 2) +OFFSET 1; + +-- TC 03.03 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + ORDER BY t2c, t2d + OFFSET 2) +GROUP BY t1b +ORDER BY t1b DESC NULLS FIRST +OFFSET 1; + +-- OFFSET with NOT IN +-- TC 03.04 +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b > 6 + OFFSET 2); + +-- OFFSET with NOT IN correlated +SELECT count(*) +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b < t1b + OFFSET 2); + +-- TC 03.05 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + ORDER BY t2b DESC nulls first, t2d + OFFSET 1) +GROUP BY t1b +ORDER BY t1b NULLS last +OFFSET 1; + +-- SPARK-46526: LIMIT over correlated predicate that references only the outer table. +SELECT COUNT(DISTINCT(t1a)) +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + WHERE t1a IS NOT NULL + LIMIT 10); + +SELECT COUNT(DISTINCT(t1a)) +FROM t1 +WHERE t1d IN (SELECT MAX(t2d) + FROM t2 + WHERE t1a IS NOT NULL + LIMIT 10); + +SELECT COUNT(DISTINCT(t1a)) +FROM t1 +WHERE t1d IN (SELECT DISTINCT t2d + FROM t2 + WHERE t1a IS NOT NULL + LIMIT 10); + +set spark.sql.optimizer.decorrelateExistsIn.enabled = false; +-- LIMIT is not supported in correlated IN, unless the DECORRELATE_EXISTS_AND_IN_SUBQUERIES +-- is enabled. +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b = t1b + LIMIT 2 + OFFSET 2); +set spark.sql.optimizer.decorrelateExistsIn.enabled = true; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-multiple-columns.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-multiple-columns.sql new file mode 100644 index 000000000000..c403c2d66ab1 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-multiple-columns.sql @@ -0,0 +1,131 @@ +-- A test suite for multiple columns in predicate in parent side, subquery, and both predicate subquery +-- It includes correlated cases. + +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + +--ONLY_IF spark +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- TC 01.01 +SELECT t1a, + t1b, + t1h +FROM t1 +WHERE ( t1a, t1h ) NOT IN (SELECT t2a, + t2h + FROM t2 + WHERE t2a = t1a + ORDER BY t2a) +AND t1a = 'val1a'; + +-- TC 01.02 +SELECT t1a, + t1b, + t1d +FROM t1 +WHERE ( t1b, t1d ) IN (SELECT t2b, + t2d + FROM t2 + WHERE t2i IN (SELECT t3i + FROM t3 + WHERE t2b > t3b)); + +-- TC 01.03 +SELECT t1a, + t1b, + t1d +FROM t1 +WHERE ( t1b, t1d ) NOT IN (SELECT t2b, + t2d + FROM t2 + WHERE t2h IN (SELECT t3h + FROM t3 + WHERE t2b > t3b)) +AND t1a = 'val1a'; + +-- TC 01.04 +SELECT t2a +FROM (SELECT t2a + FROM t2 + WHERE ( t2a, t2b ) IN (SELECT t1a, + t1b + FROM t1) + UNION ALL + SELECT t2a + FROM t2 + WHERE ( t2a, t2b ) IN (SELECT t1a, + t1b + FROM t1) + UNION DISTINCT + SELECT t2a + FROM t2 + WHERE ( t2a, t2b ) IN (SELECT t3a, + t3b + FROM t3)) AS t4; + +-- TC 01.05 +WITH cte1 AS +( + SELECT t1a, + t1b + FROM t1 + WHERE ( + t1b, t1d) IN + ( + SELECT t2b, + t2d + FROM t2 + WHERE t1c = t2c)) +SELECT * +FROM ( + SELECT * + FROM cte1 + JOIN cte1 cte2 + on cte1.t1b = cte2.t1b) s; + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-null-semantics.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-null-semantics.sql new file mode 100644 index 000000000000..d739580113ac --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-null-semantics.sql @@ -0,0 +1,68 @@ +--ONLY_IF spark +create temp view v (c) as values (1), (null); +create temp view v_empty (e) as select 1 where false; + +-- Note: tables and temp views hit different optimization/execution codepaths: expressions over temp views are evaled at query compilation time by ConvertToLocalRelation +create table t(c int) using json; +insert into t values (1), (null); +create table t2(d int) using json; +insert into t2 values (2); +create table t_empty(e int) using json; + + + +set spark.sql.legacy.nullInEmptyListBehavior = false; + +-- null IN (empty subquery) +-- Correct results: c in (emptylist) should always be false + +select c, c in (select e from t_empty) from t; +select c, c in (select e from v_empty) from v; +select c, c not in (select e from t_empty) from t; +select c, c not in (select e from v_empty) from v; + +-- constant null IN (empty subquery) - rewritten by NullPropagation rule +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding; + +select null in (select e from t_empty); +select null in (select e from v_empty); +select null not in (select e from t_empty); +select null not in (select e from v_empty); + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; +-- IN subquery which is not rewritten to join - here we use IN in the ON condition because that is a case that doesn't get rewritten to join in RewritePredicateSubquery, so we can observe the execution behavior of InSubquery directly +-- Correct results: column t2.d should be NULL because the ON condition is always false +select * from t left join t2 on (t.c in (select e from t_empty)) is null; +select * from t left join t2 on (t.c not in (select e from t_empty)) is null; + +-- Should have the same results as above with optimize IN subqueries enabled +set spark.sql.optimizer.optimizeUncorrelatedInSubqueriesInJoinCondition.enabled=true; + +-- IN subquery which IS rewritten to join +select * from t left join t2 on (t.c in (select e from t_empty)) is null; +select * from t left join t2 on (t.c not in (select e from t_empty)) is null; + +set spark.sql.legacy.nullInEmptyListBehavior = true; +-- Disable optimize IN subqueries to joins because it affects null semantics +set spark.sql.optimizer.optimizeUncorrelatedInSubqueriesInJoinCondition.enabled=false; + +-- constant null IN (empty subquery) - rewritten by NullPropagation rule +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding; + +select null in (select e from t_empty); +select null in (select e from v_empty); +select null not in (select e from t_empty); +select null not in (select e from v_empty); + +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation; +-- IN subquery which is not rewritten to join - here we use IN in the ON condition because that is a case that doesn't get rewritten to join in RewritePredicateSubquery, so we can observe the execution behavior of InSubquery directly +-- Correct results: column t2.d should be NULL because the ON condition is always false +select * from t left join t2 on (t.c in (select e from t_empty)) is null; +select * from t left join t2 on (t.c not in (select e from t_empty)) is null; + +reset spark.sql.legacy.nullInEmptyListBehavior; +reset spark.sql.optimizer.optimizeUncorrelatedInSubqueriesInJoinCondition.enabled; + +drop table t; +drop table t2; +drop table t_empty; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-nullability.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-nullability.sql new file mode 100644 index 000000000000..30f3a25f0e85 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-nullability.sql @@ -0,0 +1,15 @@ +-- SPARK-43413: Tests for IN subquery nullability +--ONLY_IF spark + +create temp view t0 as select 1 as a_nonnullable; +create temp view t1 as select cast(null as int) as b_nullable; +create temp view t2 as select 2 as c; + +select * from t0 where a_nonnullable in (select b_nullable from t1); +select * from t0 where (a_nonnullable in (select b_nullable from t1)) <=> true; +select * from t0 where a_nonnullable not in (select b_nullable from t1); +select * from t0 where (a_nonnullable not in (select b_nullable from t1)) <=> true; + +-- IN subqueries in ON conditions are not rewritten to joins in RewritePredicateSubquery +select * from t0 left join t2 on (a_nonnullable IN (select b_nullable from t1)) is null; +select * from t0 left join t2 on (a_nonnullable IN (select b_nullable from t1)) <=> true; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-order-by.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-order-by.sql new file mode 100644 index 000000000000..8bf49a1c2d99 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-order-by.sql @@ -0,0 +1,206 @@ +-- A test suite for ORDER BY in parent side, subquery, and both predicate subquery +-- It includes correlated cases. + +-- Test sort operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +--CONFIG_DIM2 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM2 spark.sql.optimizeNullAwareAntiJoin=false +--ONLY_IF spark + +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- ORDER BY in parent side +-- TC 01.01 +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2) +ORDER BY t1a; + +-- TC 01.02 +SELECT t1a +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a) +ORDER BY t1b DESC; + +-- TC 01.03 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) +ORDER BY 2 DESC nulls last; + +-- TC 01.04 +SELECT Count(DISTINCT( t1a )) +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a) +ORDER BY Count(DISTINCT( t1a )); + +-- ORDER BY in subquery +-- TC 01.05 +SELECT * +FROM t1 +WHERE t1b IN (SELECT t2c + FROM t2 + ORDER BY t2d); + +-- ORDER BY in BOTH +-- TC 01.06 +SELECT * +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1b = t2b + ORDER BY Min(t2b)) +ORDER BY t1c DESC nulls first, t1a DESC, t1d DESC, t1h; + +-- TC 01.07 +SELECT t1a, + t1b, + t1h +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a + ORDER BY t2b DESC nulls first) + OR t1h IN (SELECT t2h + FROM t2 + WHERE t1h > t2h) +ORDER BY t1h DESC nulls last; + +-- ORDER BY with NOT IN +-- TC 01.08 +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2) +ORDER BY t1a; + +-- TC 01.09 +SELECT t1a, + t1b +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t1a = t2a) +ORDER BY t1b DESC nulls last; + +-- TC 01.10 +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + ORDER BY t2a DESC nulls first) + and t1c IN (SELECT t2c + FROM t2 + ORDER BY t2b DESC nulls last) +ORDER BY t1c DESC nulls last; + +-- GROUP BY and ORDER BY +-- TC 01.11 +SELECT * +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + GROUP BY t2a + ORDER BY t2a DESC); + +-- TC 01.12 +SELECT t1a, + Count(DISTINCT( t1b )) +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2a + ORDER BY t2a) +GROUP BY t1a, + t1h +ORDER BY t1a; + +-- GROUP BY and ORDER BY with NOT IN +-- TC 01.13 +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT Min(t2b) + FROM t2 + GROUP BY t2a + ORDER BY t2a); + +-- TC 01.14 +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1b NOT IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2c + ORDER BY t2c DESC nulls last) +GROUP BY t1a; + +-- TC 01.15 +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1h NOT IN (SELECT t2h + FROM t2 + where t1a = t2a + order by t2d DESC nulls first + ) +GROUP BY t1a, + t1b +ORDER BY t1b DESC nulls last; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-set-operations.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-set-operations.sql new file mode 100644 index 000000000000..c6b6a338c9b1 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-set-operations.sql @@ -0,0 +1,582 @@ +-- A test suite for set-operations in parent side, subquery, and both predicate subquery +-- It includes correlated cases. +--ONLY_IF spark + +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- UNION, UNION ALL, UNION DISTINCT, INTERSECT and EXCEPT in the parent +-- TC 01.01 +SELECT t2a, + t2b, + t2c, + t2h, + t2i +FROM (SELECT * + FROM t2 + WHERE t2a IN (SELECT t1a + FROM t1) + UNION ALL + SELECT * + FROM t3 + WHERE t3a IN (SELECT t1a + FROM t1)) AS t3 +WHERE t2i IS NOT NULL AND + 2 * t2b = t2c +ORDER BY t2c DESC nulls first; + +-- TC 01.02 +SELECT t2a, + t2b, + t2d, + Count(DISTINCT( t2h )), + t2i +FROM (SELECT * + FROM t2 + WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t2b = t1b) + UNION + SELECT * + FROM t1 + WHERE t1a IN (SELECT t3a + FROM t3 + WHERE t1c = t3c)) AS t3 +GROUP BY t2a, + t2b, + t2d, + t2i +ORDER BY t2d DESC; + +-- TC 01.03 +SELECT t2a, + t2b, + t2c, + Min(t2d) +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b) +GROUP BY t2a, t2b, t2c +UNION ALL +SELECT t2a, + t2b, + t2c, + Max(t2d) +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t2c = t1c) +GROUP BY t2a, t2b, t2c +UNION +SELECT t3a, + t3b, + t3c, + Min(t3d) +FROM t3 +WHERE t3a IN (SELECT t2a + FROM t2 + WHERE t3c = t2c) +GROUP BY t3a, t3b, t3c +UNION DISTINCT +SELECT t1a, + t1b, + t1c, + Max(t1d) +FROM t1 +WHERE t1a IN (SELECT t3a + FROM t3 + WHERE t3d = t1d) +GROUP BY t1a, t1b, t1c; + +-- TC 01.04 +SELECT DISTINCT( t2a ), + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i +UNION +SELECT DISTINCT( t2a ), + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t2c = t1c) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i +HAVING t2b IS NOT NULL; + +-- TC 01.05 +SELECT t2a, + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2a IN (SELECT DISTINCT(t1a) + FROM t1 + WHERE t1b = t2b) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i + +UNION +SELECT DISTINCT( t2a ), + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2b IN (SELECT Max(t1b) + FROM t1 + WHERE t2c = t1c) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i +HAVING t2b IS NOT NULL +UNION DISTINCT +SELECT t2a, + t2b, + t2c, + t2d, + t2h, + t2i +FROM t2 +WHERE t2d IN (SELECT min(t1d) + FROM t1 + WHERE t2c = t1c); + +-- TC 01.06 +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b AND + t1d < t2d) +INTERSECT +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2b IN (SELECT Max(t1b) + FROM t1 + WHERE t2c = t1c) +EXCEPT +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2d IN (SELECT Min(t3d) + FROM t3 + WHERE t2c = t3c) +UNION ALL +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2c IN (SELECT Max(t1c) + FROM t1 + WHERE t1d = t2d); + +-- UNION, UNION ALL, UNION DISTINCT, INTERSECT and EXCEPT in the subquery +-- TC 01.07 +SELECT DISTINCT(t1a), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a IN (SELECT t3a + FROM (SELECT t2a t3a + FROM t2 + UNION ALL + SELECT t2a t3a + FROM t2) AS t3 + UNION + SELECT t2a + FROM (SELECT t2a + FROM t2 + WHERE t2b > 6 + UNION + SELECT t2a + FROM t2 + WHERE t2b > 6) AS t4 + UNION DISTINCT + SELECT t2a + FROM (SELECT t2a + FROM t2 + WHERE t2b > 6 + UNION DISTINCT + SELECT t1a + FROM t1 + WHERE t1b > 6) AS t5) +GROUP BY t1a, t1b, t1c, t1d +HAVING t1c IS NOT NULL AND t1b IS NOT NULL +ORDER BY t1c DESC, t1a DESC; + +-- TC 01.08 +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN (SELECT t2b + FROM (SELECT t2b + FROM t2 + WHERE t2b > 6 + INTERSECT + SELECT t1b + FROM t1 + WHERE t1b > 6) AS t3 + WHERE t2b = t1b); + +-- TC 01.09 +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1h IN (SELECT t2h + FROM (SELECT t2h + FROM t2 + EXCEPT + SELECT t3h + FROM t3) AS t3) +ORDER BY t1b DESC NULLs first, t1c DESC NULLs last; + +-- UNION, UNION ALL, UNION DISTINCT, INTERSECT and EXCEPT in the parent and subquery +-- TC 01.10 +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN + ( + SELECT t2b + FROM ( + SELECT t2b + FROM t2 + WHERE t2b > 6 + INTERSECT + SELECT t1b + FROM t1 + WHERE t1b > 6) AS t3) +UNION DISTINCT +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN + ( + SELECT t2b + FROM ( + SELECT t2b + FROM t2 + WHERE t2b > 6 + EXCEPT + SELECT t1b + FROM t1 + WHERE t1b > 6) AS t4 + WHERE t2b = t1b) +ORDER BY t1c DESC NULLS last, t1a DESC; + +-- TC 01.11 +SELECT * +FROM (SELECT * + FROM (SELECT * + FROM t2 + WHERE t2h IN (SELECT t1h + FROM t1 + WHERE t1a = t2a) + UNION DISTINCT + SELECT * + FROM t1 + WHERE t1h IN (SELECT t3h + FROM t3 + UNION + SELECT t1h + FROM t1) + UNION + SELECT * + FROM t3 + WHERE t3a IN (SELECT t2a + FROM t2 + UNION ALL + SELECT t1a + FROM t1 + WHERE t1b > 0) + INTERSECT + SELECT * + FROM T1 + WHERE t1b IN (SELECT t3b + FROM t3 + UNION DISTINCT + SELECT t2b + FROM t2 + ) + EXCEPT + SELECT * + FROM t2 + WHERE t2h IN (SELECT t1i + FROM t1)) t4 + WHERE t4.t2b IN (SELECT Min(t3b) + FROM t3 + WHERE t4.t2a = t3a)); + +-- UNION, UNION ALL, UNION DISTINCT, INTERSECT and EXCEPT for NOT IN +-- TC 01.12 +SELECT t2a, + t2b, + t2c, + t2i +FROM (SELECT * + FROM t2 + WHERE t2a NOT IN (SELECT t1a + FROM t1 + UNION + SELECT t3a + FROM t3) + UNION ALL + SELECT * + FROM t2 + WHERE t2a NOT IN (SELECT t1a + FROM t1 + INTERSECT + SELECT t2a + FROM t2)) AS t3 +WHERE t3.t2a NOT IN (SELECT t1a + FROM t1 + INTERSECT + SELECT t2a + FROM t2) + AND t2c IS NOT NULL +ORDER BY t2a; + +-- TC 01.13 +SELECT Count(DISTINCT(t1a)), + t1b, + t1c, + t1i +FROM t1 +WHERE t1b NOT IN + ( + SELECT t2b + FROM ( + SELECT t2b + FROM t2 + WHERE t2b NOT IN + ( + SELECT t1b + FROM t1) + UNION + SELECT t1b + FROM t1 + WHERE t1b NOT IN + ( + SELECT t3b + FROM t3) + UNION + distinct SELECT t3b + FROM t3 + WHERE t3b NOT IN + ( + SELECT t2b + FROM t2)) AS t3 + WHERE t2b = t1b) +GROUP BY t1a, + t1b, + t1c, + t1i +HAVING t1b NOT IN + ( + SELECT t2b + FROM t2 + WHERE t2c IS NULL + EXCEPT + SELECT t3b + FROM t3) +ORDER BY t1c DESC NULLS LAST, t1i; + +-- Correlated set ops inside IN - unsupported + +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + UNION ALL + SELECT t3a + FROM t3); + +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + UNION DISTINCT + SELECT t3a + FROM t3); + +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + INTERSECT ALL + SELECT t3a + FROM t3); + +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + INTERSECT DISTINCT + SELECT t3a + FROM t3); + +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + EXCEPT ALL + SELECT t3a + FROM t3); + +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + EXCEPT DISTINCT + SELECT t3a + FROM t3); + +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + UNION ALL + SELECT t3a + FROM t3); + +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + UNION DISTINCT + SELECT t3a + FROM t3); + +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + INTERSECT ALL + SELECT t3a + FROM t3); + +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + INTERSECT DISTINCT + SELECT t3a + FROM t3); + +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + EXCEPT ALL + SELECT t3a + FROM t3); + +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + EXCEPT DISTINCT + SELECT t3a + FROM t3); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-subquery-in-join-condition.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-subquery-in-join-condition.sql new file mode 100644 index 000000000000..c906390c99c3 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-subquery-in-join-condition.sql @@ -0,0 +1,90 @@ +-- Test that correlated EXISTS subqueries in join conditions are supported. +--ONLY_IF spark + +-- Permutations of the test: +-- 1. In / Not In +-- 2. Reference left / right child +-- 3. Join type: inner / left outer / right outer / full outer / left semi / left anti +-- 4. AND or OR for the join condition + +CREATE TEMP VIEW x(x1, x2) AS VALUES + (2, 1), + (1, 1), + (3, 4); + +CREATE TEMP VIEW y(y1, y2) AS VALUES + (0, 2), + (1, 4), + (4, 11); + +CREATE TEMP VIEW z(z1, z2) AS VALUES + (4, 2), + (3, 3), + (8, 1); + + +--Correlated IN, REFERENCE LEFT, INNER JOIN +select * from x inner join y on x1 = y1 and x2 IN (select z1 from z where z2 = x2) order by x1, x2, y1, y2; + +--Correlated NOT IN, REFERENCE LEFT, INNER JOIN +select * from x inner join y on x1 = y1 and x2 not IN (select z1 from z where z2 = x2) order by x1, x2, y1, y2; + +--Correlated IN, REFERENCE RIGHT, INNER JOIN +select * from x inner join y on x1 = y1 and y2 IN (select z1 from z where z2 = y2) order by x1, x2, y1, y2; + +--Correlated NOT IN, REFERENCE RIGHT, INNER JOIN +select * from x inner join y on x1 = y1 and y2 not IN (select z1 from z where z2 = y2) order by x1, x2, y1, y2; + +-- Same as above, but for left join +select * from x left join y on x1 = y1 and x2 IN (select z1 from z where z2 = x2) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 and x2 not IN (select z1 from z where z2 = x2) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 and y2 IN (select z1 from z where z2 = y2) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 and y2 not IN (select z1 from z where z2 = y2) order by x1, x2, y1, y2; + +-- Same as above, but for right join +select * from x right join y on x1 = y1 and x2 IN (select z1 from z where z2 = x2) order by x1, x2, y1, y2; +select * from x right join y on x1 = y1 and x2 not IN (select z1 from z where z2 = x2) order by x1, x2, y1, y2; +select * from x right join y on x1 = y1 and y2 IN (select z1 from z where z2 = y2) order by x1, x2, y1, y2; +select * from x right join y on x1 = y1 and y2 not IN (select z1 from z where z2 = y2) order by x1, x2, y1, y2; + +-- Same as above, but for left semi join +select * from x left semi join y on x1 = y1 and x2 IN (select z1 from z where z2 = x2) order by x1, x2; +select * from x left semi join y on x1 = y1 and x2 not IN (select z1 from z where z2 = x2) order by x1, x2; +select * from x left semi join y on x1 = y1 and y2 IN (select z1 from z where z2 = y2) order by x1, x2; +select * from x left semi join y on x1 = y1 and y2 not IN (select z1 from z where z2 = y2) order by x1, x2; + +-- Same as above, but for left anti join +select * from x left anti join y on x1 = y1 and x2 IN (select z1 from z where z2 = x2) order by x1, x2; +select * from x left anti join y on x1 = y1 and x2 not IN (select z1 from z where z2 = x2) order by x1, x2; +select * from x left anti join y on x1 = y1 and y2 IN (select z1 from z where z2 = y2) order by x1, x2; +select * from x left anti join y on x1 = y1 and y2 not IN (select z1 from z where z2 = y2) order by x1, x2; + +-- Same as above, but for full outer join +select * from x full outer join y on x1 = y1 and x2 IN (select z1 from z where z2 = x2) order by x1, x2, y1, y2; +select * from x full outer join y on x1 = y1 and x2 not IN (select z1 from z where z2 = x2) order by x1, x2, y1, y2; +select * from x full outer join y on x1 = y1 and y2 IN (select z1 from z where z2 = y2) order by x1, x2, y1, y2; +select * from x full outer join y on x1 = y1 and y2 not IN (select z1 from z where z2 = y2) order by x1, x2, y1, y2; + +-- OR instead of AND in the join condition +select * from x inner join y on x1 = y1 or x2 IN (select z1 from z where z1 = x1) order by x1, x2, y1, y2; +select * from x inner join y on x1 = y1 or x2 not IN (select z1 from z where z1 = x1) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 or x2 IN (select z1 from z where z1 = x1) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 or x2 not IN (select z1 from z where z1 = x1) order by x1, x2, y1, y2; +select * from x inner join y on x1 = y1 or y2 IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2; +select * from x inner join y on x1 = y1 or y2 not IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 or y2 IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 or y2 not IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2; + +-- Transitive predicates to test if inferring filters can cause issues. +select * from x inner join y on x1 = y1 and x2 IN (select z1 from z where z1 = x1) order by x1, x2, y1, y2; +select * from x inner join y on x1 = y1 and x2 not IN (select z1 from z where z1 = x1) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 and x2 IN (select z1 from z where z1 = x1) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 and x2 not IN (select z1 from z where z1 = x1) order by x1, x2, y1, y2; +select * from x inner join y on x1 = y1 and y2 IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2; +select * from x inner join y on x1 = y1 and y2 not IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 and y2 IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 and y2 not IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2; + +-- Correlated subquery references both left and right children, errors +select * from x left join y on x1 = y1 and x2 IN (select z1 from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2; +select * from x left join y on x1 = y1 and x2 not IN (select z1 from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-with-cte.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-with-cte.sql new file mode 100644 index 000000000000..8d08cfb4da34 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/in-with-cte.sql @@ -0,0 +1,291 @@ +-- A test suite for in with cte in parent side, subquery, and both predicate subquery +-- It includes correlated cases. +--ONLY_IF spark + +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- outside CTE +-- TC 01.01 +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1a = "val1a") +SELECT t1a, + t1b, + t1c, + t1d, + t1h +FROM t1 +WHERE t1b IN (SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b > 0); + +-- TC 01.02 +WITH cte1 AS +( + SELECT t1a, + t1b + FROM t1) +SELECT count(distinct(t1a)), t1b, t1c +FROM t1 +WHERE t1b IN + ( + SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b > 0 + UNION + SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b > 5 + UNION ALL + SELECT cte1.t1b + FROM cte1 + INTERSECT + SELECT cte1.t1b + FROM cte1 + UNION + SELECT cte1.t1b + FROM cte1 ) +GROUP BY t1a, t1b, t1c +HAVING t1c IS NOT NULL; + +-- TC 01.03 +WITH cte1 AS +( + SELECT t1a, + t1b, + t1c, + t1d, + t1e + FROM t1) +SELECT t1a, + t1b, + t1c, + t1h +FROM t1 +WHERE t1c IN + ( + SELECT cte1.t1c + FROM cte1 + JOIN cte1 cte2 + on cte1.t1b > cte2.t1b + FULL OUTER JOIN cte1 cte3 + ON cte1.t1c = cte3.t1c + LEFT JOIN cte1 cte4 + ON cte1.t1d = cte4.t1d + INNER JOIN cte1 cte5 + ON cte1.t1b < cte5.t1b + LEFT OUTER JOIN cte1 cte6 + ON cte1.t1d > cte6.t1d); + +-- CTE inside and outside +-- TC 01.04 +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1b IN (SELECT t2b + FROM t2 + RIGHT JOIN t1 + ON t1c = t2c + LEFT JOIN t3 + ON t2d = t3d) + AND t1a = "val1b") +SELECT * +FROM (SELECT * + FROM cte1 + JOIN cte1 cte2 + ON cte1.t1b > 5 + AND cte1.t1a = cte2.t1a + FULL OUTER JOIN cte1 cte3 + ON cte1.t1a = cte3.t1a + INNER JOIN cte1 cte4 + ON cte1.t1b = cte4.t1b) s; + +-- TC 01.05 +WITH cte1 AS +( + SELECT t1a, + t1b, + t1h + FROM t1 + WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1b < t2b)) +SELECT Count(DISTINCT t1a), + t1b +FROM ( + SELECT cte1.t1a, + cte1.t1b + FROM cte1 + JOIN cte1 cte2 + on cte1.t1h >= cte2.t1h) s +WHERE t1b IN + ( + SELECT t1b + FROM t1) +GROUP BY t1b; + +-- TC 01.06 +WITH cte1 AS +( + SELECT t1a, + t1b, + t1c + FROM t1 + WHERE t1b IN + ( + SELECT t2b + FROM t2 FULL OUTER JOIN T3 on t2a = t3a + WHERE t1c = t2c) AND + t1a = "val1b") +SELECT * +FROM ( + SELECT * + FROM cte1 + INNER JOIN cte1 cte2 ON cte1.t1a = cte2.t1a + RIGHT OUTER JOIN cte1 cte3 ON cte1.t1b = cte3.t1b + LEFT OUTER JOIN cte1 cte4 ON cte1.t1c = cte4.t1c + ) s +; + +-- TC 01.07 +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1c = t2c)) +SELECT Count(DISTINCT( s.t1a )), + s.t1b +FROM (SELECT cte1.t1a, + cte1.t1b + FROM cte1 + RIGHT OUTER JOIN cte1 cte2 + ON cte1.t1a = cte2.t1a) s +GROUP BY s.t1b; + +-- TC 01.08 +WITH cte1 AS +( + SELECT t1a, + t1b + FROM t1 + WHERE t1b IN + ( + SELECT t2b + FROM t2 + WHERE t1c = t2c)) +SELECT DISTINCT(s.t1b) +FROM ( + SELECT cte1.t1b + FROM cte1 + LEFT OUTER JOIN cte1 cte2 + ON cte1.t1b = cte2.t1b) s +WHERE s.t1b IN + ( + SELECT t1.t1b + FROM t1 INNER + JOIN cte1 + ON t1.t1a = cte1.t1a); + +-- CTE with NOT IN +-- TC 01.09 +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1a = "val1d") +SELECT t1a, + t1b, + t1c, + t1h +FROM t1 +WHERE t1b NOT IN (SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b < 0) AND + t1c > 10; + +-- TC 01.10 +WITH cte1 AS +( + SELECT t1a, + t1b, + t1c, + t1d, + t1h + FROM t1 + WHERE t1d NOT IN + ( + SELECT t2d + FROM t2 + FULL OUTER JOIN t3 ON t2a = t3a + JOIN t1 on t1b = t2b)) +SELECT t1a, + t1b, + t1c, + t1d, + t1h +FROM t1 +WHERE t1b NOT IN + ( + SELECT cte1.t1b + FROM cte1 INNER + JOIN cte1 cte2 ON cte1.t1a = cte2.t1a + RIGHT JOIN cte1 cte3 ON cte1.t1b = cte3.t1b + JOIN cte1 cte4 ON cte1.t1c = cte4.t1c) AND + t1c IS NOT NULL +ORDER BY t1c DESC; + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/nested-not-in.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/nested-not-in.sql new file mode 100644 index 000000000000..9472690d9914 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/nested-not-in.sql @@ -0,0 +1,202 @@ +-- Tests NOT-IN subqueries nested inside OR expression(s). +--ONLY_IF spark + +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", 10), + (200, "emp 2", NULL), + (300, "emp 3", 20), + (400, "emp 4", 30), + (500, "emp 5", NULL), + (600, "emp 6", 100), + (800, "emp 8", 70) +AS EMP(id, emp_name, dept_id); + +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt); + +CREATE TEMPORARY VIEW ADDRESS AS SELECT * FROM VALUES + (100, "emp 1", "addr1"), + (200, null, "addr2"), + (null, "emp 3", "addr3"), + (null, null, "addr4"), + (600, "emp 6", "addr6"), + (800, "emp 8", "addr8") +AS ADDRESS(id, emp_name, address); + +CREATE TEMPORARY VIEW S1 AS SELECT * FROM VALUES + (null, null), (5, 5), (8, 8), (11, 11) AS s1(a, b); +CREATE TEMPORARY VIEW S2 AS SELECT * FROM VALUES + (7, 7), (8, 8), (11, 11), (null, null) AS s2(c, d); + +-- null produced from both sides. +-- TC.01.01 +SELECT id, + dept_id +FROM emp +WHERE id = 600 + OR id = 500 + OR dept_id NOT IN (SELECT dept_id + FROM emp); + +-- null produced from right side +-- TC.01.02 +SELECT id, + dept_id +FROM emp +WHERE id = 800 + OR (dept_id IS NOT NULL + AND dept_id NOT IN (SELECT dept_id + FROM emp)); + +-- null produced on left side +-- TC.01.03 +SELECT id, + dept_id +FROM emp +WHERE id = 100 + OR dept_id NOT IN (SELECT dept_id + FROM emp + WHERE dept_id IS NOT NULL); + +-- no null in both left and right +-- TC.01.04 +SELECT id, + dept_id +FROM emp +WHERE id = 200 + OR (dept_id IS NOT NULL + AND dept_id + 100 NOT IN (SELECT dept_id + FROM emp + WHERE dept_id IS NOT NULL)); + +-- complex nesting +-- TC.01.05 +SELECT id, + dept_id, + emp_name +FROM emp +WHERE emp_name IN (SELECT emp_name + FROM bonus) + OR (dept_id IS NOT NULL + AND dept_id NOT IN (SELECT dept_id + FROM dept)); + +-- complex nesting, exists in disjunction with not-in +-- TC.01.06 +SELECT id, + dept_id, + emp_name +FROM emp +WHERE EXISTS (SELECT emp_name + FROM bonus + WHERE emp.emp_name = bonus.emp_name) + OR (dept_id IS NOT NULL + AND dept_id NOT IN (SELECT dept_id + FROM dept)); + +-- multiple columns in not-in +-- TC.01.07 +SELECT id, + dept_id, + emp_name +FROM emp +WHERE dept_id = 10 +OR (id, emp_name) NOT IN (SELECT id, emp_name FROM address); + +-- multiple columns in not-in +-- TC.01.08 +SELECT id, + dept_id, + emp_name +FROM emp +WHERE dept_id = 10 + OR (( id, emp_name ) NOT IN (SELECT id, + emp_name + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL) + AND id > 400 ); +-- correlated not-in along with disjunction +-- TC.01.09 +SELECT id, + dept_id, + emp_name +FROM emp +WHERE dept_id = 10 + OR emp_name NOT IN (SELECT emp_name + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL + AND emp.id = address.id); + +-- multiple not-in(s) in side disjunction` +-- TC.01.10 +SELECT id, + dept_id, + emp_name +FROM emp +WHERE id NOT IN (SELECT id + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL + AND id >= 400) + OR emp_name NOT IN (SELECT emp_name + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL + AND emp.id = address.id + AND id < 400); + +-- NOT (NOT IN (SUBQ)) +SELECT * +FROM s1 +WHERE NOT (a NOT IN (SELECT c + FROM s2)); + +-- NOT (OR (expression, IN-SUBQ)) +SELECT * +FROM s1 +WHERE NOT (a > 5 + OR a IN (SELECT c + FROM s2)); + +-- NOT (OR (expression, NOT-IN-SUB) +SELECT * +FROM s1 +WHERE NOT (a > 5 + OR a NOT IN (SELECT c + FROM s2)); + +-- NOT (AND (expression, IN-SUB)) +SELECT * +FROM s1 +WHERE NOT (a > 5 + AND a IN (SELECT c + FROM s2)); + +-- NOT (AND (expression, NOT-IN-SUBQ)) +SELECT * +FROM s1 +WHERE NOT (a > 5 + AND a NOT IN (SELECT c + FROM s2)); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/not-in-group-by.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/not-in-group-by.sql new file mode 100644 index 000000000000..97da00ac2cef --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/not-in-group-by.sql @@ -0,0 +1,107 @@ +-- A test suite for NOT IN GROUP BY in parent side, subquery, and both predicate subquery +-- It includes correlated cases. +--ONLY_IF spark + +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + + +-- correlated IN subquery +-- GROUP BY in parent side +-- TC 01.01 +SELECT t1a, + Avg(t1b) +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2) +GROUP BY t1a; + +-- TC 01.02 +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + WHERE t1h < t2h) +GROUP BY t1a; + +-- TC 01.03 +SELECT Count(*) +FROM (SELECT * + FROM t2 + WHERE t2a NOT IN (SELECT t3a + FROM t3 + WHERE t3h != t2h)) t2 +WHERE t2b NOT IN (SELECT Min(t2b) + FROM t2 + WHERE t2b = t2b + GROUP BY t2c); + +-- TC 01.04 +SELECT t1a, + max(t1b) +FROM t1 +WHERE t1c NOT IN (SELECT Max(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2a) +GROUP BY t1a; + +-- TC 01.05 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t2a NOT IN (SELECT Min(t3a) + FROM t3 + WHERE t3a = t2a + GROUP BY t3b) order by t2a); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql new file mode 100644 index 000000000000..639034c18f31 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/not-in-joins.sql @@ -0,0 +1,171 @@ +-- A test suite for not-in-joins in parent side, subquery, and both predicate subquery +-- It includes correlated cases. +--ONLY_IF spark + +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- different not JOIN in parent side +-- TC 01.01 +SELECT t1a, + t1b, + t1c, + t3a, + t3b, + t3c +FROM t1 + JOIN t3 +WHERE t1a NOT IN (SELECT t2a + FROM t2) + AND t1b = t3b; + +-- TC 01.02 +SELECT t1a, + t1b, + t1c, + count(distinct(t3a)), + t3b, + t3c +FROM t1 +FULL OUTER JOIN t3 on t1b != t3b +RIGHT JOIN t2 on t1c = t2c +where t1a NOT IN + ( + SELECT t2a + FROM t2 + WHERE t2c NOT IN + ( + SELECT t1c + FROM t1 + WHERE t1a = t2a)) +AND t1b != t3b +AND t1d = t2d +GROUP BY t1a, t1b, t1c, t3a, t3b, t3c +HAVING count(distinct(t3a)) >= 1 +ORDER BY t1a, t3b; + +-- TC 01.03 +SELECT t1a, + t1b, + t1c, + t1d, + t1h +FROM t1 +WHERE t1a NOT IN + ( + SELECT t2a + FROM t2 + LEFT JOIN t3 on t2b = t3b + WHERE t1d = t2d + ) +AND t1d NOT IN + ( + SELECT t2d + FROM t2 + RIGHT JOIN t1 on t2e = t1e + WHERE t1a = t2a); + +-- TC 01.04 +SELECT Count(DISTINCT( t1a )), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + JOIN t1 + WHERE t2b <> t1b) +GROUP BY t1b, + t1c, + t1d +HAVING t1d NOT IN (SELECT t2d + FROM t2 + WHERE t1d = t2d) +ORDER BY t1b DESC, t1d ASC; + +-- TC 01.05 +SELECT COUNT(DISTINCT(t1a)), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a NOT IN + ( + SELECT t2a + FROM t2 INNER + JOIN t1 ON t1a = t2a) +GROUP BY t1b, + t1c, + t1d +HAVING t1b < sum(t1c); + +-- TC 01.06 +SELECT COUNT(DISTINCT(t1a)), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a NOT IN + ( + SELECT t2a + FROM t2 INNER + JOIN t1 + ON t1a = t2a) +AND t1d NOT IN + ( + SELECT t2d + FROM t2 + INNER JOIN t3 + ON t2b = t3b ) +GROUP BY t1b, + t1c, + t1d +HAVING t1b < sum(t1c); + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql new file mode 100644 index 000000000000..e9f3fda9c1b9 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql @@ -0,0 +1,43 @@ +-- Unit tests for simple NOT IN predicate subquery across multiple columns. +-- +-- See not-in-single-column-unit-tests.sql for an introduction. +-- This file has the same test cases as not-in-unit-tests-multi-column.sql with literals instead of +-- subqueries. Small changes have been made to the literals to make them typecheck. + +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false +--ONLY_IF spark + +CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES + (null, null), + (null, 1.0), + (2, 3.0), + (4, 5.0) + AS m(a, b); + +-- Case 1 (not possible to write a literal with no rows, so we ignore it.) +-- (subquery is empty -> row is returned) + +-- Cases 2, 3 and 4 are currently broken, so I have commented them out here. +-- Filed https://issues.apache.org/jira/browse/SPARK-24395 to fix and restore these test cases. + + -- Case 5 + -- (one null column with no match -> row is returned) +SELECT * +FROM m +WHERE b = 1.0 -- Matches (null, 1.0) + AND (a, b) NOT IN ((2, 3.0)); + + -- Case 6 + -- (no null columns with match -> row not returned) +SELECT * +FROM m +WHERE b = 3.0 -- Matches (2, 3.0) + AND (a, b) NOT IN ((2, 3.0)); + + -- Case 7 + -- (no null columns with no match -> row is returned) +SELECT * +FROM m +WHERE b = 5.0 -- Matches (4, 5.0) + AND (a, b) NOT IN ((2, 3.0)); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column.sql new file mode 100644 index 000000000000..d5a30950b7fe --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-multi-column.sql @@ -0,0 +1,102 @@ +-- Unit tests for simple NOT IN predicate subquery across multiple columns. +-- +-- See not-in-single-column-unit-tests.sql for an introduction. +-- +-- Test cases for multi-column ``WHERE a NOT IN (SELECT c FROM r ...)'': +-- | # | does subquery include null? | do filter columns contain null? | a = c? | b = d? | row included in result? | +-- | 1 | empty | * | * | * | yes | +-- | 2 | 1+ row has null for all columns | * | * | * | no | +-- | 3 | no row has null for all columns | (yes, yes) | * | * | no | +-- | 4 | no row has null for all columns | (no, yes) | yes | * | no | +-- | 5 | no row has null for all columns | (no, yes) | no | * | yes | +-- | 6 | no | (no, no) | yes | yes | no | +-- | 7 | no | (no, no) | _ | _ | yes | +-- +-- This can be generalized to include more tests for more columns, but it covers the main cases +-- when there is more than one column. + +--ONLY_IF spark +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false + +CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES + (null, null), + (null, 1.0), + (2, 3.0), + (4, 5.0) + AS m(a, b); + +CREATE TEMPORARY VIEW s AS SELECT * FROM VALUES + (null, null), + (0, 1.0), + (2, 3.0), + (4, null) + AS s(c, d); + + -- Case 1 + -- (subquery is empty -> row is returned) +SELECT * +FROM m +WHERE (a, b) NOT IN (SELECT * + FROM s + WHERE d > 5.0) -- Matches no rows +; + + -- Case 2 + -- (subquery contains a row with null in all columns -> row not returned) +SELECT * +FROM m +WHERE (a, b) NOT IN (SELECT * + FROM s + WHERE c IS NULL AND d IS NULL) -- Matches only (null, null) +; + + -- Case 3 + -- (probe-side columns are all null -> row not returned) +SELECT * +FROM m +WHERE a IS NULL AND b IS NULL -- Matches only (null, null) + AND (a, b) NOT IN (SELECT * + FROM s + WHERE c IS NOT NULL) -- Matches (0, 1.0), (2, 3.0), (4, null) +; + + -- Case 4 + -- (one column null, other column matches a row in the subquery result -> row not returned) +SELECT * +FROM m +WHERE b = 1.0 -- Matches (null, 1.0) + AND (a, b) NOT IN (SELECT * + FROM s + WHERE c IS NOT NULL) -- Matches (0, 1.0), (2, 3.0), (4, null) +; + + -- Case 5 + -- (one null column with no match -> row is returned) +SELECT * +FROM m +WHERE b = 1.0 -- Matches (null, 1.0) + AND (a, b) NOT IN (SELECT * + FROM s + WHERE c = 2) -- Matches (2, 3.0) +; + + -- Case 6 + -- (no null columns with match -> row not returned) +SELECT * +FROM m +WHERE b = 3.0 -- Matches (2, 3.0) + AND (a, b) NOT IN (SELECT * + FROM s + WHERE c = 2) -- Matches (2, 3.0) +; + + -- Case 7 + -- (no null columns with no match -> row is returned) +SELECT * +FROM m +WHERE b = 5.0 -- Matches (4, 5.0) + AND (a, b) NOT IN (SELECT * + FROM s + WHERE c = 2) -- Matches (2, 3.0) +; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql new file mode 100644 index 000000000000..2c3aed95a13b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql @@ -0,0 +1,46 @@ +-- Unit tests for simple NOT IN with a literal expression of a single column +-- +-- More information can be found in not-in-unit-tests-single-column.sql. +-- This file has the same test cases as not-in-unit-tests-single-column.sql with literals instead of +-- subqueries. + +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false +--ONLY_IF spark + +CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES + (null, 1.0), + (2, 3.0), + (4, 5.0) + AS m(a, b); + + -- Uncorrelated NOT IN Subquery test cases + -- Case 1 (not possible to write a literal with no rows, so we ignore it.) + -- (empty subquery -> all rows returned) + + -- Case 2 + -- (subquery includes null -> no rows returned) +SELECT * +FROM m +WHERE a NOT IN (null); + + -- Case 3 + -- (probe column is null -> row not returned) +SELECT * +FROM m +WHERE b = 1.0 -- Only matches (null, 1.0) + AND a NOT IN (2); + + -- Case 4 + -- (probe column matches subquery row -> row not returned) +SELECT * +FROM m +WHERE b = 3.0 -- Only matches (2, 3.0) + AND a NOT IN (2); + + -- Case 5 + -- (probe column does not match subquery row -> row is returned) +SELECT * +FROM m +WHERE b = 3.0 -- Only matches (2, 3.0) + AND a NOT IN (6); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column.sql new file mode 100644 index 000000000000..bbf5dce38a96 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/not-in-unit-tests-single-column.sql @@ -0,0 +1,127 @@ +-- Unit tests for simple NOT IN predicate subquery across a single column. +-- +-- ``col NOT IN expr'' is quite difficult to reason about. There are many edge cases, some of the +-- rules are confusing to the uninitiated, and precedence and treatment of null values is plain +-- unintuitive. To make this simpler to understand, I've come up with a plain English way of +-- describing the expected behavior of this query. +-- +-- - If the subquery is empty (i.e. returns no rows), the row should be returned, regardless of +-- whether the filtered columns include nulls. +-- - If the subquery contains a result with all columns null, then the row should not be returned. +-- - If for all non-null filter columns there exists a row in the subquery in which each column +-- either +-- 1. is equal to the corresponding filter column or +-- 2. is null +-- then the row should not be returned. (This includes the case where all filter columns are +-- null.) +-- - Otherwise, the row should be returned. +-- +-- Using these rules, we can come up with a set of test cases for single-column and multi-column +-- NOT IN test cases. +-- +-- Test cases for single-column ``WHERE a NOT IN (SELECT c FROM r ...)'': +-- | # | does subquery include null? | is a null? | a = c? | row with a included in result? | +-- | 1 | empty | | | yes | +-- | 2 | yes | | | no | +-- | 3 | no | yes | | no | +-- | 4 | no | no | yes | no | +-- | 5 | no | no | no | yes | +-- +-- There are also some considerations around correlated subqueries. Correlated subqueries can +-- cause cases 2, 3, or 4 to be reduced to case 1 by limiting the number of rows returned by the +-- subquery, so the row from the parent table should always be included in the output. + +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false +--ONLY_IF spark + +CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES + (null, 1.0), + (2, 3.0), + (4, 5.0) + AS m(a, b); + +CREATE TEMPORARY VIEW s AS SELECT * FROM VALUES + (null, 1.0), + (2, 3.0), + (6, 7.0) + AS s(c, d); + + -- Uncorrelated NOT IN Subquery test cases + -- Case 1 + -- (empty subquery -> all rows returned) +SELECT * +FROM m +WHERE a NOT IN (SELECT c + FROM s + WHERE d > 10.0) -- (empty subquery) +; + + -- Case 2 + -- (subquery includes null -> no rows returned) +SELECT * +FROM m +WHERE a NOT IN (SELECT c + FROM s + WHERE d = 1.0) -- Only matches (null, 1.0) +; + + -- Case 3 + -- (probe column is null -> row not returned) +SELECT * +FROM m +WHERE b = 1.0 -- Only matches (null, 1.0) + AND a NOT IN (SELECT c + FROM s + WHERE d = 3.0) -- Matches (2, 3.0) +; + + -- Case 4 + -- (probe column matches subquery row -> row not returned) +SELECT * +FROM m +WHERE b = 3.0 -- Only matches (2, 3.0) + AND a NOT IN (SELECT c + FROM s + WHERE d = 3.0) -- Matches (2, 3.0) +; + + -- Case 5 + -- (probe column does not match subquery row -> row is returned) +SELECT * +FROM m +WHERE b = 3.0 -- Only matches (2, 3.0) + AND a NOT IN (SELECT c + FROM s + WHERE d = 7.0) -- Matches (6, 7.0) +; + + -- Correlated NOT IN subquery test cases + -- Case 2->1 + -- (subquery had nulls but they are removed by correlated subquery -> all rows returned) +SELECT * +FROM m +WHERE a NOT IN (SELECT c + FROM s + WHERE d = b + 10) -- Matches no row +; + + -- Case 3->1 + -- (probe column is null but subquery returns no rows -> row is returned) +SELECT * +FROM m +WHERE b = 1.0 -- Only matches (null, 1.0) + AND a NOT IN (SELECT c + FROM s + WHERE d = b + 10) -- Matches no row +; + + -- Case 4->1 + -- (probe column matches row which is filtered out by correlated subquery -> row is returned) +SELECT * +FROM m +WHERE b = 3.0 -- Only matches (2, 3.0) + AND a NOT IN (SELECT c + FROM s + WHERE d = b + 10) -- Matches no row +; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/simple-in.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/simple-in.sql new file mode 100644 index 000000000000..98f1f3d52cbb --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/in-subquery/simple-in.sql @@ -0,0 +1,140 @@ +-- A test suite for simple IN predicate subquery +-- It includes correlated cases. + +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=true +--CONFIG_DIM1 spark.sql.optimizeNullAwareAntiJoin=false +--ONLY_IF spark + +create temporary view t1 as select * from values + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("t1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("t1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ("t2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("t1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("t2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("t1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("t1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ("t3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("t3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("t3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("t3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("t1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("t3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- correlated IN subquery +-- simple select +-- TC 01.01 +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2); + +-- TC 01.02 +SELECT * +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a); + +-- TC 01.03 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t1a != t2a); + +-- TC 01.04 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t1a = t2a + OR t1b > t2b); + +-- TC 01.05 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t2i IN (SELECT t3i + FROM t3 + WHERE t2c = t3c)); + +-- TC 01.06 +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t2a IN (SELECT t3a + FROM t3 + WHERE t2c = t3c + AND t2b IS NOT NULL)); + +-- simple select for NOT IN +-- TC 01.07 +SELECT DISTINCT( t1a ), + t1b, + t1h +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2); + +-- DDLs +create temporary view a as select * from values + (1, 1), (2, 1), (null, 1), (1, 3), (null, 3), (1, null), (null, 2) + as a(a1, a2); + +create temporary view b as select * from values + (1, 1, 2), (null, 3, 2), (1, null, 2), (1, 2, null) + as b(b1, b2, b3); + +-- TC 02.01 +SELECT a1, a2 +FROM a +WHERE a1 NOT IN (SELECT b.b1 + FROM b + WHERE a.a2 = b.b2) +; + +-- TC 02.02 +SELECT a1, a2 +FROM a +WHERE a1 NOT IN (SELECT b.b1 + FROM b + WHERE a.a2 = b.b2 + AND b.b3 > 1) +; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/negative-cases/invalid-correlation.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/negative-cases/invalid-correlation.sql new file mode 100644 index 000000000000..0b83ecf43a30 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/negative-cases/invalid-correlation.sql @@ -0,0 +1,83 @@ +-- The test file contains negative test cases +-- of invalid queries where error messages are expected. +--ONLY_IF spark + +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES + (1, 2, 3) +AS t1(t1a, t1b, t1c); + +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES + (1, 0, 1) +AS t2(t2a, t2b, t2c); + +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES + (3, 1, 2) +AS t3(t3a, t3b, t3c); + +-- TC 01.01 +-- The column t2b in the SELECT of the subquery is invalid +-- because it is neither an aggregate function nor a GROUP BY column. +SELECT t1a, t2b +FROM t1, t2 +WHERE t1b = t2c +AND t2b = (SELECT max(avg) + FROM (SELECT t2b, avg(t2b) avg + FROM t2 + WHERE t2a = t1.t1b + ) + ) +; + +-- TC 01.02 +-- Invalid due to the column t2b not part of the output from table t2. +SELECT * +FROM t1 +WHERE t1a IN (SELECT min(t2a) + FROM t2 + GROUP BY t2c + HAVING t2c IN (SELECT max(t3c) + FROM t3 + GROUP BY t3b + HAVING t3b > t2b )) +; + +-- TC 01.03 +-- Invalid due to mixure of outer and local references under an AggegatedExpression +-- in a correlated predicate +SELECT t1a +FROM t1 +GROUP BY 1 +HAVING EXISTS (SELECT t2a + FROM t2 + GROUP BY 1 + HAVING t2a < min(t1a + t2a)); + +-- TC 01.04 +-- Invalid due to mixure of outer and local references under an AggegatedExpression +SELECT t1a +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE EXISTS (SELECT 1 + FROM t3 + GROUP BY 1 + HAVING min(t2a + t3a) > 1)); + +-- TC 01.05 +-- Invalid due to outer reference appearing in projection list +SELECT t1a +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE EXISTS (SELECT min(t2a) + FROM t3)); + +CREATE TEMPORARY VIEW t1_copy AS SELECT * FROM VALUES + (1, 2, 3) +AS t1(t1a, t1b, t1c); + +-- invalid because column name `t1a` is ambiguous in the subquery. +SELECT t1.t1a +FROM t1 +JOIN t1_copy +ON EXISTS (SELECT 1 FROM t2 WHERE t2a > t1a) diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/negative-cases/subq-input-typecheck.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/negative-cases/subq-input-typecheck.sql new file mode 100644 index 000000000000..da3ed9d11a8b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/negative-cases/subq-input-typecheck.sql @@ -0,0 +1,62 @@ +-- The test file contains negative test cases +-- of invalid queries where error messages are expected. +--ONLY_IF spark + +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES + (1, 2, 3) +AS t1(t1a, t1b, t1c); + +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES + (1, 0, 1) +AS t2(t2a, t2b, t2c); + +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES + (3, 1, 2) +AS t3(t3a, t3b, t3c); + +CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES + (CAST(1 AS DOUBLE), CAST(2 AS STRING), CAST(3 AS STRING)) +AS t1(t4a, t4b, t4c); + +CREATE TEMPORARY VIEW t5 AS SELECT * FROM VALUES + (CAST('2011-01-01 01:01:01' AS TIMESTAMP), CAST(2 AS STRING), CAST(3 AS BIGINT)) +AS t1(t5a, t5b, t5c); + +-- TC 01.01 +SELECT + ( SELECT max(t2b), min(t2b) + FROM t2 + WHERE t2.t2b = t1.t1b + GROUP BY t2.t2b + ) +FROM t1; + +-- TC 01.01 +SELECT + ( SELECT max(t2b), min(t2b) + FROM t2 + WHERE t2.t2b > 0 + GROUP BY t2.t2b + ) +FROM t1; + +-- TC 01.03 +SELECT * FROM t1 +WHERE +t1a IN (SELECT t2a, t2b + FROM t2 + WHERE t1a = t2a); + +-- TC 01.04 +SELECT * FROM T1 +WHERE +(t1a, t1b) IN (SELECT t2a + FROM t2 + WHERE t1a = t2a); +-- TC 01.05 +SELECT * FROM t4 +WHERE +(t4a, t4b, t4c) IN (SELECT t5a, + t5b, + t5c + FROM t5); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/scalar-subquery/nested-scalar-subquery-count-bug.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/scalar-subquery/nested-scalar-subquery-count-bug.sql new file mode 100644 index 000000000000..fe9152a27fe8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/scalar-subquery/nested-scalar-subquery-count-bug.sql @@ -0,0 +1,35 @@ +--ONLY_IF spark +CREATE OR REPLACE VIEW t1(a1, a2) as values (0, 1), (1, 2); +CREATE OR REPLACE VIEW t2(b1, b2) as values (0, 2), (0, 3); +CREATE OR REPLACE VIEW t3(c1, c2) as values (0, 2), (0, 3); + +set spark.sql.optimizer.decorrelateInnerQuery.enabled=true; +set spark.sql.legacy.scalarSubqueryCountBugBehavior=false; + +-- test for count bug in nested aggregates in correlated scalar subqueries +select ( select sum(cnt) from (select count(*) cnt from t2 where t1.a1 = t2.b1) ) a from t1 order by a desc; + +-- test for count bug in nested counts in correlated scalar subqueries +select ( select count(*) from (select count(*) cnt from t2 where t1.a1 = t2.b1) ) a from t1 order by a desc; + +-- test for count bug in correlated scalar subqueries with nested aggregates with multiple counts +select ( + select SUM(l.cnt + r.cnt) + from (select count(*) cnt from t2 where t1.a1 = t2.b1 having cnt = 0) l + join (select count(*) cnt from t3 where t1.a1 = t3.c1 having cnt = 0) r + on l.cnt = r.cnt +) a from t1 order by a desc; + +-- same as above, without HAVING clause +select ( + select sum(l.cnt + r.cnt) + from (select count(*) cnt from t2 where t1.a1 = t2.b1) l + join (select count(*) cnt from t3 where t1.a1 = t3.c1) r + on l.cnt = r.cnt +) a from t1 order by a desc; + +reset spark.sql.optimizer.decorrelateInnerQuery.enabled; +reset spark.sql.legacy.scalarSubqueryCountBugBehavior; +DROP VIEW t1; +DROP VIEW t2; +DROP VIEW t3; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-count-bug.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-count-bug.sql new file mode 100644 index 000000000000..b2e4bea480da --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-count-bug.sql @@ -0,0 +1,160 @@ +--CONFIG_DIM1 spark.sql.optimizer.decorrelateInnerQuery.enabled=true +--CONFIG_DIM1 spark.sql.optimizer.decorrelateInnerQuery.enabled=false + +--ONLY_IF spark +create temp view l (a, b) +as values + (1, 2.0), + (1, 2.0), + (2, 1.0), + (2, 1.0), + (3, 3.0), + (null, null), + (null, 5.0), + (6, null); + +create temp view r (c, d) +as values + (2, 3.0), + (2, 3.0), + (3, 2.0), + (4, 1.0), + (null, null), + (null, 5.0), + (6, null); + +-- count bug, empty groups should evaluate to 0 +select *, (select count(*) from r where l.a = r.c) from l; + +-- no count bug, empty groups should evaluate to null +select *, (select count(*) from r where l.a = r.c group by c) from l; +select *, (select count(*) from r where l.a = r.c group by 'constant') from l; + +-- count bug, empty groups should evaluate to false +select *, ( + select (count(*)) is null + from r + where l.a = r.c) +from l; + +-- no count bug, empty groups should evaluate to null +select *, ( + select (count(*)) is null + from r + where l.a = r.c + group by r.c) +from l; + +-- Empty groups should evaluate to 0, and groups filtered by HAVING should evaluate to NULL +select *, (select count(*) from r where l.a = r.c having count(*) <= 1) from l; + +-- Empty groups are filtered by HAVING and should evaluate to null +select *, (select count(*) from r where l.a = r.c having count(*) >= 2) from l; + + +CREATE TEMPORARY VIEW null_view(a, b) AS SELECT CAST(null AS int), CAST(null as int); + +-- SPARK-46743: count bug is still detected on top of the subquery that can be constant folded. +SELECT + ( + SELECT + COUNT(null_view.a) AS result + FROM + null_view + WHERE + null_view.a = l.a + ) +FROM + l; + + +-- Same as above but with a filter (HAVING) above the aggregate +SELECT + ( + SELECT + COUNT(null_view.a) AS result + FROM + null_view + WHERE + null_view.a = l.a + having count(*) > -1 + ) +FROM + l; + +-- Count bug over a subquery with an empty relation after optimization. +SELECT + ( + SELECT + COUNT(null_view.a) AS result + FROM + null_view + INNER JOIN r + ON r.c = null_view.a + AND r.c IS NOT NULL + WHERE + null_view.a = l.a + ) +FROM + l; + +-- Same as above but with a filter (HAVING) above the aggregate +SELECT +( + SELECT + COUNT(null_view.a) AS result + FROM + null_view + INNER JOIN r + ON r.c = null_view.a + AND r.c IS NOT NULL + WHERE + null_view.a = l.a + HAVING COUNT(*) > -1 +) +FROM + l; + +-- Same as above but with intersect +SELECT + ( + SELECT + COUNT(f.a) AS result + FROM + ( + SELECT a, b FROM null_view + INTERSECT + SELECT c, d FROM r WHERE c IS NOT NULL + ) AS f + WHERE + f.a = l.a + ) +FROM + l; + +SELECT +( + SELECT + COUNT(f.a) AS result + FROM + ( + SELECT a, b FROM null_view + INTERSECT + SELECT c, d FROM r WHERE c IS NOT NULL + ) AS f + WHERE + f.a = l.a + HAVING COUNT(*) > -1 +) +FROM + l; + + +set spark.sql.optimizer.decorrelateSubqueryLegacyIncorrectCountHandling.enabled = true; + +-- With legacy behavior flag set, both cases evaluate to 0 +select *, (select count(*) from r where l.a = r.c) from l; +select *, (select count(*) from r where l.a = r.c group by c) from l; +select *, (select count(*) from r where l.a = r.c group by 'constant') from l; + +reset spark.sql.optimizer.decorrelateSubqueryLegacyIncorrectCountHandling.enabled; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-group-by.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-group-by.sql new file mode 100644 index 000000000000..a23083e9e0e4 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-group-by.sql @@ -0,0 +1,46 @@ +-- Tests for scalar subquery with a group-by. Only a group-by that guarantees a single row result is allowed. See SPARK-48503 + +--ONLY_IF spark + +create temp view x (x1, x2) as values (1, 1), (2, 2); +create temp view y (y1, y2) as values (2, 0), (3, -1); +create temp view z (z1, z2) as values (1, 0), (1, 1); + +-- Legal queries +select * from x where (select count(*) from y where y1 = x1 group by y1) = 1; +select * from x where (select count(*) from y where y1 = x1 group by x1) = 1; +select * from x where (select count(*) from y where y1 > x1 group by x1) = 1; + +-- Group-by column equal to constant - legal +select *, (select count(*) from y where x1 = y1 and y2 = 1 group by y2) from x; +-- Group-by column equal to expression with constants and outer refs - legal +select *, (select count(*) from y where x1 = y1 and y2 = x1 + 1 group by y2) from x; +-- Group-by expression is the same as the one we filter on - legal +select *, (select count(*) from y where x1 = y1 and cast(y2 as double) = x1 + 1 + group by cast(y2 as double)) from x; +-- Group-by expression equal to an expression that depends on 2 outer refs -- legal +select *, (select count(*) from y where y2 + 1 = x1 + x2 group by y2 + 1) from x; + + +-- Illegal queries (single join disabled) +set spark.sql.optimizer.scalarSubqueryUseSingleJoin = false; +select * from x where (select count(*) from y where y1 > x1 group by y1) = 1; +select *, (select count(*) from y where y1 + y2 = x1 group by y1) from x; +select *, (select count(*) from y where x1 = y1 and y2 + 10 = x1 + 1 group by y2) from x; + +-- Same queries, with LeftSingle join +set spark.sql.optimizer.scalarSubqueryUseSingleJoin = true; +select * from x where (select count(*) from y where y1 > x1 group by y1) = 1; +select *, (select count(*) from y where y1 + y2 = x1 group by y1) from x; +select *, (select count(*) from y where x1 = y1 and y2 + 10 = x1 + 1 group by y2) from x; + + +-- Certain other operators like OUTER JOIN or UNION between the correlating filter and the group-by also can cause the scalar subquery to return multiple values and hence make the query illegal. +select *, (select count(*) from (select * from y where y1 = x1 union all select * from y) sub group by y1) from x; +select *, (select count(*) from y left join (select * from z where z1 = x1) sub on y2 = z2 group by z1) from x; -- The correlation below the join is unsupported in Spark anyway, but when we do support it this query should still be disallowed. + +-- Test legacy behavior conf +set spark.sql.legacy.scalarSubqueryAllowGroupByNonEqualityCorrelatedPredicate = true; +set spark.sql.optimizer.scalarSubqueryUseSingleJoin = false; +select * from x where (select count(*) from y where y1 > x1 group by y1) = 1; +reset spark.sql.legacy.scalarSubqueryAllowGroupByNonEqualityCorrelatedPredicate; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-predicate.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-predicate.sql new file mode 100644 index 000000000000..640ed52cff41 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-predicate.sql @@ -0,0 +1,545 @@ +-- A test suite for scalar subquery in predicate context + +--ONLY_IF spark +CREATE OR REPLACE TEMPORARY VIEW p AS VALUES (1, 1) AS T(pk, pv); +CREATE OR REPLACE TEMPORARY VIEW c AS VALUES (1, 1) AS T(ck, cv); + +-- SPARK-18814.1: Simplified version of TPCDS-Q32 +SELECT pk, cv +FROM p, c +WHERE p.pk = c.ck +AND c.cv = (SELECT avg(c1.cv) + FROM c c1 + WHERE c1.ck = p.pk); + +-- SPARK-18814.2: Adding stack of aggregates +SELECT pk, cv +FROM p, c +WHERE p.pk = c.ck +AND c.cv = (SELECT max(avg) + FROM (SELECT c1.cv, avg(c1.cv) avg + FROM c c1 + WHERE c1.ck = p.pk + GROUP BY c1.cv)); + +create temporary view t1 as select * from values + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), + ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1a', 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ('val1a', 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ('val1d', null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ('val1d', null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ('val1e', 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ('val1d', 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ('val2a', 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1c', 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ('val1b', null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ('val2e', 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1f', 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ('val1e', 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ('val1f', 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ('val1b', null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ('val3a', 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ('val3a', 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ('val3c', 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ('val3c', 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ('val1b', null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ('val1b', null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ('val3b', 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val3b', 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- Group 1: scalar subquery in predicate context +-- no correlation +-- TC 01.01 +SELECT t1a, t1b +FROM t1 +WHERE t1c = (SELECT max(t2c) + FROM t2); + +-- TC 01.02 +SELECT t1a, t1d, t1f +FROM t1 +WHERE t1c = (SELECT max(t2c) + FROM t2) +AND t1b > (SELECT min(t3b) + FROM t3); + +-- TC 01.03 +SELECT t1a, t1h +FROM t1 +WHERE t1c = (SELECT max(t2c) + FROM t2) +OR t1b = (SELECT min(t3b) + FROM t3 + WHERE t3b > 10); + +-- TC 01.04 +-- scalar subquery over outer join +SELECT t1a, t1b, t2d +FROM t1 LEFT JOIN t2 + ON t1a = t2a +WHERE t1b = (SELECT min(t3b) + FROM t3); + +-- TC 01.05 +-- test casting +SELECT t1a, t1b, t1g +FROM t1 +WHERE t1c + 5 = (SELECT max(t2e) + FROM t2); + +-- TC 01.06 +-- test casting +SELECT t1a, t1h +FROM t1 +WHERE date(t1h) = (SELECT min(t2i) + FROM t2); + +-- TC 01.07 +-- same table, expressions in scalar subquery +SELECT t2d, t1a +FROM t1, t2 +WHERE t1b = t2b +AND t2c + 1 = (SELECT max(t2c) + 1 + FROM t2, t1 + WHERE t2b = t1b); + +-- TC 01.08 +-- same table +SELECT DISTINCT t2a, max_t1g +FROM t2, (SELECT max(t1g) max_t1g, t1a + FROM t1 + GROUP BY t1a) t1 +WHERE t2a = t1a +AND max_t1g = (SELECT max(t1g) + FROM t1); + +-- TC 01.09 +-- more than one scalar subquery +SELECT t3b, t3c +FROM t3 +WHERE (SELECT max(t3c) + FROM t3 + WHERE t3b > 10) >= + (SELECT min(t3b) + FROM t3 + WHERE t3c > 0) +AND (t3b is null or t3c is null); + +-- Group 2: scalar subquery in predicate context +-- with correlation +-- TC 02.01 +SELECT t1a +FROM t1 +WHERE t1a < (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c); + +-- TC 02.02 +SELECT t1a, t1c +FROM t1 +WHERE (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) IS NULL; + +-- TC 02.03 +SELECT t1a +FROM t1 +WHERE t1a = (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c + HAVING count(*) >= 0) +OR t1i > '2014-12-31'; + +-- TC 02.03.01 +SELECT t1a +FROM t1 +WHERE t1a = (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c + HAVING count(*) >= 1) +OR t1i > '2014-12-31'; + +-- TC 02.04 +-- t1 on the right of an outer join +-- can be reduced to inner join +SELECT count(t1a) +FROM t1 RIGHT JOIN t2 +ON t1d = t2d +WHERE t1a < (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c); + +-- TC 02.05 +SELECT t1a +FROM t1 +WHERE t1b <= (SELECT max(t2b) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +AND t1b >= (SELECT min(t2b) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c); + +-- TC 02.06 +-- set op +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +INTERSECT +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c); + +-- TC 02.07.01 +-- set op +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +UNION ALL +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c); + +-- TC 02.07.02 +-- set op +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +UNION DISTINCT +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c); + +-- TC 02.08 +-- set op +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +MINUS +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c); + +-- TC 02.09 +-- in HAVING clause +SELECT t1a +FROM t1 +GROUP BY t1a, t1c +HAVING max(t1b) <= (SELECT max(t2b) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c); + +-- SPARK-44549: window function in the correlated subquery. +SELECT 1 +FROM t1 +WHERE t1b < (SELECT MAX(tmp.s) FROM ( + SELECT SUM(t2b) OVER (partition by t2c order by t2d) as s + FROM t2 WHERE t2.t2d = t1.t1d) as tmp); + +-- Same as above but with LIMIT/ORDER BY instead of MAX +SELECT 1 +FROM t1 +WHERE t1b < (SELECT SUM(t2b) OVER (partition by t2c order by t2d) as s + FROM t2 WHERE t2.t2d = t1.t1d + ORDER BY s DESC + LIMIT 1); + +-- SPARK-44549: window function in the correlated subquery with non-equi predicate. +SELECT 1 +FROM t1 +WHERE t1b < (SELECT MAX(tmp.s) FROM ( + SELECT SUM(t2b) OVER (partition by t2c order by t2d) as s + FROM t2 WHERE t2.t2d <= t1.t1d) as tmp); + +-- Same as above but with LIMIT/ORDER BY +SELECT 1 +FROM t1 +WHERE t1b < (SELECT SUM(t2b) OVER (partition by t2c order by t2d) as s + FROM t2 WHERE t2.t2d <= t1.t1d + ORDER BY s DESC + LIMIT 1); + +-- SPARK-44549: window function in the correlated subquery over joins. +SELECT t1b +FROM t1 +WHERE t1b > (SELECT MAX(tmp.s) FROM ( + SELECT RANK() OVER (partition by t3c, t2b order by t3c) as s + FROM t2, t3 where t2.t2c = t3.t3c AND t2.t2a = t1.t1a) as tmp); + +-- SPARK-44549: window function in the correlated subquery over aggregation. +SELECT t1b +FROM t1 +WHERE t1b > (SELECT MAX(tmp.s) FROM ( + SELECT RANK() OVER (partition by t3c, t3d order by t3c) as s + FROM (SELECT t3b, t3c, max(t3d) as t3d FROM t3 GROUP BY t3b, t3c) as g) as tmp) +ORDER BY t1b; + + +-- SPARK-44549: correlation in window function itself is not supported yet. +SELECT 1 +FROM t1 +WHERE t1b = (SELECT MAX(tmp.s) FROM ( + SELECT SUM(t2c) OVER (partition by t2c order by t1.t1d + t2d) as s + FROM t2) as tmp); + +-- SPARK-36191: ORDER BY/LIMIT in the correlated subquery, equi-predicate +SELECT t1a, t1b +FROM t1 +WHERE t1c = (SELECT t2c + FROM t2 + WHERE t2b < t1b + ORDER BY t2d LIMIT 1); + + +-- SPARK-36191: ORDER BY/LIMIT in the correlated subquery, non-equi-predicate +SELECT t1a, t1b +FROM t1 +WHERE t1c = (SELECT t2c + FROM t2 + WHERE t2c = t1c + ORDER BY t2c LIMIT 1); + +-- SPARK-46526: LIMIT over correlated predicate that references only the outer table. +SELECT t1a, t1b +FROM t1 +WHERE t1c = (SELECT t2c + FROM t2 + WHERE t1b < t1d + ORDER BY t2c LIMIT 1); + +SELECT t1a, t1b +FROM t1 +WHERE t1c = (SELECT MAX(t2c) + FROM t2 + WHERE t1b < t1d + ORDER BY min(t2c) LIMIT 1); + +SELECT t1a, t1b +FROM t1 +WHERE t1c = (SELECT DISTINCT t2c + FROM t2 + WHERE t1b < t1d + ORDER BY t2c LIMIT 1); + +-- Set operations in correlation path + +CREATE OR REPLACE TEMP VIEW t0(t0a, t0b) AS VALUES (1, 1), (2, 0); +CREATE OR REPLACE TEMP VIEW t1(t1a, t1b, t1c) AS VALUES (1, 1, 3); +CREATE OR REPLACE TEMP VIEW t2(t2a, t2b, t2c) AS VALUES (1, 1, 5), (2, 2, 7); + +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +); + +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +); + +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +); + +SELECT * FROM t0 WHERE t0a < +(SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +); + +SELECT * FROM t0 WHERE t0a < +(SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +); + +-- Tests for column aliasing +SELECT * FROM t0 WHERE t0a < +(SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +); + +-- Test handling of COUNT bug +SELECT * FROM t0 WHERE t0a < +(SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +); + +-- Correlated references in project +SELECT * FROM t0 WHERE t0a < +(SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + UNION ALL + SELECT t2a - t0a as d + FROM t2) +); + +-- Correlated references in aggregate - unsupported +SELECT * FROM t0 WHERE t0a < +(SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + UNION ALL + SELECT sum(t2a) + t0a as d + FROM t2) +); + +-- In HAVING clause +SELECT t0a, t0b FROM t0 +GROUP BY t0a, t0b +HAVING t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +); + +-- SPARK-43760: the result of the subquery can be NULL. +select * +from range(1, 3) t1 +where (select t2.id c + from range (1, 2) t2 where t1.id = t2.id + ) is not null; + +-- Correlated references in join predicates +SELECT * FROM t0 WHERE t0a < +(SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 JOIN t2 ON (t1a = t0a AND t2b = t1b)) +); + +SELECT * FROM t0 WHERE t0a < +(SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 JOIN t2 ON (t1a < t0a AND t2b >= t1b)) +); + +SELECT * FROM t0 WHERE t0a < +(SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 LEFT JOIN t2 ON (t1a = t0a AND t2b = t0b)) +); + +select * +from range(1, 3) t1 +where (select t2.id c + from range (1, 2) t2 where t1.id = t2.id + ) between 1 and 2; + +SELECT * +FROM t1 +WHERE (SELECT max(t2c) + FROM t2 WHERE t1b = t2b + ) between 1 and 2; + + +SELECT * FROM t0 WHERE t0a = (SELECT distinct(t1c) FROM t1 WHERE t1a = t0a); + +-- SPARK-52531: Inner aggregate expressions are properly decorrelated from outer aggregate expressions +SELECT MAX(a.col1) +FROM VALUES (1) AS a(col1) +GROUP BY a.col1 +HAVING COUNT(*) = ( + SELECT COUNT(*) + FROM VALUES (1),(1),(2),(2) AS c(col1) + WHERE c.col1 >= a.col1 + LIMIT 1 + ); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql new file mode 100644 index 000000000000..ef1e612fd744 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql @@ -0,0 +1,261 @@ +-- A test suite for scalar subquery in SELECT clause + +--ONLY_IF spark +create temporary view t1 as select * from values + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), + ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1a', 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ('val1a', 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ('val1d', null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ('val1d', null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ('val1e', 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ('val1d', 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i); + +create temporary view t2 as select * from values + ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i); + +create temporary view t3 as select * from values + ('val3a', 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ('val3a', 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ('val3c', 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ('val3c', 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ('val1b', null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ('val3b', 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val3b', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i); + +-- Group 1: scalar subquery in SELECT clause +-- no correlation +-- TC 01.01 +-- more than one scalar subquery +SELECT (SELECT min(t3d) FROM t3) min_t3d, + (SELECT max(t2h) FROM t2) max_t2h +FROM t1 +WHERE t1a = 'val1c'; + +-- TC 01.02 +-- scalar subquery in an IN subquery +SELECT t1a, count(*) +FROM t1 +WHERE t1c IN (SELECT (SELECT min(t3c) FROM t3) + FROM t2 + GROUP BY t2g + HAVING count(*) > 1) +GROUP BY t1a; + +-- TC 01.03 +-- under a set op +SELECT (SELECT min(t3d) FROM t3) min_t3d, + null +FROM t1 +WHERE t1a = 'val1c' +UNION +SELECT null, + (SELECT max(t2h) FROM t2) max_t2h +FROM t1 +WHERE t1a = 'val1c'; + +-- TC 01.04 +SELECT (SELECT min(t3c) FROM t3) min_t3d +FROM t1 +WHERE t1a = 'val1a' +INTERSECT +SELECT (SELECT min(t2c) FROM t2) min_t2d +FROM t1 +WHERE t1a = 'val1d'; + +-- TC 01.05 +SELECT q1.t1a, q2.t2a, q1.min_t3d, q2.avg_t3d +FROM (SELECT t1a, (SELECT min(t3d) FROM t3) min_t3d + FROM t1 + WHERE t1a IN ('val1e', 'val1c')) q1 + FULL OUTER JOIN + (SELECT t2a, (SELECT avg(t3d) FROM t3) avg_t3d + FROM t2 + WHERE t2a IN ('val1c', 'val2a')) q2 +ON q1.t1a = q2.t2a +AND q1.min_t3d < q2.avg_t3d; + +-- Group 2: scalar subquery in SELECT clause +-- with correlation +-- TC 02.01 +SELECT (SELECT min(t3d) FROM t3 WHERE t3.t3a = t1.t1a) min_t3d, + (SELECT max(t2h) FROM t2 WHERE t2.t2a = t1.t1a) max_t2h +FROM t1 +WHERE t1a = 'val1b'; + +-- TC 02.02 +SELECT (SELECT min(t3d) FROM t3 WHERE t3a = t1a) min_t3d +FROM t1 +WHERE t1a = 'val1b' +MINUS +SELECT (SELECT min(t3d) FROM t3) abs_min_t3d +FROM t1 +WHERE t1a = 'val1b'; + +-- TC 02.03 +SELECT t1a, t1b +FROM t1 +WHERE NOT EXISTS (SELECT (SELECT max(t2b) + FROM t2 LEFT JOIN t1 + ON t2a = t1a + WHERE t2c = t3c) dummy + FROM t3 + WHERE t3b < (SELECT max(t2b) + FROM t2 LEFT JOIN t1 + ON t2a = t1a + WHERE t2c = t3c) + AND t3a = t1a); + +-- SPARK-34876: Non-nullable aggregates should not return NULL in a correlated subquery +SELECT t1a, + (SELECT count(t2d) FROM t2 WHERE t2a = t1a) count_t2, + (SELECT count_if(t2d > 0) FROM t2 WHERE t2a = t1a) count_if_t2, + (SELECT approx_count_distinct(t2d) FROM t2 WHERE t2a = t1a) approx_count_distinct_t2, + (SELECT collect_list(t2d) FROM t2 WHERE t2a = t1a) collect_list_t2, + (SELECT sort_array(collect_set(t2d)) FROM t2 WHERE t2a = t1a) collect_set_t2, + (SELECT hex(count_min_sketch(t2d, 0.5d, 0.5d, 1)) FROM t2 WHERE t2a = t1a) collect_set_t2 +FROM t1; + +-- SPARK-36028: Allow Project to host outer references in scalar subqueries +SELECT t1c, (SELECT t1c) FROM t1; +SELECT t1c, (SELECT t1c WHERE t1c = 8) FROM t1; +SELECT t1c, t1d, (SELECT c + d FROM (SELECT t1c AS c, t1d AS d)) FROM t1; +SELECT t1c, (SELECT SUM(c) FROM (SELECT t1c AS c)) FROM t1; +SELECT t1a, (SELECT SUM(t2b) FROM t2 JOIN (SELECT t1a AS a) ON t2a = a) FROM t1; + +-- CTE in correlated scalar subqueries +CREATE OR REPLACE TEMPORARY VIEW t1 AS VALUES (0, 1), (1, 2) t1(c1, c2); +CREATE OR REPLACE TEMPORARY VIEW t2 AS VALUES (0, 2), (0, 3) t2(c1, c2); + +-- Single row subquery +SELECT c1, (WITH t AS (SELECT 1 AS a) SELECT a + c1 FROM t) FROM t1; +-- Correlation in CTE. +SELECT c1, (WITH t AS (SELECT * FROM t2 WHERE c1 = t1.c1) SELECT SUM(c2) FROM t) FROM t1; +-- Multiple CTE definitions. +SELECT c1, ( + WITH t3 AS (SELECT c1 + 1 AS c1, c2 + 1 AS c2 FROM t2), + t4 AS (SELECT * FROM t3 WHERE t1.c1 = c1) + SELECT SUM(c2) FROM t4 +) FROM t1; +-- Multiple CTE references. +SELECT c1, ( + WITH t AS (SELECT * FROM t2) + SELECT SUM(c2) FROM (SELECT c1, c2 FROM t UNION SELECT c2, c1 FROM t) r(c1, c2) + WHERE c1 = t1.c1 +) FROM t1; +-- Reference CTE in both the main query and the subquery. +WITH v AS (SELECT * FROM t2) +SELECT * FROM t1 WHERE c1 > ( + WITH t AS (SELECT * FROM t2) + SELECT COUNT(*) FROM v WHERE c1 = t1.c1 AND c1 > (SELECT SUM(c2) FROM t WHERE c1 = v.c1) +); +-- Single row subquery that references CTE in the main query. +WITH t AS (SELECT 1 AS a) +SELECT c1, (SELECT a FROM t WHERE a = c1) FROM t1; +-- Multiple CTE references with non-deterministic CTEs. +WITH +v1 AS (SELECT c1, c2, rand(0) c3 FROM t1), +v2 AS (SELECT c1, c2, rand(0) c4 FROM v1 WHERE c3 IN (SELECT c3 FROM v1)) +SELECT c1, ( + WITH v3 AS (SELECT c1, c2, rand(0) c5 FROM t2) + SELECT COUNT(*) FROM ( + SELECT * FROM v2 WHERE c1 > 0 + UNION SELECT * FROM v2 WHERE c2 > 0 + UNION SELECT * FROM v3 WHERE c2 > 0 + ) WHERE c1 = v1.c1 +) FROM v1; + +-- Multi-value subquery error +SELECT (SELECT a FROM (SELECT 1 AS a UNION ALL SELECT 2 AS a) t) AS b; + +-- SPARK-36114: Support correlated non-equality predicates +CREATE OR REPLACE TEMP VIEW t1(c1, c2) AS (VALUES (0, 1), (1, 2)); +CREATE OR REPLACE TEMP VIEW t2(c1, c2) AS (VALUES (0, 2), (0, 3)); + +-- Neumann example Q2 +CREATE OR REPLACE TEMP VIEW students(id, name, major, year) AS (VALUES + (0, 'A', 'CS', 2022), + (1, 'B', 'CS', 2022), + (2, 'C', 'Math', 2022)); +CREATE OR REPLACE TEMP VIEW exams(sid, course, curriculum, grade, date) AS (VALUES + (0, 'C1', 'CS', 4, 2020), + (0, 'C2', 'CS', 3, 2021), + (1, 'C1', 'CS', 2, 2020), + (1, 'C2', 'CS', 1, 2021)); + +SELECT students.name, exams.course +FROM students, exams +WHERE students.id = exams.sid + AND (students.major = 'CS' OR students.major = 'Games Eng') + AND exams.grade >= ( + SELECT avg(exams.grade) + 1 + FROM exams + WHERE students.id = exams.sid + OR (exams.curriculum = students.major AND students.year > exams.date)); + +-- Correlated non-equality predicates +SELECT (SELECT min(c2) FROM t2 WHERE t1.c1 > t2.c1) FROM t1; +SELECT (SELECT min(c2) FROM t2 WHERE t1.c1 >= t2.c1 AND t1.c2 < t2.c2) FROM t1; + +-- Correlated non-equality predicates with the COUNT bug. +SELECT (SELECT count(*) FROM t2 WHERE t1.c1 > t2.c1) FROM t1; + +-- Correlated equality predicates that are not supported after SPARK-35080 +SELECT c, ( + SELECT count(*) + FROM (VALUES ('ab'), ('abc'), ('bc')) t2(c) + WHERE t1.c = substring(t2.c, 1, 1) +) FROM (VALUES ('a'), ('b')) t1(c); + +SELECT c, ( + SELECT count(*) + FROM (VALUES (0, 6), (1, 5), (2, 4), (3, 3)) t1(a, b) + WHERE a + b = c +) FROM (VALUES (6)) t2(c); + +-- SPARK-43156: scalar subquery with Literal result like `COUNT(1) is null` +SELECT *, (SELECT count(1) is null FROM t2 WHERE t1.c1 = t2.c1) FROM t1; + +select (select f from (select false as f, max(c2) from t1 where t1.c1 = t1.c1)) from t2; + +-- SPARK-43596: handle IsNull when rewriting the domain join +set spark.sql.optimizer.optimizeOneRowRelationSubquery.alwaysInline=false; +WITH T AS (SELECT 1 AS a) +SELECT (SELECT sum(1) FROM T WHERE a = col OR upper(col)= 'Y') +FROM (SELECT null as col) as foo; +set spark.sql.optimizer.optimizeOneRowRelationSubquery.alwaysInline=true; + +-- SPARK-43760: the result of the subquery can be NULL. +select * from ( + select t1.id c1, ( + select t2.id c from range (1, 2) t2 + where t1.id = t2.id ) c2 + from range (1, 3) t1 ) t +where t.c2 is not null; + +-- SPARK-43838: Subquery on single table with having clause +SELECT c1, c2, (SELECT count(*) cnt FROM t1 t2 WHERE t1.c1 = t2.c1 HAVING cnt = 0) FROM t1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-set-op.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-set-op.sql new file mode 100644 index 000000000000..d282e477678c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-set-op.sql @@ -0,0 +1,642 @@ +-- Set operations in correlation path + +--ONLY_IF spark +CREATE OR REPLACE TEMP VIEW t0(t0a, t0b) AS VALUES (1, 1), (2, 0); +CREATE OR REPLACE TEMP VIEW t1(t1a, t1b, t1c) AS VALUES (1, 1, 3); +CREATE OR REPLACE TEMP VIEW t2(t2a, t2b, t2c) AS VALUES (1, 1, 5), (2, 2, 7); + + +-- UNION ALL + +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +); + +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +FROM t0; + +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +FROM t0; + +SELECT t0a, (SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +-- Tests for column aliasing +SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +-- Test handling of COUNT bug +SELECT t0a, (SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +-- Correlated references in project +SELECT t0a, (SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + UNION ALL + SELECT t2a - t0a as d + FROM t2) +) +FROM t0; + +-- Correlated references in aggregate - unsupported +SELECT t0a, (SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + UNION ALL + SELECT sum(t2a) + t0a as d + FROM t2) +) +FROM t0; + + + +-- UNION DISTINCT + +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +); + +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +FROM t0; + +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + UNION DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +FROM t0; + +SELECT t0a, (SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +-- Tests for column aliasing +SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +-- Test handling of COUNT bug +SELECT t0a, (SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +-- Correlated references in project +SELECT t0a, (SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + UNION DISTINCT + SELECT t2a - t0a as d + FROM t2) +) +FROM t0; + +-- Correlated references in aggregate - unsupported +SELECT t0a, (SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + UNION DISTINCT + SELECT sum(t2a) + t0a as d + FROM t2) +) +FROM t0; + + +-- INTERSECT ALL + +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + INTERSECT ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + INTERSECT ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +); + +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + INTERSECT ALL + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +FROM t0; + +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + INTERSECT ALL + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +FROM t0; + +SELECT t0a, (SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + INTERSECT ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +-- Tests for column aliasing +SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + INTERSECT ALL + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +-- Test handling of COUNT bug +SELECT t0a, (SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + INTERSECT ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +-- Correlated references in project +SELECT t0a, (SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + INTERSECT ALL + SELECT t2a - t0a as d + FROM t2) +) +FROM t0; + +-- Correlated references in aggregate - unsupported +SELECT t0a, (SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + INTERSECT ALL + SELECT sum(t2a) + t0a as d + FROM t2) +) +FROM t0; + + + +-- INTERSECT DISTINCT + +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + INTERSECT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + INTERSECT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +); + +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + INTERSECT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +FROM t0; + +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + INTERSECT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +FROM t0; + +SELECT t0a, (SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + INTERSECT DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +-- Tests for column aliasing +SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + INTERSECT DISTINCT + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +-- Test handling of COUNT bug +SELECT t0a, (SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + INTERSECT DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +-- Correlated references in project +SELECT t0a, (SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + INTERSECT DISTINCT + SELECT t2a - t0a as d + FROM t2) +) +FROM t0; + +-- Correlated references in aggregate - unsupported +SELECT t0a, (SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + INTERSECT DISTINCT + SELECT sum(t2a) + t0a as d + FROM t2) +) +FROM t0; + + + +-- EXCEPT ALL + +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + EXCEPT ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + EXCEPT ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +); + +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + EXCEPT ALL + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +FROM t0; + +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + EXCEPT ALL + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +FROM t0; + +SELECT t0a, (SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + EXCEPT ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +-- Tests for column aliasing +SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + EXCEPT ALL + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +-- Test handling of COUNT bug +SELECT t0a, (SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + EXCEPT ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +-- Correlated references in project +SELECT t0a, (SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + EXCEPT ALL + SELECT t2a - t0a as d + FROM t2) +) +FROM t0; + +-- Correlated references in aggregate - unsupported +SELECT t0a, (SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + EXCEPT ALL + SELECT sum(t2a) + t0a as d + FROM t2) +) +FROM t0; + + + +-- EXCEPT DISTINCT + +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + EXCEPT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + EXCEPT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +); + +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + EXCEPT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +FROM t0; + +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + EXCEPT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +FROM t0; + +SELECT t0a, (SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + EXCEPT DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +-- Tests for column aliasing +SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + EXCEPT DISTINCT + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +-- Test handling of COUNT bug +SELECT t0a, (SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + EXCEPT DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0; + +-- Correlated references in project +SELECT t0a, (SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + EXCEPT DISTINCT + SELECT t2a - t0a as d + FROM t2) +) +FROM t0; + +-- Correlated references in aggregate - unsupported +SELECT t0a, (SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + EXCEPT DISTINCT + SELECT sum(t2a) + t0a as d + FROM t2) +) +FROM t0; + +-- Correlated references in join predicates +SELECT t0a, (SELECT sum(t1b) FROM + (SELECT t1b + FROM t1 join t2 ON (t1a = t0a and t1b = t2b) + UNION ALL + SELECT t2b + FROM t1 join t2 ON (t2a = t0a and t1a = t2a)) +) +FROM t0; + + +SELECT t0a, (SELECT sum(t1b) FROM + (SELECT t1b + FROM t1 left join t2 ON (t1a = t0a and t1b = t2b) + UNION ALL + SELECT t2b + FROM t1 join t2 ON (t2a = t0a + 1 and t1a = t2a)) +) +FROM t0; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/subquery-in-from.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/subquery-in-from.sql new file mode 100644 index 000000000000..662a065cdc13 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/subquery-in-from.sql @@ -0,0 +1,15 @@ +--ONLY_IF spark +-- Aliased subqueries in FROM clause +SELECT * FROM (SELECT * FROM testData) AS t WHERE key = 1; + +FROM (SELECT * FROM testData WHERE key = 1) AS t SELECT *; + +-- Optional `AS` keyword +SELECT * FROM (SELECT * FROM testData) t WHERE key = 1; + +FROM (SELECT * FROM testData WHERE key = 1) t SELECT *; + +-- Disallow unaliased subqueries in FROM clause +SELECT * FROM (SELECT * FROM testData) WHERE key = 1; + +FROM (SELECT * FROM testData WHERE key = 1) SELECT *; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/subquery-nested-data.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/subquery-nested-data.sql new file mode 100644 index 000000000000..032c62be72ae --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/subquery-nested-data.sql @@ -0,0 +1,50 @@ +--ONLY_IF spark +drop table if exists x; +drop table if exists y; + +create table x(xm map, x2 int) using parquet; +insert into x values (map(1, 2), 3), (map(1, 4), 5), (map(2, 3), 4), (map(5, 6), 7); +create table y(ym map, y2 int) using parquet; +insert into y values (map(1, 2), 10), (map(1, 3), 20), (map(2, 3), 20), (map(8, 3), 20); + +select * from x where (select sum(y2) from y where xm[1] = ym[1]) > 2; + +-- exists, in, lateral +select * from x where exists (select * from y where xm[1] = ym[1]); +select * from x where exists (select * from y where xm[1] = ym[1] limit 1); +select * from x join lateral (select * from y where xm[1] = ym[1]); +select * from x join lateral (select * from y where xm[1] = ym[1] union all select * from y where xm[1] = ym[1] + 1); +select * from x join lateral (select * from y where xm[1] = ym[1] limit 1); +select * from x join lateral (select count(*) from y where xm[1] = ym[1] group by y2); +select * from x where xm[1] in (select ym[1] from y); +select * from x where xm[1] in (select sum(ym[1]) from y group by y2); + +-- Multiple uses of the same outer expr +select * from x where (select sum(y2) from y where xm[1] = ym[1] and xm[1] >= 1) > 2; + +-- Multiple different extracts from same map +select * from x where (select sum(y2) from y where xm[1] = ym[1] and xm[2] >= ym[2]) > 2; + +-- Multiple subqueries using same outer expr +select * from x where (select sum(y2) from y where xm[1] = ym[1]) > 2 and (select count(y2) from y where xm[1] = ym[1]) < 3; + +-- In project/aggregate/group-by +select * from x join lateral (select xm[1] - ym[1] from y); +select * from x join lateral (select xm[1], xm[1] as s1, xm[1] - ym[1] as s2 from y); +select * from x join lateral (select xm[1], sum(ym[1]), xm[1] - sum(ym[1]) from y group by xm[1]); + +-- Complex key expressions +select * from x where (select sum(y2) from y where xm[x2] = ym[1]) > 2; +select * from x where (select sum(y2) from y where xm[x2+1] = ym[1]) > 2; +select * from x where (select sum(y2) from y where xm[x2+1] = ym[1] and xm[1+x2] = ym[2]) > 2; -- Two key expressions that are semantically equal + +-- Cannot pull out expression because it references both outer and inner +select * from x where (select sum(y2) from y where xm[y2] = ym[1]) > 2; + +-- Unsupported when disabled due to DomainJoin over map +set spark.sql.optimizer.pullOutNestedDataOuterRefExpressions.enabled = false; +select * from x where (select sum(y2) from y where xm[1] = ym[1]) > 2; +reset spark.sql.optimizer.rewriteNestedDataCorrelation.enabled; + +drop table x; +drop table y; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/subquery-offset.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/subquery-offset.sql new file mode 100644 index 000000000000..e421be4f8148 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/subquery/subquery-offset.sql @@ -0,0 +1,80 @@ +--ONLY_IF spark +drop table if exists x; +drop table if exists y; + +create table x(x1 int, x2 int) using json; +insert into x values (1, 1), (2, 2); +create table y(y1 int, y2 int) using json; +insert into y values (1, 1), (1, 2), (2, 4); + +select * from x where exists (select * from y where x1 = y1 limit 1 offset 2); +select * from x join lateral (select * from y where x1 = y1 limit 1 offset 2); +select * from x where x1 in (select y1 from y limit 1 offset 2); +select * from x where (select sum(y2) from y where x1 = y1 limit 1 offset 2) > 2; + +select * from x where exists (select * from y where x1 = y1 offset 2); +select * from x join lateral (select * from y where x1 = y1 offset 2); +select * from x where x1 in (select y1 from y offset 2); +select * from x where (select sum(y2) from y where x1 = y1 offset 2) > 2; + +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id); + +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state); + +SELECT emp_name +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY state + ORDER BY state + LIMIT 2 + OFFSET 1); + +SELECT emp_name +FROM emp +JOIN LATERAL (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY state + ORDER BY state + LIMIT 2 + OFFSET 1); + +SELECT emp_name +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY state + ORDER BY state + OFFSET 1); + +SELECT emp_name +FROM emp +JOIN LATERAL (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY state + ORDER BY state + OFFSET 1); + +drop table x; +drop table y; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/table-aliases.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/table-aliases.sql new file mode 100644 index 000000000000..914bd7a7cdd5 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/table-aliases.sql @@ -0,0 +1,38 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, 1), (1, 2), (2, 1) AS testData(a, b); + +-- Table column aliases in FROM clause +SELECT * FROM testData AS t(col1, col2) WHERE col1 = 1; + +SELECT * FROM testData AS t(col1, col2) WHERE col1 = 2; + +-- Aliasing the wrong number of columns in the FROM clause +SELECT * FROM testData AS t(col1, col2, col3); + +SELECT * FROM testData AS t(col1); + +-- Check alias duplication +SELECT a AS col1, b AS col2 FROM testData AS t(c, d); + +-- Subquery aliases in FROM clause +SELECT * FROM (SELECT 1 AS a, 1 AS b) t(col1, col2); + +SELECT t.* FROM (SELECT 1 AS a, 1 AS b) t(col1, col2); + +SELECT col1, col2 FROM (SELECT 1 AS a, 1 AS b) t(col1, col2); + +SELECT t.col1, t.col2 FROM (SELECT 1 AS a, 1 AS b) t(col1, col2); + +-- Aliases for join relations in FROM clause +CREATE OR REPLACE TEMPORARY VIEW src1 AS SELECT * FROM VALUES (1, "a"), (2, "b"), (3, "c") AS src1(id, v1); + +CREATE OR REPLACE TEMPORARY VIEW src2 AS SELECT * FROM VALUES (2, 1.0), (3, 3.2), (1, 8.5) AS src2(id, v2); + +SELECT * FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d); + +SELECT dst.* FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d); + +-- Negative examples after aliasing +SELECT src1.* FROM src1 a ORDER BY id LIMIT 1; + +SELECT src1.id FROM (SELECT * FROM src1 ORDER BY id LIMIT 1) a; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/table-valued-functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/table-valued-functions.sql new file mode 100644 index 000000000000..9bf41f71d47f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/table-valued-functions.sql @@ -0,0 +1,130 @@ +-- unresolved function +select * from dummy(3); + +-- range call with end +select * from range(6 + cos(3)); + +-- range call with start and end +select * from range(5, 10); + +-- range call with step +select * from range(0, 10, 2); + +-- range call with numPartitions +select * from range(0, 10, 1, 200); + +-- range call with invalid number of arguments +select * from range(1, 1, 1, 1, 1); + +-- range call with null +select * from range(1, null); + +-- range call with incompatible type +select * from range(array(1, 2, 3)); + +-- range call with illegal step +select * from range(0, 5, 0); + +-- range call with a mixed-case function name +select * from RaNgE(2); + +-- range call with alias +select i from range(0, 2) t(i); + +-- range with non-foldable input +select * from range(0, (select 1)); +select * from values (0, 1) t(c1, c2), lateral range(0, c2); + +-- explode +select * from explode(array(1, 2)); +select * from explode(map('a', 1, 'b', 2)); + +-- explode with empty values +select * from explode(array()); +select * from explode(map()); + +-- explode with column aliases +select * from explode(array(1, 2)) t(c1); +select * from explode(map('a', 1, 'b', 2)) t(k, v); + +-- explode with non-deterministic values +select * from explode(array(rand(0))); + +-- explode with erroneous input +select * from explode(null); +select * from explode(null) t(c1); +select * from explode(1); +select * from explode(1, 2); +select * from explode(explode(array(1))); +select * from explode(array(1, 2)) t(c1, c2); + +-- explode_outer +select * from explode_outer(array(1, 2)); +select * from explode_outer(map('a', 1, 'b', 2)); +select * from explode_outer(array()); +select * from explode_outer(map()); + +-- table-valued functions with join +select * from range(2) join explode(array(1, 2)); +select * from range(2) join explode_outer(array()); + +-- inline +select * from inline(array(struct(1, 'a'), struct(2, 'b'))); +select * from inline(array(struct(1, 'a'), struct(2, 'b'))) t(x, y); +select * from inline(array_remove(array(struct(1, 'a')), struct(1, 'a'))); + +-- inline with erroneous input +select * from inline(null); +select * from inline(array(struct(1, 2), struct(2, 3))) t(a, b, c); + +-- inline_outer +select * from inline_outer(array(struct(1, 'a'), struct(2, 'b'))); +select * from inline_outer(array_remove(array(struct(1, 'a')), struct(1, 'a'))); + +-- posexplode +select * from posexplode(array()); +select * from posexplode(array(1, 2)); +select * from posexplode(array(1, 2)) t(pos, x); +select * from posexplode(map()); +select * from posexplode(map('a', 1, 'b', 2)); +select * from posexplode(map('a', 1, 'b', 2)) t(pos, k, v); + +-- posexplode with erroneous input +select * from posexplode(1); +select * from posexplode(1, 2); +select * from posexplode(explode(array(1))); +select * from posexplode(array(1, 2)) t(x); + +-- posexplode +select * from posexplode_outer(array()); +select * from posexplode_outer(array(1, 2)); +select * from posexplode_outer(map()); +select * from posexplode_outer(map('a', 1, 'b', 2)); + +-- json_tuple +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b'); +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'c'); +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'a'); +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x, y); +select * from json_tuple('{"a": bad, "b": string}', 'a', 'b'); + +-- json_tuple with erroneous input +select * from json_tuple(); +select * from json_tuple('{"a": 1}'); +select * from json_tuple('{"a": 1}', 1); +select * from json_tuple('{"a": 1}', null); +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x); + +-- stack +select * from stack(1, 1, 2, 3); +select * from stack(2, 1, 2, 3); +select * from stack(3, 1, 2, 3) t(x); +select * from stack(4, 1, 2, 3) t(x); +select * from stack(2, 1, 1.1, 'a', 2, 2.2, 'b') t(a, b, c); +select * from stack(2, 1, 1.1, null, 2, null, 'b') t(a, b, c); + +-- stack with erroneous input +select * from stack(); +select * from stack(2, 1, 2, 3) t(a, b, c); +select * from stack(2, 1, '1.1', 'a', 2, 2.2, 'b'); +select * from stack(2, explode(array(1, 2, 3))); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/tablesample-negative.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/tablesample-negative.sql new file mode 100644 index 000000000000..72508f59bee2 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/tablesample-negative.sql @@ -0,0 +1,14 @@ +-- Negative testcases for tablesample +CREATE DATABASE mydb1; +USE mydb1; +CREATE TABLE t1 USING parquet AS SELECT 1 AS i1; + +-- Negative tests: negative percentage +SELECT mydb1.t1 FROM t1 TABLESAMPLE (-1 PERCENT); + +-- Negative tests: percentage over 100 +-- The TABLESAMPLE clause samples without replacement, so the value of PERCENT must not exceed 100 +SELECT mydb1.t1 FROM t1 TABLESAMPLE (101 PERCENT); + +-- reset +DROP DATABASE mydb1 CASCADE; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timestamp-ltz.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timestamp-ltz.sql new file mode 100644 index 000000000000..28fe4539855c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timestamp-ltz.sql @@ -0,0 +1,18 @@ +-- timestamp_ltz literals and constructors +--CONFIG_DIM1 spark.sql.timestampType=TIMESTAMP_LTZ +--CONFIG_DIM2 spark.sql.timestampType=TIMESTAMP_NTZ + +select timestamp_ltz'2016-12-31 00:12:00', timestamp_ltz'2016-12-31'; + +select to_timestamp_ltz(null), to_timestamp_ltz('2016-12-31 00:12:00'), to_timestamp_ltz('2016-12-31', 'yyyy-MM-dd'); +-- `to_timestamp_ltz` can also take date input +select to_timestamp_ltz(to_date(null)), to_timestamp_ltz(to_date('2016-12-31')); +-- `to_timestamp_ltz` can also take timestamp_ntz input +select to_timestamp_ltz(to_timestamp_ntz(null)), to_timestamp_ltz(to_timestamp_ntz('2016-12-31 00:12:00')); + +-- TimestampLTZ numeric fields constructor +SELECT make_timestamp_ltz(2021, 07, 11, 6, 30, 45.678); +SELECT make_timestamp_ltz(2021, 07, 11, 6, 30, 45.678, 'CET'); +SELECT make_timestamp_ltz(2021, 07, 11, 6, 30, 60.007); + +SELECT convert_timezone('Europe/Brussels', timestamp_ltz'2022-03-23 00:00:00 America/Los_Angeles'); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timestamp.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timestamp.sql new file mode 100644 index 000000000000..8e005c995d4d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timestamp.sql @@ -0,0 +1,172 @@ +-- timestamp literals, functions and operations + +select timestamp '2019-01-01\t'; +select timestamp '2019-01-01中文'; + +-- invalid: year too large +select timestamp'4294967297'; +-- invalid: minute field can have at most 2 digits +select timestamp'2021-01-01T12:30:4294967297.123456'; + +select current_timestamp = current_timestamp; +-- under ANSI mode, `current_timestamp` can't be a function name. +select current_timestamp() = current_timestamp(); + +select localtimestamp() = localtimestamp(); + +-- timestamp numeric fields constructor +SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678); +SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678, 'CET'); +SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007); +SELECT make_timestamp(1, 1, 1, 1, 1, 1); +SELECT make_timestamp(1, 1, 1, 1, 1, 60); +SELECT make_timestamp(1, 1, 1, 1, 1, 61); +SELECT make_timestamp(1, 1, 1, 1, 1, null); +SELECT make_timestamp(1, 1, 1, 1, 1, 59.999999); +SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999); +SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999); + +-- [SPARK-31710] TIMESTAMP_SECONDS, TIMESTAMP_MILLISECONDS and TIMESTAMP_MICROSECONDS that always create timestamp_ltz +select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null); +select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23)); +select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null); +select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null); +-- overflow exception +select TIMESTAMP_SECONDS(1230219000123123); +select TIMESTAMP_SECONDS(-1230219000123123); +select TIMESTAMP_MILLIS(92233720368547758); +select TIMESTAMP_MILLIS(-92233720368547758); +-- truncate exception +select TIMESTAMP_SECONDS(0.1234567); +-- truncation is OK for float/double +select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567)); + +-- [SPARK-22333]: timeFunctionCall has conflicts with columnReference +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation; +create temporary view ttf1 as select * from values + (1, 2), + (2, 3) + as ttf1(`current_date`, `current_timestamp`); +select typeof(current_date), typeof(current_timestamp) from ttf1; + +create temporary view ttf2 as select * from values + (1, 2), + (2, 3) + as ttf2(a, b); +select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2; +select a, b from ttf2 order by a, current_date; + + +-- UNIX_SECONDS, UNIX_MILLISECONDS and UNIX_MICROSECONDS +select UNIX_SECONDS(timestamp'2020-12-01 14:30:08Z'), UNIX_SECONDS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_SECONDS(null); +select UNIX_MILLIS(timestamp'2020-12-01 14:30:08Z'), UNIX_MILLIS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MILLIS(null); +select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MICROS(null); + +select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd'); +select to_timestamp(1); +-- variable-length second fraction tests +select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +-- second fraction exceeded max variable length +select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +-- special cases +select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]'); +select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]'); +select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]'); +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]'); +select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]'); +select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS"); +select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm"); +select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm"); +select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm"); +select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm"); + +select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'"); +select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd"); + +select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS"); -- middle +select to_timestamp("2019-10-06T10:11:12'", "yyyy-MM-dd'T'HH:mm:ss''"); -- tail +select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss"); -- head +select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss"); -- head but as single quote + +-- missing fields +select to_timestamp("16", "dd"); +select to_timestamp("02-29", "MM-dd"); +select to_timestamp("2019 40", "yyyy mm"); +select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss"); + +-- timestamp - timestamp +select timestamp'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'; +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678'; +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01'; +-- if one side is string/null literal, promote it to timestamp type. +select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'; +select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'; +select timestamp'2011-11-11 11:11:11' - null; +select null - timestamp'2011-11-11 11:11:11'; +-- invalid: non-literal string column +create temporary view ts_view as select '2011-11-11 11:11:11' str; +select str - timestamp'2011-11-11 11:11:11' from ts_view; +select timestamp'2011-11-11 11:11:11' - str from ts_view; + +-- invalid: timestamp + string literal +select timestamp'2011-11-11 11:11:11' + '1'; +select '1' + timestamp'2011-11-11 11:11:11'; + +-- null result: timestamp + null +select timestamp'2011-11-11 11:11:11' + null; +select null + timestamp'2011-11-11 11:11:11'; + +-- timestamp +/- interval and interval + timestamp +select timestamp'2011-11-11 11:11:11' + interval '2' day, + timestamp'2011-11-11 11:11:11' - interval '2-2' year to month, + timestamp'2011-11-11 11:11:11' + interval '-2' second, + timestamp'2011-11-11 11:11:11' - interval '12:12:12.123456789' hour to second, + - interval 2 years + timestamp'2011-11-11 11:11:11', + interval '1 12' day to hour + timestamp'2011-11-11 11:11:11'; +-- promote date to timestamp if the interval is hour to second. +select date '2012-01-01' - interval 3 hours, + date '2012-01-01' + interval '12:12:12' hour to second, + interval '2' minute + date '2012-01-01'; + +-- Unsupported narrow text style +select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG'); +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE'); +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE'); +select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE'); +select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')); +select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')); + +-- Add a number of units to a timestamp or a date +select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03'); +select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03'); +select timestampadd(YEAR, 1, date'2022-02-15'); +select timestampadd(SECOND, -1, date'2022-02-15'); + +select timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03'); +select timestampadd('SECOND', -1, date'2022-02-15'); + +-- Get the difference between timestamps in the specified units +select timestampdiff(MONTH, timestamp'2022-02-14 01:02:03', timestamp'2022-01-14 01:02:03'); +select timestampdiff(MINUTE, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03'); +select timestampdiff(YEAR, date'2022-02-15', date'2023-02-15'); +select timestampdiff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59'); + +select timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03'); +select timestampdiff('YEAR', date'2022-02-15', date'2023-02-15'); + +select timediff(QUARTER, timestamp'2023-08-10 01:02:03', timestamp'2022-01-14 01:02:03'); +select timediff(HOUR, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 12:00:03'); +select timediff(DAY, date'2022-02-15', date'2023-02-15'); +select timediff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59'); + +select timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03'); +select timediff('YEAR', date'2020-02-15', date'2023-02-15'); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timestampNTZ/datetime-special-ansi.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timestampNTZ/datetime-special-ansi.sql new file mode 100644 index 000000000000..527f81fa5abf --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timestampNTZ/datetime-special-ansi.sql @@ -0,0 +1,2 @@ +--SET spark.sql.ansi.enabled = true +--IMPORT datetime-special.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timestampNTZ/datetime-special.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timestampNTZ/datetime-special.sql new file mode 100644 index 000000000000..a9aa3f3f6a3a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timestampNTZ/datetime-special.sql @@ -0,0 +1,2 @@ +--SET spark.sql.ansi.enabled = false +--IMPORT datetime-special.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timestampNTZ/timestamp-ansi.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timestampNTZ/timestamp-ansi.sql new file mode 100644 index 000000000000..e28cb24ff9fe --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timestampNTZ/timestamp-ansi.sql @@ -0,0 +1,2 @@ +--SET spark.sql.ansi.enabled = true +--IMPORT timestamp.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timestampNTZ/timestamp.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timestampNTZ/timestamp.sql new file mode 100644 index 000000000000..47988ee65fb7 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timestampNTZ/timestamp.sql @@ -0,0 +1,2 @@ +--SET spark.sql.ansi.enabled = false +--IMPORT timestamp.sql diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timezone.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timezone.sql new file mode 100644 index 000000000000..105f2816fb54 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/timezone.sql @@ -0,0 +1,15 @@ +-- valid time zones +SET TIME ZONE 'Asia/Hong_Kong'; +SET TIME ZONE 'GMT+1'; +SET TIME ZONE INTERVAL 10 HOURS; +SET TIME ZONE INTERVAL '15:40:32' HOUR TO SECOND; +SET TIME ZONE LOCAL; + +-- invalid time zone +SET TIME ZONE; +SET TIME ZONE 'invalid/zone'; +SET TIME ZONE INTERVAL 3 DAYS; +SET TIME ZONE INTERVAL 24 HOURS; +SET TIME ZONE INTERVAL '19:40:32' HOUR TO SECOND; +SET TIME ZONE INTERVAL 10 HOURS 'GMT+1'; +SET TIME ZONE INTERVAL 10 HOURS 1 MILLISECOND; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/to_from_avro.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/to_from_avro.sql new file mode 100644 index 000000000000..12541ff26e24 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/to_from_avro.sql @@ -0,0 +1,21 @@ +-- Create some temporary test data. +create table t as + select named_struct('u', named_struct('member0', member0, 'member1', member1)) as s + from values (1, null), (null, 'a') tab(member0, member1); +declare avro_schema string; +set variable avro_schema = + '{ "type": "record", "name": "struct", "fields": [{ "name": "u", "type": ["int","string"] }] }'; + +-- Exercise invalid SQL syntax when calling the 'from_avro' and 'to_avro' functions. +select from_avro(s, 42, map()) from t; +select from_avro(s, avro_schema, 42) from t; +select to_avro(s, 42) from t; + +-- Avro is not loaded in this testing environment, so queries calling the 'from_avro' or 'to_avro' +-- SQL functions that otherwise pass analysis return appropriate "Avro not loaded" errors here. +select to_avro(s, avro_schema) as result from t; +select from_avro(result, avro_schema, map()).u from (select null as result); + +-- Clean up. +drop temporary variable avro_schema; +drop table t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/transform.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/transform.sql new file mode 100644 index 000000000000..8570496d439e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/transform.sql @@ -0,0 +1,422 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW t AS SELECT * FROM VALUES +('1', true, unhex('537061726B2053514C'), tinyint(1), 1, smallint(100), bigint(1), float(1.0), 1.0, Decimal(1.0), timestamp('1997-01-02'), date('2000-04-01')), +('2', false, unhex('537061726B2053514C'), tinyint(2), 2, smallint(200), bigint(2), float(2.0), 2.0, Decimal(2.0), timestamp('1997-01-02 03:04:05'), date('2000-04-02')), +('3', true, unhex('537061726B2053514C'), tinyint(3), 3, smallint(300), bigint(3), float(3.0), 3.0, Decimal(3.0), timestamp('1997-02-10 17:32:01-08'), date('2000-04-03')) +AS t(a, b, c, d, e, f, g, h, i, j, k, l); + +CREATE OR REPLACE TEMPORARY VIEW script_trans AS SELECT * FROM VALUES +(1, 2, 3), +(4, 5, 6), +(7, 8, 9) +AS script_trans(a, b, c); + +CREATE OR REPLACE TEMPORARY VIEW complex_trans AS SELECT * FROM VALUES +(1, 1), +(1, 1), +(2, 2), +(2, 2), +(3, 3), +(2, 2), +(3, 3), +(1, 1), +(3, 3) +as complex_trans(a, b); + +SELECT TRANSFORM(a) +USING 'cat' AS (a) +FROM t; + +-- common supported data types between no serde and serde transform +SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( + SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) + USING 'cat' AS ( + a string, + b boolean, + c binary, + d tinyint, + e int, + f smallint, + g long, + h float, + i double, + j decimal(38, 18), + k timestamp, + l date) + FROM t +) tmp; + +-- common supported data types between no serde and serde transform +SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( + SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) + USING 'cat' AS ( + a string, + b string, + c string, + d string, + e string, + f string, + g string, + h string, + i string, + j string, + k string, + l string) + FROM t +) tmp; + +-- SPARK-32388 handle schema less +SELECT TRANSFORM(a) +USING 'cat' +FROM t; + +SELECT TRANSFORM(a, b) +USING 'cat' +FROM t; + +SELECT TRANSFORM(a, b, c) +USING 'cat' +FROM t; + +-- return null when return string incompatible (no serde) +SELECT TRANSFORM(a, b, c, d, e, f, g, h, i) +USING 'cat' AS (a int, b short, c long, d byte, e float, f double, g decimal(38, 18), h date, i timestamp) +FROM VALUES +('a','','1231a','a','213.21a','213.21a','0a.21d','2000-04-01123','1997-0102 00:00:') tmp(a, b, c, d, e, f, g, h, i); + +-- SPARK-28227: transform can't run with aggregation +SELECT TRANSFORM(b, max(a), sum(f)) +USING 'cat' AS (a, b) +FROM t +GROUP BY b; + +-- transform use MAP +MAP a, b USING 'cat' AS (a, b) FROM t; + +-- transform use REDUCE +REDUCE a, b USING 'cat' AS (a, b) FROM t; + +-- transform with defined row format delimit +SELECT TRANSFORM(a, b, c, null) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' +USING 'cat' AS (a, b, c, d) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' +FROM t; + +SELECT TRANSFORM(a, b, c, null) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' +USING 'cat' AS (d) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' +FROM t; + +SELECT TRANSFORM(a, b, c, null) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' +USING 'cat' AS (a, b, c, d) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' +FROM t; + +SELECT TRANSFORM(a, b, c, null) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' +USING 'cat' AS (a, b, c, d) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' +FROM t; + +SELECT TRANSFORM(a, b, c, null) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'XXXX' +USING 'cat' AS (a, b, c, d) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' +FROM t; + +SELECT TRANSFORM(a, b, c, null) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS '\n' +USING 'cat' AS (a, b, c, d) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' +FROM t; + +-- transform with defined row format delimit handle schema with correct type +SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( + SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' + USING 'cat' AS ( + a string, + b boolean, + c binary, + d tinyint, + e int, + f smallint, + g long, + h float, + i double, + j decimal(38, 18), + k timestamp, + l date) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' + FROM t +) tmp; + +-- transform with defined row format delimit handle schema with wrong type +SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( + SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' + USING 'cat' AS ( + a string, + b long, + c binary, + d tinyint, + e int, + f smallint, + g long, + h float, + i double, + j decimal(38, 18), + k int, + l long) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' + FROM t +) tmp; + +-- transform with defined row format delimit LINE TERMINATED BY only support '\n' +SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( + SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '@' + NULL DEFINED AS 'NULL' + USING 'cat' AS ( + a string, + b string, + c string, + d string, + e string, + f string, + g string, + h string, + i string, + j string, + k string, + l string) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '@' + NULL DEFINED AS 'NULL' + FROM t +) tmp; + +SELECT TRANSFORM(b, a, CAST(c AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4; + +SELECT TRANSFORM(1, 2, 3) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4; + +SELECT TRANSFORM(1, 2) + USING 'cat' AS (a INT, b INT) +FROM script_trans +LIMIT 1; + +SELECT TRANSFORM( + b, a, + CASE + WHEN c > 100 THEN 1 + WHEN c < 100 THEN 2 + ELSE 3 END) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4; + +SELECT TRANSFORM(b, a, c + 1) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4; + +SELECT TRANSFORM(*) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4; + +SELECT TRANSFORM(b, MAX(a), CAST(SUM(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +GROUP BY b; + +SELECT TRANSFORM(b, MAX(a) FILTER (WHERE a > 3), CAST(SUM(c) AS STRING)) + USING 'cat' AS (a,b,c) +FROM script_trans +WHERE a <= 4 +GROUP BY b; + +SELECT TRANSFORM(b, MAX(a), CAST(sum(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 2 +GROUP BY b; + +SELECT TRANSFORM(b, MAX(a), CAST(SUM(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +GROUP BY b +HAVING MAX(a) > 0; + +SELECT TRANSFORM(b, MAX(a), CAST(SUM(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +GROUP BY b +HAVING MAX(a) > 1; + +SELECT TRANSFORM(b, MAX(a) OVER w, CAST(SUM(c) OVER w AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +WINDOW w AS (PARTITION BY b ORDER BY a); + +SELECT TRANSFORM(b, MAX(a), CAST(SUM(c) AS STRING), myCol, myCol2) + USING 'cat' AS (a STRING, b STRING, c STRING, d ARRAY, e STRING) +FROM script_trans +LATERAL VIEW explode(array(array(1,2,3))) myTable AS myCol +LATERAL VIEW explode(myTable.myCol) myTable2 AS myCol2 +WHERE a <= 4 +GROUP BY b, myCol, myCol2 +HAVING max(a) > 1; + +FROM( + FROM script_trans + SELECT TRANSFORM(a, b) + USING 'cat' AS (`a` INT, b STRING) +) t +SELECT a + 1; + +FROM( + SELECT TRANSFORM(a, SUM(b)) + USING 'cat' AS (`a` INT, b STRING) + FROM script_trans + GROUP BY a +) t +SELECT (b + 1) AS result +ORDER BY result; + +MAP k / 10 USING 'cat' AS (one) FROM (SELECT 10 AS k); + +FROM (SELECT 1 AS key, 100 AS value) src +MAP src.*, src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value + USING 'cat' AS (k, v, tkey, ten, one, tvalue); + +SELECT TRANSFORM(1) + USING 'cat' AS (a) +FROM script_trans +HAVING true; + +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=true; + +SELECT TRANSFORM(1) + USING 'cat' AS (a) +FROM script_trans +HAVING true; + +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=false; + +-- SPARK-34634: self join using CTE contains transform +WITH temp AS ( + SELECT TRANSFORM(a) USING 'cat' AS (b string) FROM t +) +SELECT t1.b FROM temp t1 JOIN temp t2 ON t1.b = t2.b; + +SELECT TRANSFORM(DISTINCT b, a, c) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4; + +SELECT TRANSFORM(ALL b, a, c) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4; + +-- SPARK-35070: TRANSFORM not support alias in inputs +SELECT TRANSFORM(b AS b_1, MAX(a), CAST(sum(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 2 +GROUP BY b; + +SELECT TRANSFORM(b b_1, MAX(a), CAST(sum(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 2 +GROUP BY b; + +SELECT TRANSFORM(b, MAX(a) AS max_a, CAST(sum(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 2 +GROUP BY b; + +-- SPARK-33985: TRANSFORM with CLUSTER BY/ORDER BY/SORT BY +FROM ( + SELECT TRANSFORM(a, b) + USING 'cat' AS (a, b) + FROM complex_trans + CLUSTER BY a +) map_output +SELECT TRANSFORM(a, b) + USING 'cat' AS (a, b); + +FROM ( + SELECT TRANSFORM(a, b) + USING 'cat' AS (a, b) + FROM complex_trans + ORDER BY a +) map_output +SELECT TRANSFORM(a, b) + USING 'cat' AS (a, b); + +SELECT TRANSFORM (a, b) + USING 'cat' AS (a CHAR(10), b VARCHAR(10)) +FROM VALUES('apache', 'spark') t(a, b); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try-string-functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try-string-functions.sql new file mode 100644 index 000000000000..4ff3e69da67a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try-string-functions.sql @@ -0,0 +1,48 @@ +-- try_to_binary +-- base64 valid +select try_to_binary('', 'base64'); +select try_to_binary(' ', 'base64'); +select try_to_binary(' ab cd ', 'base64'); +select try_to_binary(' ab c=', 'base64'); +select try_to_binary(' ab cdef= = ', 'base64'); +select try_to_binary( + concat(' b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0', + 'd2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl'), 'base64'); +-- base64 invalid +select try_to_binary('a', 'base64'); +select try_to_binary('a?', 'base64'); +select try_to_binary('abcde', 'base64'); +select try_to_binary('abcd=', 'base64'); +select try_to_binary('a===', 'base64'); +select try_to_binary('ab==f', 'base64'); +-- utf-8 +select try_to_binary( + '∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β)', 'utf-8'); +select try_to_binary('大千世界', 'utf8'); +select try_to_binary('', 'utf-8'); +select try_to_binary(' ', 'utf8'); +-- hex valid +select try_to_binary('737472696E67'); +select try_to_binary('737472696E67', 'hex'); +select try_to_binary(''); +select try_to_binary('1', 'hex'); +select try_to_binary('FF'); +select try_to_binary('123'); +select try_to_binary('12345'); +-- hex invalid +select try_to_binary('GG'); +select try_to_binary('01 AF', 'hex'); +-- 'format' parameter can be any foldable string value, not just literal. +select try_to_binary('abc', concat('utf', '-8')); +select try_to_binary(' ab cdef= = ', substr('base64whynot', 0, 6)); +select try_to_binary(' ab cdef= = ', replace('HEX0', '0')); +-- 'format' parameter is case insensitive. +select try_to_binary('abc', 'Hex'); +-- null inputs lead to null result. +select try_to_binary('abc', null); +select try_to_binary(null, 'utf-8'); +select try_to_binary(null, null); +select try_to_binary(null, cast(null as string)); +-- invalid format +select try_to_binary('abc', 1); +select try_to_binary('abc', 'invalidFormat'); \ No newline at end of file diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try_aggregates.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try_aggregates.sql new file mode 100644 index 000000000000..75b140074277 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try_aggregates.sql @@ -0,0 +1,45 @@ +-- try_sum +SELECT try_sum(col) FROM VALUES (5), (10), (15) AS tab(col); +SELECT try_sum(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col); +SELECT try_sum(col) FROM VALUES (NULL), (10), (15) AS tab(col); +SELECT try_sum(col) FROM VALUES (NULL), (NULL) AS tab(col); +SELECT try_sum(col) FROM VALUES (9223372036854775807L), (1L) AS tab(col); +-- test overflow in Decimal(38, 0) +SELECT try_sum(col) FROM VALUES (98765432109876543210987654321098765432BD), (98765432109876543210987654321098765432BD) AS tab(col); + +SELECT try_sum(col) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col); +SELECT try_sum(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col); +SELECT try_sum(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col); +SELECT try_sum(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col); + +-- errors in child should be shown in ANSI mode +SELECT try_sum(col / 0) FROM VALUES (5), (10), (15) AS tab(col); +SELECT try_sum(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col); +SELECT try_sum(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col); +SELECT try_sum(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col); + +SELECT try_sum(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col); +SELECT try_sum(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col); + +-- try_avg +SELECT try_avg(col) FROM VALUES (5), (10), (15) AS tab(col); +SELECT try_avg(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col); +SELECT try_avg(col) FROM VALUES (NULL), (10), (15) AS tab(col); +SELECT try_avg(col) FROM VALUES (NULL), (NULL) AS tab(col); +SELECT try_avg(col) FROM VALUES (9223372036854775807L), (1L) AS tab(col); +-- test overflow in Decimal(38, 0) +SELECT try_avg(col) FROM VALUES (98765432109876543210987654321098765432BD), (98765432109876543210987654321098765432BD) AS tab(col); + +SELECT try_avg(col) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col); +SELECT try_avg(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col); +SELECT try_avg(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col); +SELECT try_avg(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col); + +-- errors in child should be shown in ANSI mode +SELECT try_avg(col / 0) FROM VALUES (5), (10), (15) AS tab(col); +SELECT try_avg(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col); +SELECT try_avg(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col); +SELECT try_avg(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col); + +SELECT try_avg(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col); +SELECT try_avg(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try_arithmetic.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try_arithmetic.sql new file mode 100644 index 000000000000..c7957e0f392b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try_arithmetic.sql @@ -0,0 +1,88 @@ +-- Numeric + Numeric +SELECT try_add(1, 1); +SELECT try_add(2147483647, 1); +SELECT try_add(2147483647, decimal(1)); +SELECT try_add(2147483647, "1"); +SELECT try_add(-2147483648, -1); +SELECT try_add(9223372036854775807L, 1); +SELECT try_add(-9223372036854775808L, -1); +SELECT try_add(1, 1.0 / 0.0); + +-- Date + Integer +SELECT try_add(date'2021-01-01', 1); +SELECT try_add(1, date'2021-01-01'); + +-- Date + Interval +SELECT try_add(date'2021-01-01', interval 2 year); +SELECT try_add(date'2021-01-01', interval 2 second); +SELECT try_add(interval 2 year, date'2021-01-01'); +SELECT try_add(interval 2 second, date'2021-01-01'); + +-- Timestamp + Interval +SELECT try_add(timestamp_ltz'2021-01-01 00:00:00', interval 2 year); +SELECT try_add(timestamp_ntz'2021-01-01 00:00:00', interval 2 second); +SELECT try_add(interval 2 year, timestamp_ltz'2021-01-01 00:00:00'); +SELECT try_add(interval 2 second, timestamp_ntz'2021-01-01 00:00:00'); + +-- Interval + Interval +SELECT try_add(interval 2 year, interval 2 year); +SELECT try_add(interval 2 second, interval 2 second); +SELECT try_add(interval 2 year, interval 2 second); +SELECT try_add(interval 2147483647 month, interval 2 month); +SELECT try_add(interval 106751991 day, interval 3 day); + +-- Numeric / Numeric +SELECT try_divide(1, 0.5); +SELECT try_divide(1, 0); +SELECT try_divide(0, 0); +SELECT try_divide(1, (2147483647 + 1)); +SELECT try_divide(1L, (9223372036854775807L + 1L)); +SELECT try_divide(1, 1.0 / 0.0); +SELECT try_divide(1, decimal(0)); +SELECT try_divide(1, "0"); + +-- Interval / Numeric +SELECT try_divide(interval 2 year, 2); +SELECT try_divide(interval 2 second, 2); +SELECT try_divide(interval 2 year, 0); +SELECT try_divide(interval 2 second, 0); +SELECT try_divide(interval 2147483647 month, 0.5); +SELECT try_divide(interval 106751991 day, 0.5); + +-- Numeric - Numeric +SELECT try_subtract(1, 1); +SELECT try_subtract(2147483647, -1); +SELECT try_subtract(2147483647, decimal(-1)); +SELECT try_subtract(2147483647, "-1"); +SELECT try_subtract(-2147483648, 1); +SELECT try_subtract(9223372036854775807L, -1); +SELECT try_subtract(-9223372036854775808L, 1); +SELECT try_subtract(1, (2147483647 + 1)); +SELECT try_subtract(1L, (9223372036854775807L + 1L)); +SELECT try_subtract(1, 1.0 / 0.0); + +-- Interval - Interval +SELECT try_subtract(interval 2 year, interval 3 year); +SELECT try_subtract(interval 3 second, interval 2 second); +SELECT try_subtract(interval 2147483647 month, interval -2 month); +SELECT try_subtract(interval 106751991 day, interval -3 day); + +-- Numeric * Numeric +SELECT try_multiply(2, 3); +SELECT try_multiply(2147483647, -2); +SELECT try_multiply(2147483647, decimal(-2)); +SELECT try_multiply(2147483647, "-2"); +SELECT try_multiply(-2147483648, 2); +SELECT try_multiply(9223372036854775807L, 2); +SELECT try_multiply(-9223372036854775808L, -2); +SELECT try_multiply(1, (2147483647 + 1)); +SELECT try_multiply(1L, (9223372036854775807L + 1L)); +SELECT try_multiply(1, 1.0 / 0.0); + +-- Interval * Numeric +SELECT try_multiply(interval 2 year, 2); +SELECT try_multiply(interval 2 second, 2); +SELECT try_multiply(interval 2 year, 0); +SELECT try_multiply(interval 2 second, 0); +SELECT try_multiply(interval 2147483647 month, 2); +SELECT try_multiply(interval 106751991 day, 2); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try_cast.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try_cast.sql new file mode 100644 index 000000000000..63553babfc5c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try_cast.sql @@ -0,0 +1,50 @@ +-- TRY_CAST string representing a valid fractional number to integral should truncate the number + +-- TRY_CAST string which are not numbers to integral should return null +SELECT TRY_CAST('abc' AS int); +SELECT TRY_CAST('abc' AS long); + +-- TRY_CAST empty string to integral should return null +SELECT TRY_CAST('' AS int); +SELECT TRY_CAST('' AS long); + +-- TRY_CAST null to integral should return null +SELECT TRY_CAST(NULL AS int); +SELECT TRY_CAST(NULL AS long); + +-- TRY_CAST invalid decimal string to integral should return null +SELECT TRY_CAST('123.a' AS int); +SELECT TRY_CAST('123.a' AS long); + +-- '-2147483648' is the smallest int value +SELECT TRY_CAST('-2147483648' AS int); +SELECT TRY_CAST('-2147483649' AS int); + +-- '2147483647' is the largest int value +SELECT TRY_CAST('2147483647' AS int); +SELECT TRY_CAST('2147483648' AS int); + +-- '-9223372036854775808' is the smallest long value +SELECT TRY_CAST('-9223372036854775808' AS long); +SELECT TRY_CAST('-9223372036854775809' AS long); + +-- '9223372036854775807' is the largest long value +SELECT TRY_CAST('9223372036854775807' AS long); +SELECT TRY_CAST('9223372036854775808' AS long); + +-- TRY_CAST string to interval and interval to string +SELECT TRY_CAST('interval 3 month 1 hour' AS interval); +SELECT TRY_CAST('abc' AS interval); + +-- TRY_CAST string to boolean +select TRY_CAST('true' as boolean); +select TRY_CAST('false' as boolean); +select TRY_CAST('abc' as boolean); + +-- TRY_CAST string to date +SELECT TRY_CAST("2021-01-01" AS date); +SELECT TRY_CAST("2021-101-01" AS date); + +-- TRY_CAST string to timestamp +SELECT TRY_CAST("2021-01-01 00:00:00" AS timestamp); +SELECT TRY_CAST("2021-101-01 00:00:00" AS timestamp); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try_datetime_functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try_datetime_functions.sql new file mode 100644 index 000000000000..7cf67dce2ae0 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try_datetime_functions.sql @@ -0,0 +1,6 @@ +select try_to_timestamp(null), try_to_timestamp('2016-12-31 00:12:00'), try_to_timestamp('2016-12-31', 'yyyy-MM-dd'); +select try_to_timestamp(1); +select try_to_timestamp('2016-12-31 abc'); +select try_to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]'); +select try_to_timestamp("02-29", "MM-dd"); +select try_to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE'); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try_element_at.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try_element_at.sql new file mode 100644 index 000000000000..c02c2dcb34fd --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try_element_at.sql @@ -0,0 +1,11 @@ +-- array input +SELECT try_element_at(array(1, 2, 3), 0); +SELECT try_element_at(array(1, 2, 3), 1); +SELECT try_element_at(array(1, 2, 3), 3); +SELECT try_element_at(array(1, 2, 3), 4); +SELECT try_element_at(array(1, 2, 3), -1); +SELECT try_element_at(array(1, 2, 3), -4); + +-- map input +SELECT try_element_at(map('a','b'), 'a'); +SELECT try_element_at(map('a','b'), 'abc'); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try_reflect.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try_reflect.sql new file mode 100644 index 000000000000..dd2bce7ef1f8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/try_reflect.sql @@ -0,0 +1,15 @@ +-- positive +SELECT try_reflect("java.util.UUID", "fromString", "a5cf6c42-0c85-418f-af6c-3e4e5b1328f2"); +SELECT try_reflect("java.lang.String", "valueOf", 1); +SELECT try_reflect("java.lang.Math", "max", 2, 3); +SELECT try_reflect("java.lang.Math", "min", 2, 3); +SELECT try_reflect("java.lang.Integer", "valueOf", "10", 16); + +-- negative +SELECT try_reflect("java.util.UUID", "fromString", "b"); +SELECT try_reflect("java.net.URLDecoder", "decode", "%"); +SELECT try_reflect("java.wrongclass.Math", "max", 2, 3); +SELECT try_reflect("java.lang.Math", "wrongmethod", 2, 3); +SELECT try_reflect("java.lang.Math"); +SELECT try_reflect("java.lang.Math", "round", 2.5); +SELECT try_reflect("java.lang.Object", "toString"); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/arrayJoin.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/arrayJoin.sql new file mode 100644 index 000000000000..99729c007b10 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/arrayJoin.sql @@ -0,0 +1,11 @@ +SELECT array_join(array(true, false), ', '); +SELECT array_join(array(2Y, 1Y), ', '); +SELECT array_join(array(2S, 1S), ', '); +SELECT array_join(array(2, 1), ', '); +SELECT array_join(array(2L, 1L), ', '); +SELECT array_join(array(9223372036854775809, 9223372036854775808), ', '); +SELECT array_join(array(2.0D, 1.0D), ', '); +SELECT array_join(array(float(2.0), float(1.0)), ', '); +SELECT array_join(array(date '2016-03-14', date '2016-03-13'), ', '); +SELECT array_join(array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), ', '); +SELECT array_join(array('a', 'b'), ', '); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/binaryComparison.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/binaryComparison.sql new file mode 100644 index 000000000000..522322ac480b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/binaryComparison.sql @@ -0,0 +1,287 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +-- Binary Comparison + +CREATE TEMPORARY VIEW t AS SELECT 1; + +SELECT cast(1 as binary) = '1' FROM t; +SELECT cast(1 as binary) > '2' FROM t; +SELECT cast(1 as binary) >= '2' FROM t; +SELECT cast(1 as binary) < '2' FROM t; +SELECT cast(1 as binary) <= '2' FROM t; +SELECT cast(1 as binary) <> '2' FROM t; +SELECT cast(1 as binary) = cast(null as string) FROM t; +SELECT cast(1 as binary) > cast(null as string) FROM t; +SELECT cast(1 as binary) >= cast(null as string) FROM t; +SELECT cast(1 as binary) < cast(null as string) FROM t; +SELECT cast(1 as binary) <= cast(null as string) FROM t; +SELECT cast(1 as binary) <> cast(null as string) FROM t; +SELECT '1' = cast(1 as binary) FROM t; +SELECT '2' > cast(1 as binary) FROM t; +SELECT '2' >= cast(1 as binary) FROM t; +SELECT '2' < cast(1 as binary) FROM t; +SELECT '2' <= cast(1 as binary) FROM t; +SELECT '2' <> cast(1 as binary) FROM t; +SELECT cast(null as string) = cast(1 as binary) FROM t; +SELECT cast(null as string) > cast(1 as binary) FROM t; +SELECT cast(null as string) >= cast(1 as binary) FROM t; +SELECT cast(null as string) < cast(1 as binary) FROM t; +SELECT cast(null as string) <= cast(1 as binary) FROM t; +SELECT cast(null as string) <> cast(1 as binary) FROM t; +SELECT cast(1 as tinyint) = '1' FROM t; +SELECT cast(1 as tinyint) > '2' FROM t; +SELECT cast(1 as tinyint) >= '2' FROM t; +SELECT cast(1 as tinyint) < '2' FROM t; +SELECT cast(1 as tinyint) <= '2' FROM t; +SELECT cast(1 as tinyint) <> '2' FROM t; +SELECT cast(1 as tinyint) = cast(null as string) FROM t; +SELECT cast(1 as tinyint) > cast(null as string) FROM t; +SELECT cast(1 as tinyint) >= cast(null as string) FROM t; +SELECT cast(1 as tinyint) < cast(null as string) FROM t; +SELECT cast(1 as tinyint) <= cast(null as string) FROM t; +SELECT cast(1 as tinyint) <> cast(null as string) FROM t; +SELECT '1' = cast(1 as tinyint) FROM t; +SELECT '2' > cast(1 as tinyint) FROM t; +SELECT '2' >= cast(1 as tinyint) FROM t; +SELECT '2' < cast(1 as tinyint) FROM t; +SELECT '2' <= cast(1 as tinyint) FROM t; +SELECT '2' <> cast(1 as tinyint) FROM t; +SELECT cast(null as string) = cast(1 as tinyint) FROM t; +SELECT cast(null as string) > cast(1 as tinyint) FROM t; +SELECT cast(null as string) >= cast(1 as tinyint) FROM t; +SELECT cast(null as string) < cast(1 as tinyint) FROM t; +SELECT cast(null as string) <= cast(1 as tinyint) FROM t; +SELECT cast(null as string) <> cast(1 as tinyint) FROM t; +SELECT cast(1 as smallint) = '1' FROM t; +SELECT cast(1 as smallint) > '2' FROM t; +SELECT cast(1 as smallint) >= '2' FROM t; +SELECT cast(1 as smallint) < '2' FROM t; +SELECT cast(1 as smallint) <= '2' FROM t; +SELECT cast(1 as smallint) <> '2' FROM t; +SELECT cast(1 as smallint) = cast(null as string) FROM t; +SELECT cast(1 as smallint) > cast(null as string) FROM t; +SELECT cast(1 as smallint) >= cast(null as string) FROM t; +SELECT cast(1 as smallint) < cast(null as string) FROM t; +SELECT cast(1 as smallint) <= cast(null as string) FROM t; +SELECT cast(1 as smallint) <> cast(null as string) FROM t; +SELECT '1' = cast(1 as smallint) FROM t; +SELECT '2' > cast(1 as smallint) FROM t; +SELECT '2' >= cast(1 as smallint) FROM t; +SELECT '2' < cast(1 as smallint) FROM t; +SELECT '2' <= cast(1 as smallint) FROM t; +SELECT '2' <> cast(1 as smallint) FROM t; +SELECT cast(null as string) = cast(1 as smallint) FROM t; +SELECT cast(null as string) > cast(1 as smallint) FROM t; +SELECT cast(null as string) >= cast(1 as smallint) FROM t; +SELECT cast(null as string) < cast(1 as smallint) FROM t; +SELECT cast(null as string) <= cast(1 as smallint) FROM t; +SELECT cast(null as string) <> cast(1 as smallint) FROM t; +SELECT cast(1 as int) = '1' FROM t; +SELECT cast(1 as int) > '2' FROM t; +SELECT cast(1 as int) >= '2' FROM t; +SELECT cast(1 as int) < '2' FROM t; +SELECT cast(1 as int) <= '2' FROM t; +SELECT cast(1 as int) <> '2' FROM t; +SELECT cast(1 as int) = cast(null as string) FROM t; +SELECT cast(1 as int) > cast(null as string) FROM t; +SELECT cast(1 as int) >= cast(null as string) FROM t; +SELECT cast(1 as int) < cast(null as string) FROM t; +SELECT cast(1 as int) <= cast(null as string) FROM t; +SELECT cast(1 as int) <> cast(null as string) FROM t; +SELECT '1' = cast(1 as int) FROM t; +SELECT '2' > cast(1 as int) FROM t; +SELECT '2' >= cast(1 as int) FROM t; +SELECT '2' < cast(1 as int) FROM t; +SELECT '2' <> cast(1 as int) FROM t; +SELECT '2' <= cast(1 as int) FROM t; +SELECT cast(null as string) = cast(1 as int) FROM t; +SELECT cast(null as string) > cast(1 as int) FROM t; +SELECT cast(null as string) >= cast(1 as int) FROM t; +SELECT cast(null as string) < cast(1 as int) FROM t; +SELECT cast(null as string) <> cast(1 as int) FROM t; +SELECT cast(null as string) <= cast(1 as int) FROM t; +SELECT cast(1 as bigint) = '1' FROM t; +SELECT cast(1 as bigint) > '2' FROM t; +SELECT cast(1 as bigint) >= '2' FROM t; +SELECT cast(1 as bigint) < '2' FROM t; +SELECT cast(1 as bigint) <= '2' FROM t; +SELECT cast(1 as bigint) <> '2' FROM t; +SELECT cast(1 as bigint) = cast(null as string) FROM t; +SELECT cast(1 as bigint) > cast(null as string) FROM t; +SELECT cast(1 as bigint) >= cast(null as string) FROM t; +SELECT cast(1 as bigint) < cast(null as string) FROM t; +SELECT cast(1 as bigint) <= cast(null as string) FROM t; +SELECT cast(1 as bigint) <> cast(null as string) FROM t; +SELECT '1' = cast(1 as bigint) FROM t; +SELECT '2' > cast(1 as bigint) FROM t; +SELECT '2' >= cast(1 as bigint) FROM t; +SELECT '2' < cast(1 as bigint) FROM t; +SELECT '2' <= cast(1 as bigint) FROM t; +SELECT '2' <> cast(1 as bigint) FROM t; +SELECT cast(null as string) = cast(1 as bigint) FROM t; +SELECT cast(null as string) > cast(1 as bigint) FROM t; +SELECT cast(null as string) >= cast(1 as bigint) FROM t; +SELECT cast(null as string) < cast(1 as bigint) FROM t; +SELECT cast(null as string) <= cast(1 as bigint) FROM t; +SELECT cast(null as string) <> cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) = '1' FROM t; +SELECT cast(1 as decimal(10, 0)) > '2' FROM t; +SELECT cast(1 as decimal(10, 0)) >= '2' FROM t; +SELECT cast(1 as decimal(10, 0)) < '2' FROM t; +SELECT cast(1 as decimal(10, 0)) <> '2' FROM t; +SELECT cast(1 as decimal(10, 0)) <= '2' FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(null as string) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(null as string) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(null as string) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(null as string) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(null as string) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(null as string) FROM t; +SELECT '1' = cast(1 as decimal(10, 0)) FROM t; +SELECT '2' > cast(1 as decimal(10, 0)) FROM t; +SELECT '2' >= cast(1 as decimal(10, 0)) FROM t; +SELECT '2' < cast(1 as decimal(10, 0)) FROM t; +SELECT '2' <= cast(1 as decimal(10, 0)) FROM t; +SELECT '2' <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(null as string) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(null as string) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(null as string) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(null as string) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(null as string) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(null as string) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) = '1' FROM t; +SELECT cast(1 as double) > '2' FROM t; +SELECT cast(1 as double) >= '2' FROM t; +SELECT cast(1 as double) < '2' FROM t; +SELECT cast(1 as double) <= '2' FROM t; +SELECT cast(1 as double) <> '2' FROM t; +SELECT cast(1 as double) = cast(null as string) FROM t; +SELECT cast(1 as double) > cast(null as string) FROM t; +SELECT cast(1 as double) >= cast(null as string) FROM t; +SELECT cast(1 as double) < cast(null as string) FROM t; +SELECT cast(1 as double) <= cast(null as string) FROM t; +SELECT cast(1 as double) <> cast(null as string) FROM t; +SELECT '1' = cast(1 as double) FROM t; +SELECT '2' > cast(1 as double) FROM t; +SELECT '2' >= cast(1 as double) FROM t; +SELECT '2' < cast(1 as double) FROM t; +SELECT '2' <= cast(1 as double) FROM t; +SELECT '2' <> cast(1 as double) FROM t; +SELECT cast(null as string) = cast(1 as double) FROM t; +SELECT cast(null as string) > cast(1 as double) FROM t; +SELECT cast(null as string) >= cast(1 as double) FROM t; +SELECT cast(null as string) < cast(1 as double) FROM t; +SELECT cast(null as string) <= cast(1 as double) FROM t; +SELECT cast(null as string) <> cast(1 as double) FROM t; +SELECT cast(1 as float) = '1' FROM t; +SELECT cast(1 as float) > '2' FROM t; +SELECT cast(1 as float) >= '2' FROM t; +SELECT cast(1 as float) < '2' FROM t; +SELECT cast(1 as float) <= '2' FROM t; +SELECT cast(1 as float) <> '2' FROM t; +SELECT cast(1 as float) = cast(null as string) FROM t; +SELECT cast(1 as float) > cast(null as string) FROM t; +SELECT cast(1 as float) >= cast(null as string) FROM t; +SELECT cast(1 as float) < cast(null as string) FROM t; +SELECT cast(1 as float) <= cast(null as string) FROM t; +SELECT cast(1 as float) <> cast(null as string) FROM t; +SELECT '1' = cast(1 as float) FROM t; +SELECT '2' > cast(1 as float) FROM t; +SELECT '2' >= cast(1 as float) FROM t; +SELECT '2' < cast(1 as float) FROM t; +SELECT '2' <= cast(1 as float) FROM t; +SELECT '2' <> cast(1 as float) FROM t; +SELECT cast(null as string) = cast(1 as float) FROM t; +SELECT cast(null as string) > cast(1 as float) FROM t; +SELECT cast(null as string) >= cast(1 as float) FROM t; +SELECT cast(null as string) < cast(1 as float) FROM t; +SELECT cast(null as string) <= cast(1 as float) FROM t; +SELECT cast(null as string) <> cast(1 as float) FROM t; +-- the following queries return 1 if the search condition is satisfied +-- and returns nothing if the search condition is not satisfied +SELECT '1996-09-09' = date('1996-09-09') FROM t; +SELECT '1996-9-10' > date('1996-09-09') FROM t; +SELECT '1996-9-10' >= date('1996-09-09') FROM t; +SELECT '1996-9-10' < date('1996-09-09') FROM t; +SELECT '1996-9-10' <= date('1996-09-09') FROM t; +SELECT '1996-9-10' <> date('1996-09-09') FROM t; +SELECT cast(null as string) = date('1996-09-09') FROM t; +SELECT cast(null as string)> date('1996-09-09') FROM t; +SELECT cast(null as string)>= date('1996-09-09') FROM t; +SELECT cast(null as string)< date('1996-09-09') FROM t; +SELECT cast(null as string)<= date('1996-09-09') FROM t; +SELECT cast(null as string)<> date('1996-09-09') FROM t; +SELECT date('1996-09-09') = '1996-09-09' FROM t; +SELECT date('1996-9-10') > '1996-09-09' FROM t; +SELECT date('1996-9-10') >= '1996-09-09' FROM t; +SELECT date('1996-9-10') < '1996-09-09' FROM t; +SELECT date('1996-9-10') <= '1996-09-09' FROM t; +SELECT date('1996-9-10') <> '1996-09-09' FROM t; +SELECT date('1996-09-09') = cast(null as string) FROM t; +SELECT date('1996-9-10') > cast(null as string) FROM t; +SELECT date('1996-9-10') >= cast(null as string) FROM t; +SELECT date('1996-9-10') < cast(null as string) FROM t; +SELECT date('1996-9-10') <= cast(null as string) FROM t; +SELECT date('1996-9-10') <> cast(null as string) FROM t; +SELECT '1996-09-09 12:12:12.4' = timestamp('1996-09-09 12:12:12.4') FROM t; +SELECT '1996-09-09 12:12:12.5' > timestamp('1996-09-09 12:12:12.4') FROM t; +SELECT '1996-09-09 12:12:12.5' >= timestamp('1996-09-09 12:12:12.4') FROM t; +SELECT '1996-09-09 12:12:12.5' < timestamp('1996-09-09 12:12:12.4') FROM t; +SELECT '1996-09-09 12:12:12.5' <= timestamp('1996-09-09 12:12:12.4') FROM t; +SELECT '1996-09-09 12:12:12.5' <> timestamp('1996-09-09 12:12:12.4') FROM t; +SELECT cast(null as string) = timestamp('1996-09-09 12:12:12.4') FROM t; +SELECT cast(null as string) > timestamp('1996-09-09 12:12:12.4') FROM t; +SELECT cast(null as string) >= timestamp('1996-09-09 12:12:12.4') FROM t; +SELECT cast(null as string) < timestamp('1996-09-09 12:12:12.4') FROM t; +SELECT cast(null as string) <= timestamp('1996-09-09 12:12:12.4') FROM t; +SELECT cast(null as string) <> timestamp('1996-09-09 12:12:12.4') FROM t; +SELECT timestamp('1996-09-09 12:12:12.4' )= '1996-09-09 12:12:12.4' FROM t; +SELECT timestamp('1996-09-09 12:12:12.5' )> '1996-09-09 12:12:12.4' FROM t; +SELECT timestamp('1996-09-09 12:12:12.5' )>= '1996-09-09 12:12:12.4' FROM t; +SELECT timestamp('1996-09-09 12:12:12.5' )< '1996-09-09 12:12:12.4' FROM t; +SELECT timestamp('1996-09-09 12:12:12.5' )<= '1996-09-09 12:12:12.4' FROM t; +SELECT timestamp('1996-09-09 12:12:12.5' )<> '1996-09-09 12:12:12.4' FROM t; +SELECT timestamp('1996-09-09 12:12:12.4' )= cast(null as string) FROM t; +SELECT timestamp('1996-09-09 12:12:12.5' )> cast(null as string) FROM t; +SELECT timestamp('1996-09-09 12:12:12.5' )>= cast(null as string) FROM t; +SELECT timestamp('1996-09-09 12:12:12.5' )< cast(null as string) FROM t; +SELECT timestamp('1996-09-09 12:12:12.5' )<= cast(null as string) FROM t; +SELECT timestamp('1996-09-09 12:12:12.5' )<> cast(null as string) FROM t; +SELECT ' ' = X'0020' FROM t; +SELECT ' ' > X'001F' FROM t; +SELECT ' ' >= X'001F' FROM t; +SELECT ' ' < X'001F' FROM t; +SELECT ' ' <= X'001F' FROM t; +SELECT ' ' <> X'001F' FROM t; +SELECT cast(null as string) = X'0020' FROM t; +SELECT cast(null as string) > X'001F' FROM t; +SELECT cast(null as string) >= X'001F' FROM t; +SELECT cast(null as string) < X'001F' FROM t; +SELECT cast(null as string) <= X'001F' FROM t; +SELECT cast(null as string) <> X'001F' FROM t; +SELECT X'0020' = ' ' FROM t; +SELECT X'001F' > ' ' FROM t; +SELECT X'001F' >= ' ' FROM t; +SELECT X'001F' < ' ' FROM t; +SELECT X'001F' <= ' ' FROM t; +SELECT X'001F' <> ' ' FROM t; +SELECT X'0020' = cast(null as string) FROM t; +SELECT X'001F' > cast(null as string) FROM t; +SELECT X'001F' >= cast(null as string) FROM t; +SELECT X'001F' < cast(null as string) FROM t; +SELECT X'001F' <= cast(null as string) FROM t; +SELECT X'001F' <> cast(null as string) FROM t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/booleanEquality.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/booleanEquality.sql new file mode 100644 index 000000000000..442f2355f8e3 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/booleanEquality.sql @@ -0,0 +1,122 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +CREATE TEMPORARY VIEW t AS SELECT 1; + +SELECT true = cast(1 as tinyint) FROM t; +SELECT true = cast(1 as smallint) FROM t; +SELECT true = cast(1 as int) FROM t; +SELECT true = cast(1 as bigint) FROM t; +SELECT true = cast(1 as float) FROM t; +SELECT true = cast(1 as double) FROM t; +SELECT true = cast(1 as decimal(10, 0)) FROM t; +SELECT true = cast(1 as string) FROM t; +SELECT true = cast('1' as binary) FROM t; +SELECT true = cast(1 as boolean) FROM t; +SELECT true = cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT true = cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT true <=> cast(1 as tinyint) FROM t; +SELECT true <=> cast(1 as smallint) FROM t; +SELECT true <=> cast(1 as int) FROM t; +SELECT true <=> cast(1 as bigint) FROM t; +SELECT true <=> cast(1 as float) FROM t; +SELECT true <=> cast(1 as double) FROM t; +SELECT true <=> cast(1 as decimal(10, 0)) FROM t; +SELECT true <=> cast(1 as string) FROM t; +SELECT true <=> cast('1' as binary) FROM t; +SELECT true <=> cast(1 as boolean) FROM t; +SELECT true <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT true <=> cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) = true FROM t; +SELECT cast(1 as smallint) = true FROM t; +SELECT cast(1 as int) = true FROM t; +SELECT cast(1 as bigint) = true FROM t; +SELECT cast(1 as float) = true FROM t; +SELECT cast(1 as double) = true FROM t; +SELECT cast(1 as decimal(10, 0)) = true FROM t; +SELECT cast(1 as string) = true FROM t; +SELECT cast('1' as binary) = true FROM t; +SELECT cast(1 as boolean) = true FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = true FROM t; +SELECT cast('2017-12-11 09:30:00' as date) = true FROM t; + +SELECT cast(1 as tinyint) <=> true FROM t; +SELECT cast(1 as smallint) <=> true FROM t; +SELECT cast(1 as int) <=> true FROM t; +SELECT cast(1 as bigint) <=> true FROM t; +SELECT cast(1 as float) <=> true FROM t; +SELECT cast(1 as double) <=> true FROM t; +SELECT cast(1 as decimal(10, 0)) <=> true FROM t; +SELECT cast(1 as string) <=> true FROM t; +SELECT cast('1' as binary) <=> true FROM t; +SELECT cast(1 as boolean) <=> true FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> true FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <=> true FROM t; + +SELECT false = cast(0 as tinyint) FROM t; +SELECT false = cast(0 as smallint) FROM t; +SELECT false = cast(0 as int) FROM t; +SELECT false = cast(0 as bigint) FROM t; +SELECT false = cast(0 as float) FROM t; +SELECT false = cast(0 as double) FROM t; +SELECT false = cast(0 as decimal(10, 0)) FROM t; +SELECT false = cast(0 as string) FROM t; +SELECT false = cast('0' as binary) FROM t; +SELECT false = cast(0 as boolean) FROM t; +SELECT false = cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT false = cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT false <=> cast(0 as tinyint) FROM t; +SELECT false <=> cast(0 as smallint) FROM t; +SELECT false <=> cast(0 as int) FROM t; +SELECT false <=> cast(0 as bigint) FROM t; +SELECT false <=> cast(0 as float) FROM t; +SELECT false <=> cast(0 as double) FROM t; +SELECT false <=> cast(0 as decimal(10, 0)) FROM t; +SELECT false <=> cast(0 as string) FROM t; +SELECT false <=> cast('0' as binary) FROM t; +SELECT false <=> cast(0 as boolean) FROM t; +SELECT false <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT false <=> cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(0 as tinyint) = false FROM t; +SELECT cast(0 as smallint) = false FROM t; +SELECT cast(0 as int) = false FROM t; +SELECT cast(0 as bigint) = false FROM t; +SELECT cast(0 as float) = false FROM t; +SELECT cast(0 as double) = false FROM t; +SELECT cast(0 as decimal(10, 0)) = false FROM t; +SELECT cast(0 as string) = false FROM t; +SELECT cast('0' as binary) = false FROM t; +SELECT cast(0 as boolean) = false FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = false FROM t; +SELECT cast('2017-12-11 09:30:00' as date) = false FROM t; + +SELECT cast(0 as tinyint) <=> false FROM t; +SELECT cast(0 as smallint) <=> false FROM t; +SELECT cast(0 as int) <=> false FROM t; +SELECT cast(0 as bigint) <=> false FROM t; +SELECT cast(0 as float) <=> false FROM t; +SELECT cast(0 as double) <=> false FROM t; +SELECT cast(0 as decimal(10, 0)) <=> false FROM t; +SELECT cast(0 as string) <=> false FROM t; +SELECT cast('0' as binary) <=> false FROM t; +SELECT cast(0 as boolean) <=> false FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> false FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <=> false FROM t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/caseWhenCoercion.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/caseWhenCoercion.sql new file mode 100644 index 000000000000..a780529fdca8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/caseWhenCoercion.sql @@ -0,0 +1,174 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +CREATE TEMPORARY VIEW t AS SELECT 1; + +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as tinyint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as smallint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as int) END FROM t; +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as bigint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as float) END FROM t; +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as double) END FROM t; +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as decimal(10, 0)) END FROM t; +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as string) END FROM t; +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2' as binary) END FROM t; +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as boolean) END FROM t; +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t; +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00' as date) END FROM t; + +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as tinyint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as smallint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as int) END FROM t; +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as bigint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as float) END FROM t; +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as double) END FROM t; +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as decimal(10, 0)) END FROM t; +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as string) END FROM t; +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2' as binary) END FROM t; +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as boolean) END FROM t; +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t; +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00' as date) END FROM t; + +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as tinyint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as smallint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as int) END FROM t; +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as bigint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as float) END FROM t; +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as double) END FROM t; +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as decimal(10, 0)) END FROM t; +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as string) END FROM t; +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2' as binary) END FROM t; +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as boolean) END FROM t; +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t; +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2017-12-11 09:30:00' as date) END FROM t; + +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as tinyint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as smallint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as int) END FROM t; +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as bigint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as float) END FROM t; +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as double) END FROM t; +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as decimal(10, 0)) END FROM t; +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as string) END FROM t; +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2' as binary) END FROM t; +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as boolean) END FROM t; +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t; +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2017-12-11 09:30:00' as date) END FROM t; + +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as tinyint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as smallint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as int) END FROM t; +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as bigint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as float) END FROM t; +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as double) END FROM t; +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as decimal(10, 0)) END FROM t; +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as string) END FROM t; +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2' as binary) END FROM t; +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as boolean) END FROM t; +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t; +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2017-12-11 09:30:00' as date) END FROM t; + +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as tinyint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as smallint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as int) END FROM t; +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as bigint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as float) END FROM t; +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as double) END FROM t; +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as decimal(10, 0)) END FROM t; +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as string) END FROM t; +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2' as binary) END FROM t; +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as boolean) END FROM t; +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t; +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2017-12-11 09:30:00' as date) END FROM t; + +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as tinyint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as smallint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as int) END FROM t; +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as bigint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as float) END FROM t; +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as double) END FROM t; +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as decimal(10, 0)) END FROM t; +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as string) END FROM t; +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2' as binary) END FROM t; +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as boolean) END FROM t; +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t; +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2017-12-11 09:30:00' as date) END FROM t; + +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as tinyint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as smallint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as int) END FROM t; +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as bigint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as float) END FROM t; +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as double) END FROM t; +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as decimal(10, 0)) END FROM t; +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as string) END FROM t; +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2' as binary) END FROM t; +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as boolean) END FROM t; +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t; +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2017-12-11 09:30:00' as date) END FROM t; + +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as tinyint) END FROM t; +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as smallint) END FROM t; +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as int) END FROM t; +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as bigint) END FROM t; +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as float) END FROM t; +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as double) END FROM t; +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as decimal(10, 0)) END FROM t; +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as string) END FROM t; +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast('2' as binary) END FROM t; +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as boolean) END FROM t; +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t; +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast('2017-12-11 09:30:00' as date) END FROM t; + +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as tinyint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as smallint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as int) END FROM t; +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as bigint) END FROM t; +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as float) END FROM t; +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as double) END FROM t; +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as decimal(10, 0)) END FROM t; +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as string) END FROM t; +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2' as binary) END FROM t; +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as boolean) END FROM t; +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t; +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2017-12-11 09:30:00' as date) END FROM t; + +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as tinyint) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as smallint) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as int) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as bigint) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as float) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as double) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as decimal(10, 0)) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as string) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast('2' as binary) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as boolean) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast('2017-12-11 09:30:00' as date) END FROM t; + +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as tinyint) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as smallint) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as int) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as bigint) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as float) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as double) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as decimal(10, 0)) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as string) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast('2' as binary) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as boolean) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t; +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast('2017-12-11 09:30:00' as date) END FROM t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/concat.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/concat.sql new file mode 100644 index 000000000000..99f46dd19d0e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/concat.sql @@ -0,0 +1,157 @@ +-- Concatenate mixed inputs (output type is string) +SELECT (col1 || col2 || col3) col +FROM ( + SELECT + id col1, + string(id + 1) col2, + encode(string(id + 2), 'utf-8') col3 + FROM range(10) +); + +SELECT ((col1 || col2) || (col3 || col4) || col5) col +FROM ( + SELECT + 'prefix_' col1, + id col2, + string(id + 1) col3, + encode(string(id + 2), 'utf-8') col4, + CAST(id AS DOUBLE) col5 + FROM range(10) +); + +SELECT ((col1 || col2) || (col3 || col4)) col +FROM ( + SELECT + string(id) col1, + string(id + 1) col2, + encode(string(id + 2), 'utf-8') col3, + encode(string(id + 3), 'utf-8') col4 + FROM range(10) +); + +-- turn on concatBinaryAsString +set spark.sql.function.concatBinaryAsString=true; + +SELECT (col1 || col2) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2 + FROM range(10) +); + +SELECT (col1 || col2 || col3 || col4) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2, + encode(string(id + 2), 'utf-8') col3, + encode(string(id + 3), 'utf-8') col4 + FROM range(10) +); + +SELECT ((col1 || col2) || (col3 || col4)) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2, + encode(string(id + 2), 'utf-8') col3, + encode(string(id + 3), 'utf-8') col4 + FROM range(10) +); + +-- turn off concatBinaryAsString +set spark.sql.function.concatBinaryAsString=false; + +-- Concatenate binary inputs (output type is binary) +SELECT (col1 || col2) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2 + FROM range(10) +); + +SELECT (col1 || col2 || col3 || col4) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2, + encode(string(id + 2), 'utf-8') col3, + encode(string(id + 3), 'utf-8') col4 + FROM range(10) +); + +SELECT ((col1 || col2) || (col3 || col4)) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2, + encode(string(id + 2), 'utf-8') col3, + encode(string(id + 3), 'utf-8') col4 + FROM range(10) +); + +CREATE TEMPORARY VIEW various_arrays AS SELECT * FROM VALUES ( + array(true, false), array(true), + array(2Y, 1Y), array(3Y, 4Y), + array(2S, 1S), array(3S, 4S), + array(2, 1), array(3, 4), + array(2L, 1L), array(3L, 4L), + array(9223372036854775809, 9223372036854775808), array(9223372036854775808, 9223372036854775809), + array(2.0D, 1.0D), array(3.0D, 4.0D), + array(float(2.0), float(1.0)), array(float(3.0), float(4.0)), + array(date '2016-03-14', date '2016-03-13'), array(date '2016-03-12', date '2016-03-11'), + array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), + array(timestamp '2016-11-11 20:54:00.000'), + array('a', 'b'), array('c', 'd'), + array(array('a', 'b'), array('c', 'd')), array(array('e'), array('f')), + array(struct('a', 1), struct('b', 2)), array(struct('c', 3), struct('d', 4)), + array(map('a', 1), map('b', 2)), array(map('c', 3), map('d', 4)) +) AS various_arrays( + boolean_array1, boolean_array2, + tinyint_array1, tinyint_array2, + smallint_array1, smallint_array2, + int_array1, int_array2, + bigint_array1, bigint_array2, + decimal_array1, decimal_array2, + double_array1, double_array2, + float_array1, float_array2, + date_array1, data_array2, + timestamp_array1, timestamp_array2, + string_array1, string_array2, + array_array1, array_array2, + struct_array1, struct_array2, + map_array1, map_array2 +); + +-- Concatenate arrays of the same type +SELECT + (boolean_array1 || boolean_array2) boolean_array, + (tinyint_array1 || tinyint_array2) tinyint_array, + (smallint_array1 || smallint_array2) smallint_array, + (int_array1 || int_array2) int_array, + (bigint_array1 || bigint_array2) bigint_array, + (decimal_array1 || decimal_array2) decimal_array, + (double_array1 || double_array2) double_array, + (float_array1 || float_array2) float_array, + (date_array1 || data_array2) data_array, + (timestamp_array1 || timestamp_array2) timestamp_array, + (string_array1 || string_array2) string_array, + (array_array1 || array_array2) array_array, + (struct_array1 || struct_array2) struct_array, + (map_array1 || map_array2) map_array +FROM various_arrays; + +-- Concatenate arrays of different types +SELECT + (tinyint_array1 || smallint_array2) ts_array, + (smallint_array1 || int_array2) si_array, + (int_array1 || bigint_array2) ib_array, + (bigint_array1 || decimal_array2) bd_array, + (decimal_array1 || double_array2) dd_array, + (double_array1 || float_array2) df_array, + (string_array1 || data_array2) std_array, + (timestamp_array1 || string_array2) tst_array, + (string_array1 || int_array2) sti_array +FROM various_arrays; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/dateTimeOperations.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/dateTimeOperations.sql new file mode 100644 index 000000000000..1e9822186796 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/dateTimeOperations.sql @@ -0,0 +1,60 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +CREATE TEMPORARY VIEW t AS SELECT 1; + +select cast(1 as tinyint) + interval 2 day; +select cast(1 as smallint) + interval 2 day; +select cast(1 as int) + interval 2 day; +select cast(1 as bigint) + interval 2 day; +select cast(1 as float) + interval 2 day; +select cast(1 as double) + interval 2 day; +select cast(1 as decimal(10, 0)) + interval 2 day; +select cast('2017-12-11' as string) + interval 2 day; +select cast('2017-12-11 09:30:00' as string) + interval 2 day; +select cast('1' as binary) + interval 2 day; +select cast(1 as boolean) + interval 2 day; +select cast('2017-12-11 09:30:00.0' as timestamp) + interval 2 day; +select cast('2017-12-11 09:30:00' as date) + interval 2 day; + +select interval 2 day + cast(1 as tinyint); +select interval 2 day + cast(1 as smallint); +select interval 2 day + cast(1 as int); +select interval 2 day + cast(1 as bigint); +select interval 2 day + cast(1 as float); +select interval 2 day + cast(1 as double); +select interval 2 day + cast(1 as decimal(10, 0)); +select interval 2 day + cast('2017-12-11' as string); +select interval 2 day + cast('2017-12-11 09:30:00' as string); +select interval 2 day + cast('1' as binary); +select interval 2 day + cast(1 as boolean); +select interval 2 day + cast('2017-12-11 09:30:00.0' as timestamp); +select interval 2 day + cast('2017-12-11 09:30:00' as date); + +select cast(1 as tinyint) - interval 2 day; +select cast(1 as smallint) - interval 2 day; +select cast(1 as int) - interval 2 day; +select cast(1 as bigint) - interval 2 day; +select cast(1 as float) - interval 2 day; +select cast(1 as double) - interval 2 day; +select cast(1 as decimal(10, 0)) - interval 2 day; +select cast('2017-12-11' as string) - interval 2 day; +select cast('2017-12-11 09:30:00' as string) - interval 2 day; +select cast('1' as binary) - interval 2 day; +select cast(1 as boolean) - interval 2 day; +select cast('2017-12-11 09:30:00.0' as timestamp) - interval 2 day; +select cast('2017-12-11 09:30:00' as date) - interval 2 day; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/decimalPrecision.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/decimalPrecision.sql new file mode 100644 index 000000000000..8b04864b18ce --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/decimalPrecision.sql @@ -0,0 +1,1448 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +CREATE TEMPORARY VIEW t AS SELECT 1; + +SELECT cast(1 as tinyint) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT pmod(cast(1 as tinyint), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast(1 as tinyint), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast(1 as tinyint), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as tinyint), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast(1 as smallint), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast(1 as smallint), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast(1 as smallint), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as smallint), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast(1 as int), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast(1 as int), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast(1 as int), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as int), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast(1 as bigint), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast(1 as bigint), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast(1 as bigint), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as bigint), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast(1 as float), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast(1 as float), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast(1 as float), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as float), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast(1 as double), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast(1 as double), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast(1 as double), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as double), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast('1' as binary), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast('1' as binary), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast('1' as binary), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast('1' as binary), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(3, 0))) FROM t; +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(5, 0))) FROM t; +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(20, 0))) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as tinyint)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as tinyint)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as tinyint)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as tinyint)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as smallint)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as smallint)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as smallint)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as smallint)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as int)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as int)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as int)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as int)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as bigint)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as bigint)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as bigint)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as bigint)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as float)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as float)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as float)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as float)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as double)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as double)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as double)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as double)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(10, 0))) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as decimal(10, 0))) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as string)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as string)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as string)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as string)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast('1' as binary)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast('1' as binary)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast('1' as binary)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast('1' as binary)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as boolean)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as boolean)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as boolean)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as boolean)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; + +SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00' as date)) FROM t; +SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00' as date)) FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t; +SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast(1 as tinyint) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as tinyint) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as tinyint) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as smallint) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as smallint) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as smallint) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as int) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as int) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as int) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as bigint) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as bigint) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as bigint) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as float) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as float) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as float) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as double) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as double) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as double) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('1' as binary) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast('1' as binary) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast('1' as binary) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(3, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(5, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(20, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast(1 as tinyint) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast(1 as smallint) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast(1 as int) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as int) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast(1 as int) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast(1 as bigint) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast(1 as float) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as float) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast(1 as float) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast(1 as double) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as double) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast(1 as double) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast(1 as decimal(10, 0)) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast(1 as string) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as string) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast(1 as string) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast('1' as binary) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast('1' as binary) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast('1' as binary) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast(1 as boolean) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; + +SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t; +SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/division.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/division.sql new file mode 100644 index 000000000000..d669740ddd9c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/division.sql @@ -0,0 +1,174 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +CREATE TEMPORARY VIEW t AS SELECT 1; + +SELECT cast(1 as tinyint) / cast(1 as tinyint) FROM t; +SELECT cast(1 as tinyint) / cast(1 as smallint) FROM t; +SELECT cast(1 as tinyint) / cast(1 as int) FROM t; +SELECT cast(1 as tinyint) / cast(1 as bigint) FROM t; +SELECT cast(1 as tinyint) / cast(1 as float) FROM t; +SELECT cast(1 as tinyint) / cast(1 as double) FROM t; +SELECT cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) / cast(1 as string) FROM t; +SELECT cast(1 as tinyint) / cast('1' as binary) FROM t; +SELECT cast(1 as tinyint) / cast(1 as boolean) FROM t; +SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as smallint) / cast(1 as tinyint) FROM t; +SELECT cast(1 as smallint) / cast(1 as smallint) FROM t; +SELECT cast(1 as smallint) / cast(1 as int) FROM t; +SELECT cast(1 as smallint) / cast(1 as bigint) FROM t; +SELECT cast(1 as smallint) / cast(1 as float) FROM t; +SELECT cast(1 as smallint) / cast(1 as double) FROM t; +SELECT cast(1 as smallint) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) / cast(1 as string) FROM t; +SELECT cast(1 as smallint) / cast('1' as binary) FROM t; +SELECT cast(1 as smallint) / cast(1 as boolean) FROM t; +SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as int) / cast(1 as tinyint) FROM t; +SELECT cast(1 as int) / cast(1 as smallint) FROM t; +SELECT cast(1 as int) / cast(1 as int) FROM t; +SELECT cast(1 as int) / cast(1 as bigint) FROM t; +SELECT cast(1 as int) / cast(1 as float) FROM t; +SELECT cast(1 as int) / cast(1 as double) FROM t; +SELECT cast(1 as int) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) / cast(1 as string) FROM t; +SELECT cast(1 as int) / cast('1' as binary) FROM t; +SELECT cast(1 as int) / cast(1 as boolean) FROM t; +SELECT cast(1 as int) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as int) / cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as bigint) / cast(1 as tinyint) FROM t; +SELECT cast(1 as bigint) / cast(1 as smallint) FROM t; +SELECT cast(1 as bigint) / cast(1 as int) FROM t; +SELECT cast(1 as bigint) / cast(1 as bigint) FROM t; +SELECT cast(1 as bigint) / cast(1 as float) FROM t; +SELECT cast(1 as bigint) / cast(1 as double) FROM t; +SELECT cast(1 as bigint) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) / cast(1 as string) FROM t; +SELECT cast(1 as bigint) / cast('1' as binary) FROM t; +SELECT cast(1 as bigint) / cast(1 as boolean) FROM t; +SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as float) / cast(1 as tinyint) FROM t; +SELECT cast(1 as float) / cast(1 as smallint) FROM t; +SELECT cast(1 as float) / cast(1 as int) FROM t; +SELECT cast(1 as float) / cast(1 as bigint) FROM t; +SELECT cast(1 as float) / cast(1 as float) FROM t; +SELECT cast(1 as float) / cast(1 as double) FROM t; +SELECT cast(1 as float) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) / cast(1 as string) FROM t; +SELECT cast(1 as float) / cast('1' as binary) FROM t; +SELECT cast(1 as float) / cast(1 as boolean) FROM t; +SELECT cast(1 as float) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as float) / cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as double) / cast(1 as tinyint) FROM t; +SELECT cast(1 as double) / cast(1 as smallint) FROM t; +SELECT cast(1 as double) / cast(1 as int) FROM t; +SELECT cast(1 as double) / cast(1 as bigint) FROM t; +SELECT cast(1 as double) / cast(1 as float) FROM t; +SELECT cast(1 as double) / cast(1 as double) FROM t; +SELECT cast(1 as double) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) / cast(1 as string) FROM t; +SELECT cast(1 as double) / cast('1' as binary) FROM t; +SELECT cast(1 as double) / cast(1 as boolean) FROM t; +SELECT cast(1 as double) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as double) / cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as decimal(10, 0)) / cast(1 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as string) / cast(1 as tinyint) FROM t; +SELECT cast(1 as string) / cast(1 as smallint) FROM t; +SELECT cast(1 as string) / cast(1 as int) FROM t; +SELECT cast(1 as string) / cast(1 as bigint) FROM t; +SELECT cast(1 as string) / cast(1 as float) FROM t; +SELECT cast(1 as string) / cast(1 as double) FROM t; +SELECT cast(1 as string) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as string) / cast(1 as string) FROM t; +SELECT cast(1 as string) / cast('1' as binary) FROM t; +SELECT cast(1 as string) / cast(1 as boolean) FROM t; +SELECT cast(1 as string) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as string) / cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast('1' as binary) / cast(1 as tinyint) FROM t; +SELECT cast('1' as binary) / cast(1 as smallint) FROM t; +SELECT cast('1' as binary) / cast(1 as int) FROM t; +SELECT cast('1' as binary) / cast(1 as bigint) FROM t; +SELECT cast('1' as binary) / cast(1 as float) FROM t; +SELECT cast('1' as binary) / cast(1 as double) FROM t; +SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) / cast(1 as string) FROM t; +SELECT cast('1' as binary) / cast('1' as binary) FROM t; +SELECT cast('1' as binary) / cast(1 as boolean) FROM t; +SELECT cast('1' as binary) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast('1' as binary) / cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as boolean) / cast(1 as tinyint) FROM t; +SELECT cast(1 as boolean) / cast(1 as smallint) FROM t; +SELECT cast(1 as boolean) / cast(1 as int) FROM t; +SELECT cast(1 as boolean) / cast(1 as bigint) FROM t; +SELECT cast(1 as boolean) / cast(1 as float) FROM t; +SELECT cast(1 as boolean) / cast(1 as double) FROM t; +SELECT cast(1 as boolean) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast(1 as boolean) / cast(1 as string) FROM t; +SELECT cast(1 as boolean) / cast('1' as binary) FROM t; +SELECT cast(1 as boolean) / cast(1 as boolean) FROM t; +SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as tinyint) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as smallint) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as int) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as bigint) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as float) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as double) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as string) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('1' as binary) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as boolean) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as tinyint) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as smallint) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as int) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as bigint) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as float) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as double) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as string) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) / cast('1' as binary) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as boolean) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00' as date) FROM t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/elt.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/elt.sql new file mode 100644 index 000000000000..717616f91db0 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/elt.sql @@ -0,0 +1,44 @@ +-- Mixed inputs (output type is string) +SELECT elt(2, col1, col2, col3, col4, col5) col +FROM ( + SELECT + 'prefix_' col1, + id col2, + string(id + 1) col3, + encode(string(id + 2), 'utf-8') col4, + CAST(id AS DOUBLE) col5 + FROM range(10) +); + +SELECT elt(3, col1, col2, col3, col4) col +FROM ( + SELECT + string(id) col1, + string(id + 1) col2, + encode(string(id + 2), 'utf-8') col3, + encode(string(id + 3), 'utf-8') col4 + FROM range(10) +); + +-- turn on eltOutputAsString +set spark.sql.function.eltOutputAsString=true; + +SELECT elt(1, col1, col2) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2 + FROM range(10) +); + +-- turn off eltOutputAsString +set spark.sql.function.eltOutputAsString=false; + +-- Elt binary inputs (output type is binary) +SELECT elt(2, col1, col2) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2 + FROM range(10) +); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/ifCoercion.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/ifCoercion.sql new file mode 100644 index 000000000000..42597f169dae --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/ifCoercion.sql @@ -0,0 +1,174 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +CREATE TEMPORARY VIEW t AS SELECT 1; + +SELECT IF(true, cast(1 as tinyint), cast(2 as tinyint)) FROM t; +SELECT IF(true, cast(1 as tinyint), cast(2 as smallint)) FROM t; +SELECT IF(true, cast(1 as tinyint), cast(2 as int)) FROM t; +SELECT IF(true, cast(1 as tinyint), cast(2 as bigint)) FROM t; +SELECT IF(true, cast(1 as tinyint), cast(2 as float)) FROM t; +SELECT IF(true, cast(1 as tinyint), cast(2 as double)) FROM t; +SELECT IF(true, cast(1 as tinyint), cast(2 as decimal(10, 0))) FROM t; +SELECT IF(true, cast(1 as tinyint), cast(2 as string)) FROM t; +SELECT IF(true, cast(1 as tinyint), cast('2' as binary)) FROM t; +SELECT IF(true, cast(1 as tinyint), cast(2 as boolean)) FROM t; +SELECT IF(true, cast(1 as tinyint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT IF(true, cast(1 as tinyint), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT IF(true, cast(1 as smallint), cast(2 as tinyint)) FROM t; +SELECT IF(true, cast(1 as smallint), cast(2 as smallint)) FROM t; +SELECT IF(true, cast(1 as smallint), cast(2 as int)) FROM t; +SELECT IF(true, cast(1 as smallint), cast(2 as bigint)) FROM t; +SELECT IF(true, cast(1 as smallint), cast(2 as float)) FROM t; +SELECT IF(true, cast(1 as smallint), cast(2 as double)) FROM t; +SELECT IF(true, cast(1 as smallint), cast(2 as decimal(10, 0))) FROM t; +SELECT IF(true, cast(1 as smallint), cast(2 as string)) FROM t; +SELECT IF(true, cast(1 as smallint), cast('2' as binary)) FROM t; +SELECT IF(true, cast(1 as smallint), cast(2 as boolean)) FROM t; +SELECT IF(true, cast(1 as smallint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT IF(true, cast(1 as smallint), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT IF(true, cast(1 as int), cast(2 as tinyint)) FROM t; +SELECT IF(true, cast(1 as int), cast(2 as smallint)) FROM t; +SELECT IF(true, cast(1 as int), cast(2 as int)) FROM t; +SELECT IF(true, cast(1 as int), cast(2 as bigint)) FROM t; +SELECT IF(true, cast(1 as int), cast(2 as float)) FROM t; +SELECT IF(true, cast(1 as int), cast(2 as double)) FROM t; +SELECT IF(true, cast(1 as int), cast(2 as decimal(10, 0))) FROM t; +SELECT IF(true, cast(1 as int), cast(2 as string)) FROM t; +SELECT IF(true, cast(1 as int), cast('2' as binary)) FROM t; +SELECT IF(true, cast(1 as int), cast(2 as boolean)) FROM t; +SELECT IF(true, cast(1 as int), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT IF(true, cast(1 as int), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT IF(true, cast(1 as bigint), cast(2 as tinyint)) FROM t; +SELECT IF(true, cast(1 as bigint), cast(2 as smallint)) FROM t; +SELECT IF(true, cast(1 as bigint), cast(2 as int)) FROM t; +SELECT IF(true, cast(1 as bigint), cast(2 as bigint)) FROM t; +SELECT IF(true, cast(1 as bigint), cast(2 as float)) FROM t; +SELECT IF(true, cast(1 as bigint), cast(2 as double)) FROM t; +SELECT IF(true, cast(1 as bigint), cast(2 as decimal(10, 0))) FROM t; +SELECT IF(true, cast(1 as bigint), cast(2 as string)) FROM t; +SELECT IF(true, cast(1 as bigint), cast('2' as binary)) FROM t; +SELECT IF(true, cast(1 as bigint), cast(2 as boolean)) FROM t; +SELECT IF(true, cast(1 as bigint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT IF(true, cast(1 as bigint), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT IF(true, cast(1 as float), cast(2 as tinyint)) FROM t; +SELECT IF(true, cast(1 as float), cast(2 as smallint)) FROM t; +SELECT IF(true, cast(1 as float), cast(2 as int)) FROM t; +SELECT IF(true, cast(1 as float), cast(2 as bigint)) FROM t; +SELECT IF(true, cast(1 as float), cast(2 as float)) FROM t; +SELECT IF(true, cast(1 as float), cast(2 as double)) FROM t; +SELECT IF(true, cast(1 as float), cast(2 as decimal(10, 0))) FROM t; +SELECT IF(true, cast(1 as float), cast(2 as string)) FROM t; +SELECT IF(true, cast(1 as float), cast('2' as binary)) FROM t; +SELECT IF(true, cast(1 as float), cast(2 as boolean)) FROM t; +SELECT IF(true, cast(1 as float), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT IF(true, cast(1 as float), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT IF(true, cast(1 as double), cast(2 as tinyint)) FROM t; +SELECT IF(true, cast(1 as double), cast(2 as smallint)) FROM t; +SELECT IF(true, cast(1 as double), cast(2 as int)) FROM t; +SELECT IF(true, cast(1 as double), cast(2 as bigint)) FROM t; +SELECT IF(true, cast(1 as double), cast(2 as float)) FROM t; +SELECT IF(true, cast(1 as double), cast(2 as double)) FROM t; +SELECT IF(true, cast(1 as double), cast(2 as decimal(10, 0))) FROM t; +SELECT IF(true, cast(1 as double), cast(2 as string)) FROM t; +SELECT IF(true, cast(1 as double), cast('2' as binary)) FROM t; +SELECT IF(true, cast(1 as double), cast(2 as boolean)) FROM t; +SELECT IF(true, cast(1 as double), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT IF(true, cast(1 as double), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as tinyint)) FROM t; +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as smallint)) FROM t; +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as int)) FROM t; +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as bigint)) FROM t; +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as float)) FROM t; +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as double)) FROM t; +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as decimal(10, 0))) FROM t; +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as string)) FROM t; +SELECT IF(true, cast(1 as decimal(10, 0)), cast('2' as binary)) FROM t; +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as boolean)) FROM t; +SELECT IF(true, cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT IF(true, cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT IF(true, cast(1 as string), cast(2 as tinyint)) FROM t; +SELECT IF(true, cast(1 as string), cast(2 as smallint)) FROM t; +SELECT IF(true, cast(1 as string), cast(2 as int)) FROM t; +SELECT IF(true, cast(1 as string), cast(2 as bigint)) FROM t; +SELECT IF(true, cast(1 as string), cast(2 as float)) FROM t; +SELECT IF(true, cast(1 as string), cast(2 as double)) FROM t; +SELECT IF(true, cast(1 as string), cast(2 as decimal(10, 0))) FROM t; +SELECT IF(true, cast(1 as string), cast(2 as string)) FROM t; +SELECT IF(true, cast(1 as string), cast('2' as binary)) FROM t; +SELECT IF(true, cast(1 as string), cast(2 as boolean)) FROM t; +SELECT IF(true, cast(1 as string), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT IF(true, cast(1 as string), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT IF(true, cast('1' as binary), cast(2 as tinyint)) FROM t; +SELECT IF(true, cast('1' as binary), cast(2 as smallint)) FROM t; +SELECT IF(true, cast('1' as binary), cast(2 as int)) FROM t; +SELECT IF(true, cast('1' as binary), cast(2 as bigint)) FROM t; +SELECT IF(true, cast('1' as binary), cast(2 as float)) FROM t; +SELECT IF(true, cast('1' as binary), cast(2 as double)) FROM t; +SELECT IF(true, cast('1' as binary), cast(2 as decimal(10, 0))) FROM t; +SELECT IF(true, cast('1' as binary), cast(2 as string)) FROM t; +SELECT IF(true, cast('1' as binary), cast('2' as binary)) FROM t; +SELECT IF(true, cast('1' as binary), cast(2 as boolean)) FROM t; +SELECT IF(true, cast('1' as binary), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT IF(true, cast('1' as binary), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT IF(true, cast(1 as boolean), cast(2 as tinyint)) FROM t; +SELECT IF(true, cast(1 as boolean), cast(2 as smallint)) FROM t; +SELECT IF(true, cast(1 as boolean), cast(2 as int)) FROM t; +SELECT IF(true, cast(1 as boolean), cast(2 as bigint)) FROM t; +SELECT IF(true, cast(1 as boolean), cast(2 as float)) FROM t; +SELECT IF(true, cast(1 as boolean), cast(2 as double)) FROM t; +SELECT IF(true, cast(1 as boolean), cast(2 as decimal(10, 0))) FROM t; +SELECT IF(true, cast(1 as boolean), cast(2 as string)) FROM t; +SELECT IF(true, cast(1 as boolean), cast('2' as binary)) FROM t; +SELECT IF(true, cast(1 as boolean), cast(2 as boolean)) FROM t; +SELECT IF(true, cast(1 as boolean), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT IF(true, cast(1 as boolean), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as tinyint)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as smallint)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as int)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as bigint)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as float)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as double)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as decimal(10, 0))) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as string)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast('2' as binary)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as boolean)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as tinyint)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as smallint)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as int)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as bigint)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as float)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as double)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as decimal(10, 0))) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as string)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast('2' as binary)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as boolean)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast('2017-12-11 09:30:00' as date)) FROM t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/implicitTypeCasts.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/implicitTypeCasts.sql new file mode 100644 index 000000000000..6de22b8b7c3d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/implicitTypeCasts.sql @@ -0,0 +1,72 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +-- ImplicitTypeCasts + +CREATE TEMPORARY VIEW t AS SELECT 1; + +SELECT 1 + '2' FROM t; +SELECT 1 - '2' FROM t; +SELECT 1 * '2' FROM t; +SELECT 4 / '2' FROM t; +SELECT 1.1 + '2' FROM t; +SELECT 1.1 - '2' FROM t; +SELECT 1.1 * '2' FROM t; +SELECT 4.4 / '2' FROM t; +SELECT 1.1 + '2.2' FROM t; +SELECT 1.1 - '2.2' FROM t; +SELECT 1.1 * '2.2' FROM t; +SELECT 4.4 / '2.2' FROM t; + +-- concatenation +SELECT '$' || cast(1 as smallint) || '$' FROM t; +SELECT '$' || 1 || '$' FROM t; +SELECT '$' || cast(1 as bigint) || '$' FROM t; +SELECT '$' || cast(1.1 as float) || '$' FROM t; +SELECT '$' || cast(1.1 as double) || '$' FROM t; +SELECT '$' || 1.1 || '$' FROM t; +SELECT '$' || cast(1.1 as decimal(8,3)) || '$' FROM t; +SELECT '$' || 'abcd' || '$' FROM t; +SELECT '$' || date('1996-09-09') || '$' FROM t; +SELECT '$' || timestamp('1996-09-09 10:11:12.4' )|| '$' FROM t; + +-- length functions +SELECT length(cast(1 as smallint)) FROM t; +SELECT length(cast(1 as int)) FROM t; +SELECT length(cast(1 as bigint)) FROM t; +SELECT length(cast(1.1 as float)) FROM t; +SELECT length(cast(1.1 as double)) FROM t; +SELECT length(1.1) FROM t; +SELECT length(cast(1.1 as decimal(8,3))) FROM t; +SELECT length('four') FROM t; +SELECT length(date('1996-09-10')) FROM t; +SELECT length(timestamp('1996-09-10 10:11:12.4')) FROM t; + +-- extract +SELECT year( '1996-01-10') FROM t; +SELECT month( '1996-01-10') FROM t; +SELECT day( '1996-01-10') FROM t; +SELECT hour( '10:11:12') FROM t; +SELECT minute( '10:11:12') FROM t; +SELECT second( '10:11:12') FROM t; + +-- like +select 1 like '%' FROM t; +select date('1996-09-10') like '19%' FROM t; +select '1' like 1 FROM t; +select '1 ' like 1 FROM t; +select '1996-09-10' like date('1996-09-10') FROM t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/inConversion.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/inConversion.sql new file mode 100644 index 000000000000..39dbe7268fba --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/inConversion.sql @@ -0,0 +1,330 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +CREATE TEMPORARY VIEW t AS SELECT 1; + +SELECT cast(1 as tinyint) in (cast(1 as tinyint)) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as smallint)) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as int)) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as bigint)) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as float)) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as double)) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as decimal(10, 0))) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as string)) FROM t; +SELECT cast(1 as tinyint) in (cast('1' as binary)) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as boolean)) FROM t; +SELECT cast(1 as tinyint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast(1 as tinyint) in (cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast(1 as smallint) in (cast(1 as tinyint)) FROM t; +SELECT cast(1 as smallint) in (cast(1 as smallint)) FROM t; +SELECT cast(1 as smallint) in (cast(1 as int)) FROM t; +SELECT cast(1 as smallint) in (cast(1 as bigint)) FROM t; +SELECT cast(1 as smallint) in (cast(1 as float)) FROM t; +SELECT cast(1 as smallint) in (cast(1 as double)) FROM t; +SELECT cast(1 as smallint) in (cast(1 as decimal(10, 0))) FROM t; +SELECT cast(1 as smallint) in (cast(1 as string)) FROM t; +SELECT cast(1 as smallint) in (cast('1' as binary)) FROM t; +SELECT cast(1 as smallint) in (cast(1 as boolean)) FROM t; +SELECT cast(1 as smallint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast(1 as smallint) in (cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast(1 as int) in (cast(1 as tinyint)) FROM t; +SELECT cast(1 as int) in (cast(1 as smallint)) FROM t; +SELECT cast(1 as int) in (cast(1 as int)) FROM t; +SELECT cast(1 as int) in (cast(1 as bigint)) FROM t; +SELECT cast(1 as int) in (cast(1 as float)) FROM t; +SELECT cast(1 as int) in (cast(1 as double)) FROM t; +SELECT cast(1 as int) in (cast(1 as decimal(10, 0))) FROM t; +SELECT cast(1 as int) in (cast(1 as string)) FROM t; +SELECT cast(1 as int) in (cast('1' as binary)) FROM t; +SELECT cast(1 as int) in (cast(1 as boolean)) FROM t; +SELECT cast(1 as int) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast(1 as int) in (cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast(1 as bigint) in (cast(1 as tinyint)) FROM t; +SELECT cast(1 as bigint) in (cast(1 as smallint)) FROM t; +SELECT cast(1 as bigint) in (cast(1 as int)) FROM t; +SELECT cast(1 as bigint) in (cast(1 as bigint)) FROM t; +SELECT cast(1 as bigint) in (cast(1 as float)) FROM t; +SELECT cast(1 as bigint) in (cast(1 as double)) FROM t; +SELECT cast(1 as bigint) in (cast(1 as decimal(10, 0))) FROM t; +SELECT cast(1 as bigint) in (cast(1 as string)) FROM t; +SELECT cast(1 as bigint) in (cast('1' as binary)) FROM t; +SELECT cast(1 as bigint) in (cast(1 as boolean)) FROM t; +SELECT cast(1 as bigint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast(1 as bigint) in (cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast(1 as float) in (cast(1 as tinyint)) FROM t; +SELECT cast(1 as float) in (cast(1 as smallint)) FROM t; +SELECT cast(1 as float) in (cast(1 as int)) FROM t; +SELECT cast(1 as float) in (cast(1 as bigint)) FROM t; +SELECT cast(1 as float) in (cast(1 as float)) FROM t; +SELECT cast(1 as float) in (cast(1 as double)) FROM t; +SELECT cast(1 as float) in (cast(1 as decimal(10, 0))) FROM t; +SELECT cast(1 as float) in (cast(1 as string)) FROM t; +SELECT cast(1 as float) in (cast('1' as binary)) FROM t; +SELECT cast(1 as float) in (cast(1 as boolean)) FROM t; +SELECT cast(1 as float) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast(1 as float) in (cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast(1 as double) in (cast(1 as tinyint)) FROM t; +SELECT cast(1 as double) in (cast(1 as smallint)) FROM t; +SELECT cast(1 as double) in (cast(1 as int)) FROM t; +SELECT cast(1 as double) in (cast(1 as bigint)) FROM t; +SELECT cast(1 as double) in (cast(1 as float)) FROM t; +SELECT cast(1 as double) in (cast(1 as double)) FROM t; +SELECT cast(1 as double) in (cast(1 as decimal(10, 0))) FROM t; +SELECT cast(1 as double) in (cast(1 as string)) FROM t; +SELECT cast(1 as double) in (cast('1' as binary)) FROM t; +SELECT cast(1 as double) in (cast(1 as boolean)) FROM t; +SELECT cast(1 as double) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast(1 as double) in (cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast(1 as decimal(10, 0)) in (cast(1 as tinyint)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as smallint)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as int)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as bigint)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as float)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as double)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0))) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as string)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast('1' as binary)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as boolean)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast(1 as string) in (cast(1 as tinyint)) FROM t; +SELECT cast(1 as string) in (cast(1 as smallint)) FROM t; +SELECT cast(1 as string) in (cast(1 as int)) FROM t; +SELECT cast(1 as string) in (cast(1 as bigint)) FROM t; +SELECT cast(1 as string) in (cast(1 as float)) FROM t; +SELECT cast(1 as string) in (cast(1 as double)) FROM t; +SELECT cast(1 as string) in (cast(1 as decimal(10, 0))) FROM t; +SELECT cast(1 as string) in (cast(1 as string)) FROM t; +SELECT cast(1 as string) in (cast('1' as binary)) FROM t; +SELECT cast(1 as string) in (cast(1 as boolean)) FROM t; +SELECT cast(1 as string) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast(1 as string) in (cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast('1' as binary) in (cast(1 as tinyint)) FROM t; +SELECT cast('1' as binary) in (cast(1 as smallint)) FROM t; +SELECT cast('1' as binary) in (cast(1 as int)) FROM t; +SELECT cast('1' as binary) in (cast(1 as bigint)) FROM t; +SELECT cast('1' as binary) in (cast(1 as float)) FROM t; +SELECT cast('1' as binary) in (cast(1 as double)) FROM t; +SELECT cast('1' as binary) in (cast(1 as decimal(10, 0))) FROM t; +SELECT cast('1' as binary) in (cast(1 as string)) FROM t; +SELECT cast('1' as binary) in (cast('1' as binary)) FROM t; +SELECT cast('1' as binary) in (cast(1 as boolean)) FROM t; +SELECT cast('1' as binary) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast('1' as binary) in (cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT true in (cast(1 as tinyint)) FROM t; +SELECT true in (cast(1 as smallint)) FROM t; +SELECT true in (cast(1 as int)) FROM t; +SELECT true in (cast(1 as bigint)) FROM t; +SELECT true in (cast(1 as float)) FROM t; +SELECT true in (cast(1 as double)) FROM t; +SELECT true in (cast(1 as decimal(10, 0))) FROM t; +SELECT true in (cast(1 as string)) FROM t; +SELECT true in (cast('1' as binary)) FROM t; +SELECT true in (cast(1 as boolean)) FROM t; +SELECT true in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT true in (cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as tinyint)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as smallint)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as int)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as bigint)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as float)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as double)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as decimal(10, 0))) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as string)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2' as binary)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as boolean)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as tinyint)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as smallint)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as int)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as bigint)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as float)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as double)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as decimal(10, 0))) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as string)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2' as binary)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as boolean)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as tinyint)) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as smallint)) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as int)) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as bigint)) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as float)) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as double)) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as decimal(10, 0))) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as string)) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast('1' as binary)) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as boolean)) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as tinyint)) FROM t; +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as smallint)) FROM t; +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as int)) FROM t; +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as bigint)) FROM t; +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as float)) FROM t; +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as double)) FROM t; +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as decimal(10, 0))) FROM t; +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as string)) FROM t; +SELECT cast(1 as smallint) in (cast(1 as smallint), cast('1' as binary)) FROM t; +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as boolean)) FROM t; +SELECT cast(1 as smallint) in (cast(1 as smallint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast(1 as smallint) in (cast(1 as smallint), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast(1 as int) in (cast(1 as int), cast(1 as tinyint)) FROM t; +SELECT cast(1 as int) in (cast(1 as int), cast(1 as smallint)) FROM t; +SELECT cast(1 as int) in (cast(1 as int), cast(1 as int)) FROM t; +SELECT cast(1 as int) in (cast(1 as int), cast(1 as bigint)) FROM t; +SELECT cast(1 as int) in (cast(1 as int), cast(1 as float)) FROM t; +SELECT cast(1 as int) in (cast(1 as int), cast(1 as double)) FROM t; +SELECT cast(1 as int) in (cast(1 as int), cast(1 as decimal(10, 0))) FROM t; +SELECT cast(1 as int) in (cast(1 as int), cast(1 as string)) FROM t; +SELECT cast(1 as int) in (cast(1 as int), cast('1' as binary)) FROM t; +SELECT cast(1 as int) in (cast(1 as int), cast(1 as boolean)) FROM t; +SELECT cast(1 as int) in (cast(1 as int), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast(1 as int) in (cast(1 as int), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as tinyint)) FROM t; +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as smallint)) FROM t; +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as int)) FROM t; +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as bigint)) FROM t; +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as float)) FROM t; +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as double)) FROM t; +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as decimal(10, 0))) FROM t; +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as string)) FROM t; +SELECT cast(1 as bigint) in (cast(1 as bigint), cast('1' as binary)) FROM t; +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as boolean)) FROM t; +SELECT cast(1 as bigint) in (cast(1 as bigint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast(1 as bigint) in (cast(1 as bigint), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast(1 as float) in (cast(1 as float), cast(1 as tinyint)) FROM t; +SELECT cast(1 as float) in (cast(1 as float), cast(1 as smallint)) FROM t; +SELECT cast(1 as float) in (cast(1 as float), cast(1 as int)) FROM t; +SELECT cast(1 as float) in (cast(1 as float), cast(1 as bigint)) FROM t; +SELECT cast(1 as float) in (cast(1 as float), cast(1 as float)) FROM t; +SELECT cast(1 as float) in (cast(1 as float), cast(1 as double)) FROM t; +SELECT cast(1 as float) in (cast(1 as float), cast(1 as decimal(10, 0))) FROM t; +SELECT cast(1 as float) in (cast(1 as float), cast(1 as string)) FROM t; +SELECT cast(1 as float) in (cast(1 as float), cast('1' as binary)) FROM t; +SELECT cast(1 as float) in (cast(1 as float), cast(1 as boolean)) FROM t; +SELECT cast(1 as float) in (cast(1 as float), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast(1 as float) in (cast(1 as float), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast(1 as double) in (cast(1 as double), cast(1 as tinyint)) FROM t; +SELECT cast(1 as double) in (cast(1 as double), cast(1 as smallint)) FROM t; +SELECT cast(1 as double) in (cast(1 as double), cast(1 as int)) FROM t; +SELECT cast(1 as double) in (cast(1 as double), cast(1 as bigint)) FROM t; +SELECT cast(1 as double) in (cast(1 as double), cast(1 as float)) FROM t; +SELECT cast(1 as double) in (cast(1 as double), cast(1 as double)) FROM t; +SELECT cast(1 as double) in (cast(1 as double), cast(1 as decimal(10, 0))) FROM t; +SELECT cast(1 as double) in (cast(1 as double), cast(1 as string)) FROM t; +SELECT cast(1 as double) in (cast(1 as double), cast('1' as binary)) FROM t; +SELECT cast(1 as double) in (cast(1 as double), cast(1 as boolean)) FROM t; +SELECT cast(1 as double) in (cast(1 as double), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast(1 as double) in (cast(1 as double), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as tinyint)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as smallint)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as int)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as bigint)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as float)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as double)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as decimal(10, 0))) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as string)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast('1' as binary)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as boolean)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast(1 as string) in (cast(1 as string), cast(1 as tinyint)) FROM t; +SELECT cast(1 as string) in (cast(1 as string), cast(1 as smallint)) FROM t; +SELECT cast(1 as string) in (cast(1 as string), cast(1 as int)) FROM t; +SELECT cast(1 as string) in (cast(1 as string), cast(1 as bigint)) FROM t; +SELECT cast(1 as string) in (cast(1 as string), cast(1 as float)) FROM t; +SELECT cast(1 as string) in (cast(1 as string), cast(1 as double)) FROM t; +SELECT cast(1 as string) in (cast(1 as string), cast(1 as decimal(10, 0))) FROM t; +SELECT cast(1 as string) in (cast(1 as string), cast(1 as string)) FROM t; +SELECT cast(1 as string) in (cast(1 as string), cast('1' as binary)) FROM t; +SELECT cast(1 as string) in (cast(1 as string), cast(1 as boolean)) FROM t; +SELECT cast(1 as string) in (cast(1 as string), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast(1 as string) in (cast(1 as string), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as tinyint)) FROM t; +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as smallint)) FROM t; +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as int)) FROM t; +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as bigint)) FROM t; +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as float)) FROM t; +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as double)) FROM t; +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as decimal(10, 0))) FROM t; +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as string)) FROM t; +SELECT cast('1' as binary) in (cast('1' as binary), cast('1' as binary)) FROM t; +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as boolean)) FROM t; +SELECT cast('1' as binary) in (cast('1' as binary), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast('1' as binary) in (cast('1' as binary), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as tinyint)) FROM t; +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as smallint)) FROM t; +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as int)) FROM t; +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as bigint)) FROM t; +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as float)) FROM t; +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as double)) FROM t; +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as decimal(10, 0))) FROM t; +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as string)) FROM t; +SELECT cast('1' as boolean) in (cast('1' as boolean), cast('1' as binary)) FROM t; +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as boolean)) FROM t; +SELECT cast('1' as boolean) in (cast('1' as boolean), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast('1' as boolean) in (cast('1' as boolean), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as tinyint)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as smallint)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as int)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as bigint)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as float)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as double)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as decimal(10, 0))) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as string)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast('1' as binary)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as boolean)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as tinyint)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as smallint)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as int)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as bigint)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as float)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as double)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as decimal(10, 0))) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as string)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast('1' as binary)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as boolean)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast('2017-12-11 09:30:00' as date)) FROM t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/mapZipWith.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/mapZipWith.sql new file mode 100644 index 000000000000..1727ee725db2 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/mapZipWith.sql @@ -0,0 +1,78 @@ +CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES ( + map(true, false), + map(2Y, 1Y), + map(2S, 1S), + map(2, 1), + map(2L, 1L), + map(922337203685477897945456575809789456, 922337203685477897945456575809789456), + map(9.22337203685477897945456575809789456, 9.22337203685477897945456575809789456), + map(2.0D, 1.0D), + map(float(2.0), float(1.0)), + map(date '2016-03-14', date '2016-03-13'), + map(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), + map('true', 'false', '2', '1'), + map('2016-03-14', '2016-03-13'), + map('2016-11-15 20:54:00.000', '2016-11-12 20:54:00.000'), + map('922337203685477897945456575809789456', 'text'), + map(array(1L, 2L), array(1L, 2L)), map(array(1, 2), array(1, 2)), + map(struct(1S, 2L), struct(1S, 2L)), map(struct(1, 2), struct(1, 2)) +) AS various_maps( + boolean_map, + tinyint_map, + smallint_map, + int_map, + bigint_map, + decimal_map1, decimal_map2, + double_map, + float_map, + date_map, + timestamp_map, + string_map1, string_map2, string_map3, string_map4, + array_map1, array_map2, + struct_map1, struct_map2 +); + +SELECT map_zip_with(tinyint_map, smallint_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps; + +SELECT map_zip_with(smallint_map, int_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps; + +SELECT map_zip_with(int_map, bigint_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps; + +SELECT map_zip_with(double_map, float_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps; + +SELECT map_zip_with(decimal_map1, decimal_map2, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps; + +SELECT map_zip_with(decimal_map1, int_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps; + +SELECT map_zip_with(decimal_map1, double_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps; + +SELECT map_zip_with(decimal_map2, int_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps; + +SELECT map_zip_with(decimal_map2, double_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps; + +SELECT map_zip_with(string_map1, int_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps; + +SELECT map_zip_with(string_map2, date_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps; + +SELECT map_zip_with(timestamp_map, string_map3, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps; + +SELECT map_zip_with(decimal_map1, string_map4, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps; + +SELECT map_zip_with(array_map1, array_map2, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps; + +SELECT map_zip_with(struct_map1, struct_map2, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/mapconcat.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/mapconcat.sql new file mode 100644 index 000000000000..60895020fcc8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/mapconcat.sql @@ -0,0 +1,92 @@ +CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES ( + map(true, false), map(false, true), + map(1Y, 2Y), map(3Y, 4Y), + map(1S, 2S), map(3S, 4S), + map(4, 6), map(7, 8), + map(6L, 7L), map(8L, 9L), + map(9223372036854775809, 9223372036854775808), map(9223372036854775808, 9223372036854775809), + map(1.0D, 2.0D), map(3.0D, 4.0D), + map(float(1.0D), float(2.0D)), map(float(3.0D), float(4.0D)), + map(date '2016-03-14', date '2016-03-13'), map(date '2016-03-12', date '2016-03-11'), + map(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), + map(timestamp '2016-11-11 20:54:00.000', timestamp '2016-11-09 20:54:00.000'), + map('a', 'b'), map('c', 'd'), + map(array('a', 'b'), array('c', 'd')), map(array('e'), array('f')), + map(struct('a', 1), struct('b', 2)), map(struct('c', 3), struct('d', 4)), + map('a', 1), map('c', 2), + map(1, 'a'), map(2, 'c') +) AS various_maps ( + boolean_map1, boolean_map2, + tinyint_map1, tinyint_map2, + smallint_map1, smallint_map2, + int_map1, int_map2, + bigint_map1, bigint_map2, + decimal_map1, decimal_map2, + double_map1, double_map2, + float_map1, float_map2, + date_map1, date_map2, + timestamp_map1, + timestamp_map2, + string_map1, string_map2, + array_map1, array_map2, + struct_map1, struct_map2, + string_int_map1, string_int_map2, + int_string_map1, int_string_map2 +); + +-- Concatenate maps of the same type +SELECT + map_concat(boolean_map1, boolean_map2) boolean_map, + map_concat(tinyint_map1, tinyint_map2) tinyint_map, + map_concat(smallint_map1, smallint_map2) smallint_map, + map_concat(int_map1, int_map2) int_map, + map_concat(bigint_map1, bigint_map2) bigint_map, + map_concat(decimal_map1, decimal_map2) decimal_map, + map_concat(float_map1, float_map2) float_map, + map_concat(double_map1, double_map2) double_map, + map_concat(date_map1, date_map2) date_map, + map_concat(timestamp_map1, timestamp_map2) timestamp_map, + map_concat(string_map1, string_map2) string_map, + map_concat(array_map1, array_map2) array_map, + map_concat(struct_map1, struct_map2) struct_map, + map_concat(string_int_map1, string_int_map2) string_int_map, + map_concat(int_string_map1, int_string_map2) int_string_map +FROM various_maps; + +-- Concatenate maps of different types +SELECT + map_concat(tinyint_map1, smallint_map2) ts_map, + map_concat(smallint_map1, int_map2) si_map, + map_concat(int_map1, bigint_map2) ib_map, + map_concat(bigint_map1, decimal_map2) bd_map, + map_concat(decimal_map1, float_map2) df_map, + map_concat(string_map1, date_map2) std_map, + map_concat(timestamp_map1, string_map2) tst_map, + map_concat(string_map1, int_map2) sti_map, + map_concat(int_string_map1, tinyint_map2) istt_map +FROM various_maps; + +-- Concatenate map of incompatible types 1 +SELECT + map_concat(tinyint_map1, array_map1) tm_map +FROM various_maps; + +-- Concatenate map of incompatible types 2 +SELECT + map_concat(boolean_map1, int_map2) bi_map +FROM various_maps; + +-- Concatenate map of incompatible types 3 +SELECT + map_concat(int_map1, struct_map2) is_map +FROM various_maps; + +-- Concatenate map of incompatible types 4 +SELECT + map_concat(struct_map1, array_map2) ma_map +FROM various_maps; + +-- Concatenate map of incompatible types 5 +SELECT + map_concat(int_map1, array_map2) ms_map +FROM various_maps; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/promoteStrings.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/promoteStrings.sql new file mode 100644 index 000000000000..a5603a184578 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/promoteStrings.sql @@ -0,0 +1,364 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +CREATE TEMPORARY VIEW t AS SELECT 1; + +-- Binary arithmetic +SELECT '1' + cast(1 as tinyint) FROM t; +SELECT '1' + cast(1 as smallint) FROM t; +SELECT '1' + cast(1 as int) FROM t; +SELECT '1' + cast(1 as bigint) FROM t; +SELECT '1' + cast(1 as float) FROM t; +SELECT '1' + cast(1 as double) FROM t; +SELECT '1' + cast(1 as decimal(10, 0)) FROM t; +SELECT '1' + '1' FROM t; +SELECT '1' + cast('1' as binary) FROM t; +SELECT '1' + cast(1 as boolean) FROM t; +SELECT '1' + cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT '1' + cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT '1' - cast(1 as tinyint) FROM t; +SELECT '1' - cast(1 as smallint) FROM t; +SELECT '1' - cast(1 as int) FROM t; +SELECT '1' - cast(1 as bigint) FROM t; +SELECT '1' - cast(1 as float) FROM t; +SELECT '1' - cast(1 as double) FROM t; +SELECT '1' - cast(1 as decimal(10, 0)) FROM t; +SELECT '1' - '1' FROM t; +SELECT '1' - cast('1' as binary) FROM t; +SELECT '1' - cast(1 as boolean) FROM t; +SELECT '1' - cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT '1' - cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT '1' * cast(1 as tinyint) FROM t; +SELECT '1' * cast(1 as smallint) FROM t; +SELECT '1' * cast(1 as int) FROM t; +SELECT '1' * cast(1 as bigint) FROM t; +SELECT '1' * cast(1 as float) FROM t; +SELECT '1' * cast(1 as double) FROM t; +SELECT '1' * cast(1 as decimal(10, 0)) FROM t; +SELECT '1' * '1' FROM t; +SELECT '1' * cast('1' as binary) FROM t; +SELECT '1' * cast(1 as boolean) FROM t; +SELECT '1' * cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT '1' * cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT '1' / cast(1 as tinyint) FROM t; +SELECT '1' / cast(1 as smallint) FROM t; +SELECT '1' / cast(1 as int) FROM t; +SELECT '1' / cast(1 as bigint) FROM t; +SELECT '1' / cast(1 as float) FROM t; +SELECT '1' / cast(1 as double) FROM t; +SELECT '1' / cast(1 as decimal(10, 0)) FROM t; +SELECT '1' / '1' FROM t; +SELECT '1' / cast('1' as binary) FROM t; +SELECT '1' / cast(1 as boolean) FROM t; +SELECT '1' / cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT '1' / cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT '1' % cast(1 as tinyint) FROM t; +SELECT '1' % cast(1 as smallint) FROM t; +SELECT '1' % cast(1 as int) FROM t; +SELECT '1' % cast(1 as bigint) FROM t; +SELECT '1' % cast(1 as float) FROM t; +SELECT '1' % cast(1 as double) FROM t; +SELECT '1' % cast(1 as decimal(10, 0)) FROM t; +SELECT '1' % '1' FROM t; +SELECT '1' % cast('1' as binary) FROM t; +SELECT '1' % cast(1 as boolean) FROM t; +SELECT '1' % cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT '1' % cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT pmod('1', cast(1 as tinyint)) FROM t; +SELECT pmod('1', cast(1 as smallint)) FROM t; +SELECT pmod('1', cast(1 as int)) FROM t; +SELECT pmod('1', cast(1 as bigint)) FROM t; +SELECT pmod('1', cast(1 as float)) FROM t; +SELECT pmod('1', cast(1 as double)) FROM t; +SELECT pmod('1', cast(1 as decimal(10, 0))) FROM t; +SELECT pmod('1', '1') FROM t; +SELECT pmod('1', cast('1' as binary)) FROM t; +SELECT pmod('1', cast(1 as boolean)) FROM t; +SELECT pmod('1', cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT pmod('1', cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT cast(1 as tinyint) + '1' FROM t; +SELECT cast(1 as smallint) + '1' FROM t; +SELECT cast(1 as int) + '1' FROM t; +SELECT cast(1 as bigint) + '1' FROM t; +SELECT cast(1 as float) + '1' FROM t; +SELECT cast(1 as double) + '1' FROM t; +SELECT cast(1 as decimal(10, 0)) + '1' FROM t; +SELECT cast('1' as binary) + '1' FROM t; +SELECT cast(1 as boolean) + '1' FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + '1' FROM t; +SELECT cast('2017-12-11 09:30:00' as date) + '1' FROM t; + +SELECT cast(1 as tinyint) - '1' FROM t; +SELECT cast(1 as smallint) - '1' FROM t; +SELECT cast(1 as int) - '1' FROM t; +SELECT cast(1 as bigint) - '1' FROM t; +SELECT cast(1 as float) - '1' FROM t; +SELECT cast(1 as double) - '1' FROM t; +SELECT cast(1 as decimal(10, 0)) - '1' FROM t; +SELECT cast('1' as binary) - '1' FROM t; +SELECT cast(1 as boolean) - '1' FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - '1' FROM t; +SELECT cast('2017-12-11 09:30:00' as date) - '1' FROM t; + +SELECT cast(1 as tinyint) * '1' FROM t; +SELECT cast(1 as smallint) * '1' FROM t; +SELECT cast(1 as int) * '1' FROM t; +SELECT cast(1 as bigint) * '1' FROM t; +SELECT cast(1 as float) * '1' FROM t; +SELECT cast(1 as double) * '1' FROM t; +SELECT cast(1 as decimal(10, 0)) * '1' FROM t; +SELECT cast('1' as binary) * '1' FROM t; +SELECT cast(1 as boolean) * '1' FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) * '1' FROM t; +SELECT cast('2017-12-11 09:30:00' as date) * '1' FROM t; + +SELECT cast(1 as tinyint) / '1' FROM t; +SELECT cast(1 as smallint) / '1' FROM t; +SELECT cast(1 as int) / '1' FROM t; +SELECT cast(1 as bigint) / '1' FROM t; +SELECT cast(1 as float) / '1' FROM t; +SELECT cast(1 as double) / '1' FROM t; +SELECT cast(1 as decimal(10, 0)) / '1' FROM t; +SELECT cast('1' as binary) / '1' FROM t; +SELECT cast(1 as boolean) / '1' FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / '1' FROM t; +SELECT cast('2017-12-11 09:30:00' as date) / '1' FROM t; + +SELECT cast(1 as tinyint) % '1' FROM t; +SELECT cast(1 as smallint) % '1' FROM t; +SELECT cast(1 as int) % '1' FROM t; +SELECT cast(1 as bigint) % '1' FROM t; +SELECT cast(1 as float) % '1' FROM t; +SELECT cast(1 as double) % '1' FROM t; +SELECT cast(1 as decimal(10, 0)) % '1' FROM t; +SELECT cast('1' as binary) % '1' FROM t; +SELECT cast(1 as boolean) % '1' FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % '1' FROM t; +SELECT cast('2017-12-11 09:30:00' as date) % '1' FROM t; + +SELECT pmod(cast(1 as tinyint), '1') FROM t; +SELECT pmod(cast(1 as smallint), '1') FROM t; +SELECT pmod(cast(1 as int), '1') FROM t; +SELECT pmod(cast(1 as bigint), '1') FROM t; +SELECT pmod(cast(1 as float), '1') FROM t; +SELECT pmod(cast(1 as double), '1') FROM t; +SELECT pmod(cast(1 as decimal(10, 0)), '1') FROM t; +SELECT pmod(cast('1' as binary), '1') FROM t; +SELECT pmod(cast(1 as boolean), '1') FROM t; +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), '1') FROM t; +SELECT pmod(cast('2017-12-11 09:30:00' as date), '1') FROM t; + +-- Equality +SELECT '1' = cast(1 as tinyint) FROM t; +SELECT '1' = cast(1 as smallint) FROM t; +SELECT '1' = cast(1 as int) FROM t; +SELECT '1' = cast(1 as bigint) FROM t; +SELECT '1' = cast(1 as float) FROM t; +SELECT '1' = cast(1 as double) FROM t; +SELECT '1' = cast(1 as decimal(10, 0)) FROM t; +SELECT '1' = '1' FROM t; +SELECT '1' = cast('1' as binary) FROM t; +SELECT '1' = cast(1 as boolean) FROM t; +SELECT '1' = cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT '1' = cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) = '1' FROM t; +SELECT cast(1 as smallint) = '1' FROM t; +SELECT cast(1 as int) = '1' FROM t; +SELECT cast(1 as bigint) = '1' FROM t; +SELECT cast(1 as float) = '1' FROM t; +SELECT cast(1 as double) = '1' FROM t; +SELECT cast(1 as decimal(10, 0)) = '1' FROM t; +SELECT cast('1' as binary) = '1' FROM t; +SELECT cast(1 as boolean) = '1' FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = '1' FROM t; +SELECT cast('2017-12-11 09:30:00' as date) = '1' FROM t; + +SELECT '1' <=> cast(1 as tinyint) FROM t; +SELECT '1' <=> cast(1 as smallint) FROM t; +SELECT '1' <=> cast(1 as int) FROM t; +SELECT '1' <=> cast(1 as bigint) FROM t; +SELECT '1' <=> cast(1 as float) FROM t; +SELECT '1' <=> cast(1 as double) FROM t; +SELECT '1' <=> cast(1 as decimal(10, 0)) FROM t; +SELECT '1' <=> '1' FROM t; +SELECT '1' <=> cast('1' as binary) FROM t; +SELECT '1' <=> cast(1 as boolean) FROM t; +SELECT '1' <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT '1' <=> cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) <=> '1' FROM t; +SELECT cast(1 as smallint) <=> '1' FROM t; +SELECT cast(1 as int) <=> '1' FROM t; +SELECT cast(1 as bigint) <=> '1' FROM t; +SELECT cast(1 as float) <=> '1' FROM t; +SELECT cast(1 as double) <=> '1' FROM t; +SELECT cast(1 as decimal(10, 0)) <=> '1' FROM t; +SELECT cast('1' as binary) <=> '1' FROM t; +SELECT cast(1 as boolean) <=> '1' FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> '1' FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <=> '1' FROM t; + +-- Binary comparison +SELECT '1' < cast(1 as tinyint) FROM t; +SELECT '1' < cast(1 as smallint) FROM t; +SELECT '1' < cast(1 as int) FROM t; +SELECT '1' < cast(1 as bigint) FROM t; +SELECT '1' < cast(1 as float) FROM t; +SELECT '1' < cast(1 as double) FROM t; +SELECT '1' < cast(1 as decimal(10, 0)) FROM t; +SELECT '1' < '1' FROM t; +SELECT '1' < cast('1' as binary) FROM t; +SELECT '1' < cast(1 as boolean) FROM t; +SELECT '1' < cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT '1' < cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT '1' <= cast(1 as tinyint) FROM t; +SELECT '1' <= cast(1 as smallint) FROM t; +SELECT '1' <= cast(1 as int) FROM t; +SELECT '1' <= cast(1 as bigint) FROM t; +SELECT '1' <= cast(1 as float) FROM t; +SELECT '1' <= cast(1 as double) FROM t; +SELECT '1' <= cast(1 as decimal(10, 0)) FROM t; +SELECT '1' <= '1' FROM t; +SELECT '1' <= cast('1' as binary) FROM t; +SELECT '1' <= cast(1 as boolean) FROM t; +SELECT '1' <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT '1' <= cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT '1' > cast(1 as tinyint) FROM t; +SELECT '1' > cast(1 as smallint) FROM t; +SELECT '1' > cast(1 as int) FROM t; +SELECT '1' > cast(1 as bigint) FROM t; +SELECT '1' > cast(1 as float) FROM t; +SELECT '1' > cast(1 as double) FROM t; +SELECT '1' > cast(1 as decimal(10, 0)) FROM t; +SELECT '1' > '1' FROM t; +SELECT '1' > cast('1' as binary) FROM t; +SELECT '1' > cast(1 as boolean) FROM t; +SELECT '1' > cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT '1' > cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT '1' >= cast(1 as tinyint) FROM t; +SELECT '1' >= cast(1 as smallint) FROM t; +SELECT '1' >= cast(1 as int) FROM t; +SELECT '1' >= cast(1 as bigint) FROM t; +SELECT '1' >= cast(1 as float) FROM t; +SELECT '1' >= cast(1 as double) FROM t; +SELECT '1' >= cast(1 as decimal(10, 0)) FROM t; +SELECT '1' >= '1' FROM t; +SELECT '1' >= cast('1' as binary) FROM t; +SELECT '1' >= cast(1 as boolean) FROM t; +SELECT '1' >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT '1' >= cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT '1' <> cast(1 as tinyint) FROM t; +SELECT '1' <> cast(1 as smallint) FROM t; +SELECT '1' <> cast(1 as int) FROM t; +SELECT '1' <> cast(1 as bigint) FROM t; +SELECT '1' <> cast(1 as float) FROM t; +SELECT '1' <> cast(1 as double) FROM t; +SELECT '1' <> cast(1 as decimal(10, 0)) FROM t; +SELECT '1' <> '1' FROM t; +SELECT '1' <> cast('1' as binary) FROM t; +SELECT '1' <> cast(1 as boolean) FROM t; +SELECT '1' <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT '1' <> cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as tinyint) < '1' FROM t; +SELECT cast(1 as smallint) < '1' FROM t; +SELECT cast(1 as int) < '1' FROM t; +SELECT cast(1 as bigint) < '1' FROM t; +SELECT cast(1 as float) < '1' FROM t; +SELECT cast(1 as double) < '1' FROM t; +SELECT cast(1 as decimal(10, 0)) < '1' FROM t; +SELECT '1' < '1' FROM t; +SELECT cast('1' as binary) < '1' FROM t; +SELECT cast(1 as boolean) < '1' FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < '1' FROM t; +SELECT cast('2017-12-11 09:30:00' as date) < '1' FROM t; + +SELECT cast(1 as tinyint) <= '1' FROM t; +SELECT cast(1 as smallint) <= '1' FROM t; +SELECT cast(1 as int) <= '1' FROM t; +SELECT cast(1 as bigint) <= '1' FROM t; +SELECT cast(1 as float) <= '1' FROM t; +SELECT cast(1 as double) <= '1' FROM t; +SELECT cast(1 as decimal(10, 0)) <= '1' FROM t; +SELECT '1' <= '1' FROM t; +SELECT cast('1' as binary) <= '1' FROM t; +SELECT cast(1 as boolean) <= '1' FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= '1' FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <= '1' FROM t; + +SELECT cast(1 as tinyint) > '1' FROM t; +SELECT cast(1 as smallint) > '1' FROM t; +SELECT cast(1 as int) > '1' FROM t; +SELECT cast(1 as bigint) > '1' FROM t; +SELECT cast(1 as float) > '1' FROM t; +SELECT cast(1 as double) > '1' FROM t; +SELECT cast(1 as decimal(10, 0)) > '1' FROM t; +SELECT '1' > '1' FROM t; +SELECT cast('1' as binary) > '1' FROM t; +SELECT cast(1 as boolean) > '1' FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > '1' FROM t; +SELECT cast('2017-12-11 09:30:00' as date) > '1' FROM t; + +SELECT cast(1 as tinyint) >= '1' FROM t; +SELECT cast(1 as smallint) >= '1' FROM t; +SELECT cast(1 as int) >= '1' FROM t; +SELECT cast(1 as bigint) >= '1' FROM t; +SELECT cast(1 as float) >= '1' FROM t; +SELECT cast(1 as double) >= '1' FROM t; +SELECT cast(1 as decimal(10, 0)) >= '1' FROM t; +SELECT '1' >= '1' FROM t; +SELECT cast('1' as binary) >= '1' FROM t; +SELECT cast(1 as boolean) >= '1' FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= '1' FROM t; +SELECT cast('2017-12-11 09:30:00' as date) >= '1' FROM t; + +SELECT cast(1 as tinyint) <> '1' FROM t; +SELECT cast(1 as smallint) <> '1' FROM t; +SELECT cast(1 as int) <> '1' FROM t; +SELECT cast(1 as bigint) <> '1' FROM t; +SELECT cast(1 as float) <> '1' FROM t; +SELECT cast(1 as double) <> '1' FROM t; +SELECT cast(1 as decimal(10, 0)) <> '1' FROM t; +SELECT '1' <> '1' FROM t; +SELECT cast('1' as binary) <> '1' FROM t; +SELECT cast(1 as boolean) <> '1' FROM t; +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> '1' FROM t; +SELECT cast('2017-12-11 09:30:00' as date) <> '1' FROM t; + +-- Functions +SELECT abs('1') FROM t; +SELECT sum('1') FROM t; +SELECT avg('1') FROM t; +SELECT stddev_pop('1') FROM t; +SELECT stddev_samp('1') FROM t; +SELECT - '1' FROM t; +SELECT + '1' FROM t; +SELECT var_pop('1') FROM t; +SELECT var_samp('1') FROM t; +SELECT skewness('1') FROM t; +SELECT kurtosis('1') FROM t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/stringCastAndExpressions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/stringCastAndExpressions.sql new file mode 100644 index 000000000000..731dbc2dcf31 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/stringCastAndExpressions.sql @@ -0,0 +1,54 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +CREATE TEMPORARY VIEW t AS SELECT 'aa' as a; + +-- casting to data types which are unable to represent the string input returns NULL +select cast(a as byte) from t; +select cast(a as short) from t; +select cast(a as int) from t; +select cast(a as long) from t; +select cast(a as float) from t; +select cast(a as double) from t; +select cast(a as decimal) from t; +select cast(a as boolean) from t; +select cast(a as timestamp) from t; +select cast(a as date) from t; +-- casting to binary works correctly +select cast(a as binary) from t; +-- casting to array, struct or map throws exception +select cast(a as array) from t; +select cast(a as struct) from t; +select cast(a as map) from t; + +-- all timestamp/date expressions return NULL if bad input strings are provided +select to_timestamp(a) from t; +select to_unix_timestamp(a) from t; +select unix_timestamp(a) from t; +select from_unixtime(a) from t; +select from_unixtime('2018-01-01', a) from t; +select next_day(a, 'MO') from t; +select next_day('2018-01-01', a) from t; +select trunc(a, 'MM') from t; +select trunc('2018-01-01', a) from t; + +-- some functions return NULL if bad input is provided +select unhex('-123'); +select sha2(a, a) from t; +select get_json_object(a, a) from t; +select json_tuple(a, a) from t; +select from_json(a, 'a INT') from t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/widenSetOperationTypes.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/widenSetOperationTypes.sql new file mode 100644 index 000000000000..66e9689850d9 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/widenSetOperationTypes.sql @@ -0,0 +1,175 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +CREATE TEMPORARY VIEW t AS SELECT 1; + +-- UNION +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as tinyint) FROM t; +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as smallint) FROM t; +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as int) FROM t; +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as bigint) FROM t; +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as float) FROM t; +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as double) FROM t; +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t; +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as string) FROM t; +SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2' as binary) FROM t; +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as boolean) FROM t; +SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as tinyint) FROM t; +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as smallint) FROM t; +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as int) FROM t; +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as bigint) FROM t; +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as float) FROM t; +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as double) FROM t; +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t; +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as string) FROM t; +SELECT cast(1 as smallint) FROM t UNION SELECT cast('2' as binary) FROM t; +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as boolean) FROM t; +SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as tinyint) FROM t; +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as smallint) FROM t; +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as int) FROM t; +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as bigint) FROM t; +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as float) FROM t; +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as double) FROM t; +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t; +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as string) FROM t; +SELECT cast(1 as int) FROM t UNION SELECT cast('2' as binary) FROM t; +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as boolean) FROM t; +SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as tinyint) FROM t; +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as smallint) FROM t; +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as int) FROM t; +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as bigint) FROM t; +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as float) FROM t; +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as double) FROM t; +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t; +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as string) FROM t; +SELECT cast(1 as bigint) FROM t UNION SELECT cast('2' as binary) FROM t; +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as boolean) FROM t; +SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as tinyint) FROM t; +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as smallint) FROM t; +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as int) FROM t; +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as bigint) FROM t; +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as float) FROM t; +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as double) FROM t; +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t; +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as string) FROM t; +SELECT cast(1 as float) FROM t UNION SELECT cast('2' as binary) FROM t; +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as boolean) FROM t; +SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as tinyint) FROM t; +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as smallint) FROM t; +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as int) FROM t; +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as bigint) FROM t; +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as float) FROM t; +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as double) FROM t; +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t; +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as string) FROM t; +SELECT cast(1 as double) FROM t UNION SELECT cast('2' as binary) FROM t; +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as boolean) FROM t; +SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as tinyint) FROM t; +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as smallint) FROM t; +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as int) FROM t; +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as bigint) FROM t; +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as float) FROM t; +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as double) FROM t; +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t; +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as string) FROM t; +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2' as binary) FROM t; +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as boolean) FROM t; +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as tinyint) FROM t; +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as smallint) FROM t; +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as int) FROM t; +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as bigint) FROM t; +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as float) FROM t; +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as double) FROM t; +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t; +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as string) FROM t; +SELECT cast(1 as string) FROM t UNION SELECT cast('2' as binary) FROM t; +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as boolean) FROM t; +SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as tinyint) FROM t; +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as smallint) FROM t; +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as int) FROM t; +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as bigint) FROM t; +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as float) FROM t; +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as double) FROM t; +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t; +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as string) FROM t; +SELECT cast('1' as binary) FROM t UNION SELECT cast('2' as binary) FROM t; +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as boolean) FROM t; +SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as tinyint) FROM t; +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as smallint) FROM t; +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as int) FROM t; +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as bigint) FROM t; +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as float) FROM t; +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as double) FROM t; +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t; +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as string) FROM t; +SELECT cast(1 as boolean) FROM t UNION SELECT cast('2' as binary) FROM t; +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as boolean) FROM t; +SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as tinyint) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as smallint) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as int) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as bigint) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as float) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as double) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as string) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2' as binary) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as boolean) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t; + +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as tinyint) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as smallint) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as int) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as bigint) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as float) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as double) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as string) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2' as binary) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as boolean) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t; +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/windowFrameCoercion.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/windowFrameCoercion.sql new file mode 100644 index 000000000000..5cd353875749 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/typeCoercion/native/windowFrameCoercion.sql @@ -0,0 +1,44 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +CREATE TEMPORARY VIEW t AS SELECT 1; + +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as tinyint)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as smallint)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as int)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as bigint)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as float)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as double)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as decimal(10, 0))) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as string)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('1' as binary)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as boolean)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00.0' as timestamp)) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00' as date)) FROM t; + +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as tinyint) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as smallint) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as int) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as bigint) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as float) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as double) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as decimal(10, 0)) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as string) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('1' as binary) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as boolean) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00.0' as timestamp) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00' as date) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udaf/udaf-group-analytics.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udaf/udaf-group-analytics.sql new file mode 100644 index 000000000000..0249d98b6be8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udaf/udaf-group-analytics.sql @@ -0,0 +1,70 @@ +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData(a, b); + +-- CUBE on overlapping columns +SELECT a + b, b, udaf(a - b) FROM testData GROUP BY a + b, b WITH CUBE; + +SELECT a, b, udaf(b) FROM testData GROUP BY a, b WITH CUBE; + +-- ROLLUP on overlapping columns +SELECT a + b, b, udaf(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP; + +SELECT a, b, udaf(b) FROM testData GROUP BY a, b WITH ROLLUP; + +CREATE OR REPLACE TEMPORARY VIEW courseSales AS SELECT * FROM VALUES +("dotNET", 2012, 10000), ("Java", 2012, 20000), ("dotNET", 2012, 5000), ("dotNET", 2013, 48000), ("Java", 2013, 30000) +AS courseSales(course, year, earnings); + +-- ROLLUP +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY course, year; +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY ROLLUP(course, year, (course, year)) ORDER BY course, year; +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY ROLLUP(course, year, (course, year), ()) ORDER BY course, year; + +-- CUBE +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, year; +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY CUBE(course, year, (course, year)) ORDER BY course, year; +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY CUBE(course, year, (course, year), ()) ORDER BY course, year; + +-- GROUPING SETS +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year); +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year, ()); +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course); +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year); + +-- Partial ROLLUP/CUBE/GROUPING SETS +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY course, CUBE(course, year) ORDER BY course, year; +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY CUBE(course, year), ROLLUP(course, year) ORDER BY course, year; +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY CUBE(course, year), ROLLUP(course, year), GROUPING SETS(course, year) ORDER BY course, year; + +-- GROUPING SETS with aggregate functions containing groupBy columns +SELECT course, udaf(earnings) AS sum FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum; +SELECT course, udaf(earnings) AS sum, GROUPING_ID(course, earnings) FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum; + +-- Aliases in SELECT could be used in ROLLUP/CUBE/GROUPING SETS +SELECT a + b AS k1, b AS k2, udaf(a - b) FROM testData GROUP BY CUBE(k1, k2); +SELECT a + b AS k, b, udaf(a - b) FROM testData GROUP BY ROLLUP(k, b); +SELECT a + b, b AS k, udaf(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k); + +-- GROUP BY use mixed Separate columns and CUBE/ROLLUP/Gr +SELECT a, b, udaf(1) FROM testData GROUP BY a, b, CUBE(a, b); +SELECT a, b, udaf(1) FROM testData GROUP BY a, b, ROLLUP(a, b); +SELECT a, b, udaf(1) FROM testData GROUP BY CUBE(a, b), ROLLUP(a, b); +SELECT a, b, udaf(1) FROM testData GROUP BY a, CUBE(a, b), ROLLUP(b); +SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS((a, b), (a), ()); +SELECT a, b, udaf(1) FROM testData GROUP BY a, CUBE(a, b), GROUPING SETS((a, b), (a), ()); +SELECT a, b, udaf(1) FROM testData GROUP BY a, CUBE(a, b), ROLLUP(a, b), GROUPING SETS((a, b), (a), ()); + +-- Support nested CUBE/ROLLUP/GROUPING SETS in GROUPING SETS +SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS(ROLLUP(a, b)); +SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b), (a), ())); + +SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS((a, b), GROUPING SETS(ROLLUP(a, b))); +SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS((a, b, a, b), (a, b, a), (a, b)); +SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b, a, b), (a, b, a), (a, b))); + +SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS(ROLLUP(a, b), CUBE(a, b)); +SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b), (a), ()), GROUPING SETS((a, b), (a), (b), ())); +SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS((a, b), (a), (), (a, b), (a), (b), ()); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udaf/udaf-group-by-ordinal.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udaf/udaf-group-by-ordinal.sql new file mode 100644 index 000000000000..ded3e74d20a7 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udaf/udaf-group-by-ordinal.sql @@ -0,0 +1,88 @@ +-- group by ordinal positions + +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b); + +-- basic case +select a, udaf(b) from data group by 1; + +-- constant case +select 1, 2, udaf(b) from data group by 1, 2; + +-- duplicate group by column +select a, 1, udaf(b) from data group by a, 1; +select a, 1, udaf(b) from data group by 1, 2; + +-- group by a non-aggregate expression's ordinal +select a, b + 2, udaf(2) from data group by a, 2; + +-- with alias +select a as aa, b + 2 as bb, udaf(2) from data group by 1, 2; + +-- foldable non-literal: this should be the same as no grouping. +select udaf(b) from data group by 1 + 0; + +-- negative case: position is an aggregate expression +select a, b, udaf(b) from data group by 3; +select a, b, udaf(b) + 2 from data group by 3; + +-- negative case: nondeterministic expression +select a, rand(0), udaf(b) +from +(select /*+ REPARTITION(1) */ a, b from data) group by a, 2; + +-- group by ordinal followed by order by +select a, udaf(a) from (select 1 as a) tmp group by 1 order by 1; + +-- group by ordinal followed by having +select udaf(a), a from (select 1 as a) tmp group by 2 having a > 0; + +-- mixed cases: group-by ordinals and aliases +select a, a AS k, udaf(b) from data group by k, 1; + +-- can use ordinal in CUBE +select a, b, udaf(1) from data group by cube(1, 2); + +-- mixed cases: can use ordinal in CUBE +select a, b, udaf(1) from data group by cube(1, b); + +-- can use ordinal with cube +select a, b, udaf(1) from data group by 1, 2 with cube; + +-- can use ordinal in ROLLUP +select a, b, udaf(1) from data group by rollup(1, 2); + +-- mixed cases: can use ordinal in ROLLUP +select a, b, udaf(1) from data group by rollup(1, b); + +-- can use ordinal with rollup +select a, b, udaf(1) from data group by 1, 2 with rollup; + +-- can use ordinal in GROUPING SETS +select a, b, udaf(1) from data group by grouping sets((1), (2), (1, 2)); + +-- mixed cases: can use ordinal in GROUPING SETS +select a, b, udaf(1) from data group by grouping sets((1), (b), (a, 2)); + +select a, b, udaf(1) from data group by a, 2 grouping sets((1), (b), (a, 2)); + +-- range error +select a, b, udaf(1) from data group by a, -1; + +select a, b, udaf(1) from data group by a, 3; + +select a, b, udaf(1) from data group by cube(-1, 2); + +select a, b, udaf(1) from data group by cube(1, 3); + +-- turn off group by ordinal +set spark.sql.groupByOrdinal=false; + +-- can now group by negative literal +select udaf(b) from data group by -1; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udaf/udaf-group-by.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udaf/udaf-group-by.sql new file mode 100644 index 000000000000..eaac13bcf6a9 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udaf/udaf-group-by.sql @@ -0,0 +1,110 @@ +-- Test aggregate operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b); + +-- Aggregate with empty GroupBy expressions. +SELECT a, udaf(b) FROM testData; +SELECT udaf(a), udaf(b) FROM testData; + +-- Aggregate with non-empty GroupBy expressions. +SELECT a, udaf(b) FROM testData GROUP BY a; +SELECT a, udaf(b) FROM testData GROUP BY b; +SELECT udaf(a), udaf(b) FROM testData GROUP BY a; + +-- Aggregate grouped by literals. +SELECT 'foo', udaf(a) FROM testData GROUP BY 1; + +-- Aggregate grouped by literals (hash aggregate). +SELECT 'foo', udaf(a) FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate grouped by literals (sort aggregate). +SELECT 'foo', udaf(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate with complex GroupBy expressions. +SELECT a + b, udaf(b) FROM testData GROUP BY a + b; +SELECT a + 2, udaf(b) FROM testData GROUP BY a + 1; +SELECT a + 1 + 1, udaf(b) FROM testData GROUP BY a + 1; + +-- Aggregate with nulls. +SELECT SKEWNESS(a), KURTOSIS(a), udaf(a), udaf(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), udaf(a) +FROM testData; + +-- Aggregate with foldable input and multiple distinct groups. +SELECT udaf(DISTINCT b), udaf(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; + +-- Aliases in SELECT could be used in GROUP BY +SELECT a AS k, udaf(b) FROM testData GROUP BY k; +SELECT a AS k, udaf(b) FROM testData GROUP BY k HAVING k > 1; + +-- GROUP BY alias with invalid col in SELECT list +SELECT a AS k, udaf(non_existing) FROM testData GROUP BY k; + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v); +SELECT k AS a, udaf(v) FROM testDataHasSameNameWithAlias GROUP BY a; + +-- turn off group by aliases +set spark.sql.groupByAliases=false; + +-- Check analysis exceptions +SELECT a AS k, udaf(b) FROM testData GROUP BY k; + +-- Aggregate with empty input and non-empty GroupBy expressions. +SELECT a, udaf(1) FROM testData WHERE false GROUP BY a; + +-- Aggregate with empty input and empty GroupBy expressions. +SELECT udaf(1) FROM testData WHERE false; +SELECT 1 FROM (SELECT udaf(1) FROM testData WHERE false) t; + +-- Aggregate with empty GroupBy expressions and filter on top +SELECT 1 from ( + SELECT 1 AS z, + udaf(a.x) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z; + +-- SPARK-25708 HAVING without GROUP BY means global aggregate +SELECT 1 FROM range(10) HAVING udaf(id) > 0; + +-- Test data +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v); + +-- empty table +SELECT udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0; + +-- all null values +SELECT udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4; + +-- aggregates are null Filtering +SELECT udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5; + +-- group by +SELECT k, udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k; + +-- having +SELECT k, udaf(v) FROM test_agg GROUP BY k HAVING udaf(v) = false; +SELECT k, udaf(v) FROM test_agg GROUP BY k HAVING udaf(v) IS NULL; + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT k, + udaf(v) AS count +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY k; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udaf/udaf-grouping-set.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udaf/udaf-grouping-set.sql new file mode 100644 index 000000000000..1217b9e5b09d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udaf/udaf-grouping-set.sql @@ -0,0 +1,47 @@ +CREATE TEMPORARY VIEW grouping AS SELECT * FROM VALUES + ("1", "2", "3", 1), + ("4", "5", "6", 1), + ("7", "8", "9", 1) + as grouping(a, b, c, d); + +-- SPARK-17849: grouping set throws NPE #1 +SELECT a, b, c, udaf(d) FROM grouping GROUP BY a, b, c GROUPING SETS (()); + +-- SPARK-17849: grouping set throws NPE #2 +SELECT a, b, c, udaf(d) FROM grouping GROUP BY a, b, c GROUPING SETS ((a)); + +-- SPARK-17849: grouping set throws NPE #3 +SELECT a, b, c, udaf(d) FROM grouping GROUP BY a, b, c GROUPING SETS ((c)); + +-- Group sets without explicit group by +SELECT c1, udaf(c2) FROM (VALUES ('x', 10, 0), ('y', 20, 0)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1); + +-- Group sets without group by and with grouping +SELECT c1, udaf(c2), grouping(c1) FROM (VALUES ('x', 10, 0), ('y', 20, 0)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1); + +-- Mutiple grouping within a grouping set +SELECT c1, c2, udaf(c3), grouping__id +FROM (VALUES ('x', 'a', 10), ('y', 'b', 20) ) AS t (c1, c2, c3) +GROUP BY GROUPING SETS ( ( c1 ), ( c2 ) ) +HAVING GROUPING__ID > 1; + +-- complex expression in grouping sets +SELECT a + b, b, udaf(c) FROM (VALUES (1,1,1),(2,2,2)) AS t(a,b,c) GROUP BY GROUPING SETS ( (a + b), (b)); + +-- complex expression in grouping sets +SELECT a + b, b, udaf(c) FROM (VALUES (1,1,1),(2,2,2)) AS t(a,b,c) GROUP BY GROUPING SETS ( (a + b), (b + a), (b)); + +-- negative tests - must have at least one grouping expression +SELECT a, b, c, udaf(d) FROM grouping GROUP BY WITH ROLLUP; + +SELECT a, b, c, udaf(d) FROM grouping GROUP BY WITH CUBE; + +-- duplicate entries in grouping sets +SELECT k1, k2, udaf(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)); + +SELECT grouping__id, k1, k2, udaf(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)); + +SELECT grouping(k1), k1, k2, udaf(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)); + +-- grouping_id function +SELECT grouping_id(k1, k2), udaf(v) from (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY k1, k2 GROUPING SETS ((k2, k1), k1); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udaf/udaf.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udaf/udaf.sql new file mode 100644 index 000000000000..0374d98feb6e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udaf/udaf.sql @@ -0,0 +1,21 @@ +-- Test aggregate operator and UDAF with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN + +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(1), (2), (3), (4) +as t1(int_col1); + +CREATE FUNCTION myDoubleAvg AS 'test.org.apache.spark.sql.MyDoubleAvg'; + +SELECT default.myDoubleAvg(int_col1) as my_avg from t1; + +SELECT default.myDoubleAvg(int_col1, 3) as my_avg from t1; + +CREATE FUNCTION udaf1 AS 'test.non.existent.udaf'; + +SELECT default.udaf1(int_col1) as udaf1 from t1; + +DROP FUNCTION myDoubleAvg; +DROP FUNCTION udaf1; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part1.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part1.sql new file mode 100644 index 000000000000..4b816fb682b5 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part1.sql @@ -0,0 +1,153 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- AGGREGATES [Part 1] +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L1-L143 + +-- avoid bit-exact output here because operations may not be bit-exact. +-- SET extra_float_digits = 0; + +-- This test file was converted from postgreSQL/aggregates_part1.sql. + +SELECT avg(udf(four)) AS avg_1 FROM onek; + +SELECT udf(avg(a)) AS avg_32 FROM aggtest WHERE a < 100; + +-- In 7.1, avg(float4) is computed using float8 arithmetic. +-- Round the result to 3 digits to avoid platform-specific results. + +select CAST(avg(udf(b)) AS Decimal(10,3)) AS avg_107_943 FROM aggtest; +-- `student` has a column with data type POINT, which is not supported by Spark [SPARK-27766] +-- SELECT avg(gpa) AS avg_3_4 FROM ONLY student; + +SELECT sum(udf(four)) AS sum_1500 FROM onek; +SELECT udf(sum(a)) AS sum_198 FROM aggtest; +SELECT udf(udf(sum(b))) AS avg_431_773 FROM aggtest; +-- `student` has a column with data type POINT, which is not supported by Spark [SPARK-27766] +-- SELECT sum(gpa) AS avg_6_8 FROM ONLY student; + +SELECT udf(max(four)) AS max_3 FROM onek; +SELECT max(udf(a)) AS max_100 FROM aggtest; +SELECT udf(udf(max(aggtest.b))) AS max_324_78 FROM aggtest; +-- `student` has a column with data type POINT, which is not supported by Spark [SPARK-27766] +-- SELECT max(student.gpa) AS max_3_7 FROM student; + +SELECT stddev_pop(udf(b)) FROM aggtest; +SELECT udf(stddev_samp(b)) FROM aggtest; +SELECT var_pop(udf(b)) FROM aggtest; +SELECT udf(var_samp(b)) FROM aggtest; + +SELECT udf(stddev_pop(CAST(b AS Decimal(38,0)))) FROM aggtest; +SELECT stddev_samp(CAST(udf(b) AS Decimal(38,0))) FROM aggtest; +SELECT udf(var_pop(CAST(b AS Decimal(38,0)))) FROM aggtest; +SELECT var_samp(udf(CAST(b AS Decimal(38,0)))) FROM aggtest; + +-- population variance is defined for a single tuple, sample variance +-- is not +SELECT udf(var_pop(1.0)), var_samp(udf(2.0)); +SELECT stddev_pop(udf(CAST(3.0 AS Decimal(38,0)))), stddev_samp(CAST(udf(4.0) AS Decimal(38,0))); + + +-- verify correct results for null and NaN inputs +select sum(udf(CAST(null AS int))) from range(1,4); +select sum(udf(CAST(null AS long))) from range(1,4); +select sum(udf(CAST(null AS Decimal(38,0)))) from range(1,4); +select sum(udf(CAST(null AS DOUBLE))) from range(1,4); +select avg(udf(CAST(null AS int))) from range(1,4); +select avg(udf(CAST(null AS long))) from range(1,4); +select avg(udf(CAST(null AS Decimal(38,0)))) from range(1,4); +select avg(udf(CAST(null AS DOUBLE))) from range(1,4); +select sum(CAST(udf('NaN') AS DOUBLE)) from range(1,4); +select avg(CAST(udf('NaN') AS DOUBLE)) from range(1,4); + +-- [SPARK-27768] verify correct results for infinite inputs +-- [SPARK-28291] UDFs cannot be evaluated within inline table definition +-- SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +-- FROM (VALUES (CAST(udf('1') AS DOUBLE)), (CAST(udf('Infinity') AS DOUBLE))) v(x); +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('Infinity'), ('1')) v(x); +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('Infinity'), ('Infinity')) v(x); +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('-Infinity'), ('Infinity')) v(x); + + +-- test accuracy with a large input offset +SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) +FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x); +SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) +FROM (VALUES (7000000000005), (7000000000007)) v(x); + +-- SQL2003 binary aggregates [SPARK-23907] +SELECT regr_count(b, a) FROM aggtest; +SELECT regr_sxx(b, a) FROM aggtest; +SELECT regr_syy(b, a) FROM aggtest; +SELECT regr_sxy(b, a) FROM aggtest; +SELECT regr_avgx(b, a), regr_avgy(b, a) FROM aggtest; +SELECT regr_r2(b, a) FROM aggtest; +SELECT regr_slope(b, a), regr_intercept(b, a) FROM aggtest; +SELECT udf(covar_pop(b, udf(a))), covar_samp(udf(b), a) FROM aggtest; +SELECT corr(b, udf(a)) FROM aggtest; + + +-- test accum and combine functions directly [SPARK-23907] +CREATE TEMPORARY VIEW regr_test AS SELECT * FROM VALUES (10,150),(20,250),(30,350),(80,540),(100,200) AS regr_test (x, y); +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test WHERE x IN (10,20,30,80); +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test; +-- SELECT float8_accum('{4,140,2900}'::float8[], 100); +-- SELECT float8_regr_accum('{4,140,2900,1290,83075,15050}'::float8[], 200, 100); +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test WHERE x IN (10,20,30); +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test WHERE x IN (80,100); +-- SELECT float8_combine('{3,60,200}'::float8[],ELECT CAST(udf(covar_pop(b, udf(a))) AS '{0,0,0}'::float8[]); +-- SELECT float8_combine('{0,0,0}'::float8[], '{2,180,200}'::float8[]); +-- SELECT float8_combine('{3,60,200}'::float8[], '{2,180,200}'::float8[]); +-- SELECT float8_regr_combine('{3,60,200,750,20000,2000}'::float8[], +-- '{0,0,0,0,0,0}'::float8[]); +-- SELECT float8_regr_combine('{0,0,0,0,0,0}'::float8[], +-- '{2,180,200,740,57800,-3400}'::float8[]); +-- SELECT float8_regr_combine('{3,60,200,750,20000,2000}'::float8[], +-- '{2,180,200,740,57800,-3400}'::float8[]); +DROP VIEW regr_test; + + +-- test count, distinct +SELECT count(udf(four)) AS cnt_1000 FROM onek; +SELECT udf(count(DISTINCT four)) AS cnt_4 FROM onek; + +select ten, udf(count(*)), sum(udf(four)) from onek +group by ten order by ten; + +select ten, count(udf(four)), udf(sum(DISTINCT four)) from onek +group by ten order by ten; + +-- user-defined aggregates +-- SELECT newavg(four) AS avg_1 FROM onek; +-- SELECT newsum(four) AS sum_1500 FROM onek; +-- SELECT newcnt(four) AS cnt_1000 FROM onek; +-- SELECT newcnt(*) AS cnt_1000 FROM onek; +-- SELECT oldcnt(*) AS cnt_1000 FROM onek; +-- SELECT sum2(q1,q2) FROM int8_tbl; + +-- test for outer-level aggregates + +-- this should work +select ten, udf(sum(distinct four)) from onek a +group by ten +having exists (select 1 from onek b where udf(sum(distinct a.four)) = b.four); + +-- this should fail because subquery has an agg of its own in WHERE +select ten, sum(distinct four) from onek a +group by ten +having exists (select 1 from onek b + where sum(distinct a.four + b.four) = udf(b.four)); + +-- [SPARK-27769] Test handling of sublinks within outer-level aggregates. +-- Per bug report from Daniel Grace. +select + (select udf(max((select i.unique2 from tenk1 i where i.unique1 = o.unique1)))) +from tenk1 o; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part2.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part2.sql new file mode 100644 index 000000000000..b4054850062b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part2.sql @@ -0,0 +1,206 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- AGGREGATES [Part 2] +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L145-L350 +-- +-- This test file was converted from postgreSQL/aggregates_part2.sql. + +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1); + +-- Test handling of Params within aggregate arguments in hashed aggregation. +-- Per bug report from Jeevan Chalke. +-- [SPARK-27877] Implement SQL-standard LATERAL subqueries +-- explain (verbose, costs off) +-- select s1, s2, sm +-- from generate_series(1, 3) s1, +-- lateral (select s2, sum(s1 + s2) sm +-- from generate_series(1, 3) s2 group by s2) ss +-- order by 1, 2; +-- select s1, s2, sm +-- from generate_series(1, 3) s1, +-- lateral (select s2, sum(s1 + s2) sm +-- from generate_series(1, 3) s2 group by s2) ss +-- order by 1, 2; + +-- [SPARK-27878] Support ARRAY(sub-SELECT) expressions +-- explain (verbose, costs off) +-- select array(select sum(x+y) s +-- from generate_series(1,3) y group by y order by s) +-- from generate_series(1,3) x; +-- select array(select sum(x+y) s +-- from generate_series(1,3) y group by y order by s) +-- from generate_series(1,3) x; + +-- [SPARK-27879] Implement bitwise integer aggregates(BIT_AND and BIT_OR) +-- +-- test for bitwise integer aggregates +-- +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (3, 3, 3, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4); + +-- empty case +SELECT BIT_AND(b1) AS n1, BIT_OR(b2) AS n2 FROM bitwise_test where 1 = 0; + +-- null case +SELECT BIT_AND(b4) AS n1, BIT_OR(b4) AS n2 FROM bitwise_test where b4 is null; + + +SELECT + BIT_AND(cast(b1 as tinyint)) AS a1, + BIT_AND(cast(b2 as smallint)) AS b1, + BIT_AND(b3) AS c1, + BIT_AND(b4) AS d1, + BIT_OR(cast(b1 as tinyint)) AS e7, + BIT_OR(cast(b2 as smallint)) AS f7, + BIT_OR(b3) AS g7, + BIT_OR(b4) AS h3 +FROM bitwise_test; + +-- +-- test boolean aggregates +-- +-- first test all possible transition and final states + +-- The result is inconsistent with PostgreSQL because our AND does not have strict mode +SELECT + -- boolean and transitions + -- null because strict + (NULL AND NULL) IS NULL AS `t`, + (TRUE AND NULL) IS NULL AS `t`, + (FALSE AND NULL) IS NULL AS `t`, + (NULL AND TRUE) IS NULL AS `t`, + (NULL AND FALSE) IS NULL AS `t`, + -- and actual computations + (TRUE AND TRUE) AS `t`, + NOT (TRUE AND FALSE) AS `t`, + NOT (FALSE AND TRUE) AS `t`, + NOT (FALSE AND FALSE) AS `t`; + +-- The result is inconsistent with PostgreSQL because our OR does not have strict mode +SELECT + -- boolean or transitions + -- null because strict + (NULL OR NULL) IS NULL AS `t`, + (TRUE OR NULL) IS NULL AS `t`, + (FALSE OR NULL) IS NULL AS `t`, + (NULL OR TRUE) IS NULL AS `t`, + (NULL OR FALSE) IS NULL AS `t`, + -- actual computations + (TRUE OR TRUE) AS `t`, + (TRUE OR FALSE) AS `t`, + (FALSE OR TRUE) AS `t`, + NOT (FALSE OR FALSE) AS `t`; + +-- [SPARK-27880] Implement boolean aggregates(BOOL_AND, BOOL_OR and EVERY) +CREATE OR REPLACE TEMPORARY VIEW bool_test AS SELECT * FROM VALUES + (TRUE, null, FALSE, null), + (FALSE, TRUE, null, null), + (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4); + +-- empty case +SELECT BOOL_AND(b1) AS n1, BOOL_OR(b3) AS n2 FROM bool_test WHERE 1 = 0; + +SELECT + BOOL_AND(b1) AS f1, + BOOL_AND(b2) AS t2, + BOOL_AND(b3) AS f3, + BOOL_AND(b4) AS n4, + BOOL_AND(NOT b2) AS f5, + BOOL_AND(NOT b3) AS t6 +FROM bool_test; + +SELECT + EVERY(b1) AS f1, + EVERY(b2) AS t2, + EVERY(b3) AS f3, + EVERY(b4) AS n4, + EVERY(NOT b2) AS f5, + EVERY(NOT b3) AS t6 +FROM bool_test; + +SELECT + BOOL_OR(b1) AS t1, + BOOL_OR(b2) AS t2, + BOOL_OR(b3) AS f3, + BOOL_OR(b4) AS n4, + BOOL_OR(NOT b2) AS f5, + BOOL_OR(NOT b3) AS t6 +FROM bool_test; + +-- +-- Test cases that should be optimized into indexscans instead of +-- the generic aggregate implementation. +-- + +-- Basic cases +-- explain +-- select min(unique1) from tenk1; +select min(udf(unique1)) from tenk1; +-- explain +-- select max(unique1) from tenk1; +select udf(max(unique1)) from tenk1; +-- explain +-- select max(unique1) from tenk1 where unique1 < 42; +select max(unique1) from tenk1 where udf(unique1) < 42; +-- explain +-- select max(unique1) from tenk1 where unique1 > 42; +select max(unique1) from tenk1 where unique1 > udf(42); + +-- the planner may choose a generic aggregate here if parallel query is +-- enabled, since that plan will be parallel safe and the "optimized" +-- plan, which has almost identical cost, will not be. we want to test +-- the optimized plan, so temporarily disable parallel query. +-- begin; +-- set local max_parallel_workers_per_gather = 0; +-- explain +-- select max(unique1) from tenk1 where unique1 > 42000; +select max(unique1) from tenk1 where udf(unique1) > 42000; +-- rollback; + +-- multi-column index (uses tenk1_thous_tenthous) +-- explain +-- select max(tenthous) from tenk1 where thousand = 33; +select max(tenthous) from tenk1 where udf(thousand) = 33; +-- explain +-- select min(tenthous) from tenk1 where thousand = 33; +select min(tenthous) from tenk1 where udf(thousand) = 33; + +-- [SPARK-17348] Correlated column is not allowed in a non-equality predicate +-- check parameter propagation into an indexscan subquery +-- explain +-- select f1, (select min(unique1) from tenk1 where unique1 > f1) AS gt +-- from int4_tbl; +-- select f1, (select min(unique1) from tenk1 where unique1 > f1) AS gt +-- from int4_tbl; + +-- check some cases that were handled incorrectly in 8.3.0 +-- explain +-- select distinct max(unique2) from tenk1; +select distinct max(udf(unique2)) from tenk1; +-- explain +-- select max(unique2) from tenk1 order by 1; +select max(unique2) from tenk1 order by udf(1); +-- explain +-- select max(unique2) from tenk1 order by max(unique2); +select max(unique2) from tenk1 order by max(udf(unique2)); +-- explain +-- select max(unique2) from tenk1 order by max(unique2)+1; +select udf(max(udf(unique2))) from tenk1 order by udf(max(unique2))+1; +-- explain +-- select max(unique2), generate_series(1,3) as g from tenk1 order by g desc; +select t1.max_unique2, udf(g) from (select max(udf(unique2)) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc; + +-- interesting corner case: constant gets optimized into a seqscan +-- explain +-- select max(100) from tenk1; +select udf(max(100)) from tenk1; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part3.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part3.sql new file mode 100644 index 000000000000..b11c8c05f310 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part3.sql @@ -0,0 +1,271 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- AGGREGATES [Part 3] +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L352-L605 + +-- This test file was converted from postgreSQL/aggregates_part3.sql. + +-- [SPARK-28865] Table inheritance +-- try it on an inheritance tree +-- create table minmaxtest(f1 int); +-- create table minmaxtest1() inherits (minmaxtest); +-- create table minmaxtest2() inherits (minmaxtest); +-- create table minmaxtest3() inherits (minmaxtest); +-- create index minmaxtesti on minmaxtest(f1); +-- create index minmaxtest1i on minmaxtest1(f1); +-- create index minmaxtest2i on minmaxtest2(f1 desc); +-- create index minmaxtest3i on minmaxtest3(f1) where f1 is not null; + +-- insert into minmaxtest values(11), (12); +-- insert into minmaxtest1 values(13), (14); +-- insert into minmaxtest2 values(15), (16); +-- insert into minmaxtest3 values(17), (18); + +-- explain (costs off) +-- select min(f1), max(f1) from minmaxtest; +-- select min(f1), max(f1) from minmaxtest; + +-- DISTINCT doesn't do anything useful here, but it shouldn't fail +-- explain (costs off) +-- select distinct min(f1), max(f1) from minmaxtest; +-- select distinct min(f1), max(f1) from minmaxtest; + +-- drop table minmaxtest cascade; + +-- check for correct detection of nested-aggregate errors +select udf(max(min(unique1))) from tenk1; +-- select (select max(min(unique1)) from int8_tbl) from tenk1; + +-- These tests only test the explain. Skip these tests. +-- +-- Test removal of redundant GROUP BY columns +-- + +-- create temp table t1 (a int, b int, c int, d int, primary key (a, b)); +-- create temp table t2 (x int, y int, z int, primary key (x, y)); +-- create temp table t3 (a int, b int, c int, primary key(a, b) deferrable); + +-- Non-primary-key columns can be removed from GROUP BY +-- explain (costs off) select * from t1 group by a,b,c,d; + +-- No removal can happen if the complete PK is not present in GROUP BY +-- explain (costs off) select a,c from t1 group by a,c,d; + +-- Test removal across multiple relations +-- explain (costs off) select * +-- from t1 inner join t2 on t1.a = t2.x and t1.b = t2.y +-- group by t1.a,t1.b,t1.c,t1.d,t2.x,t2.y,t2.z; + +-- Test case where t1 can be optimized but not t2 +-- explain (costs off) select t1.*,t2.x,t2.z +-- from t1 inner join t2 on t1.a = t2.x and t1.b = t2.y +-- group by t1.a,t1.b,t1.c,t1.d,t2.x,t2.z; + +-- Cannot optimize when PK is deferrable +-- explain (costs off) select * from t3 group by a,b,c; + +-- drop table t1; +-- drop table t2; +-- drop table t3; + +-- [SPARK-27974] Add built-in Aggregate Function: array_agg +-- +-- Test combinations of DISTINCT and/or ORDER BY +-- + +-- select array_agg(a order by b) +-- from (values (1,4),(2,3),(3,1),(4,2)) v(a,b); +-- select array_agg(a order by a) +-- from (values (1,4),(2,3),(3,1),(4,2)) v(a,b); +-- select array_agg(a order by a desc) +-- from (values (1,4),(2,3),(3,1),(4,2)) v(a,b); +-- select array_agg(b order by a desc) +-- from (values (1,4),(2,3),(3,1),(4,2)) v(a,b); + +-- select array_agg(distinct a) +-- from (values (1),(2),(1),(3),(null),(2)) v(a); +-- select array_agg(distinct a order by a) +-- from (values (1),(2),(1),(3),(null),(2)) v(a); +-- select array_agg(distinct a order by a desc) +-- from (values (1),(2),(1),(3),(null),(2)) v(a); +-- select array_agg(distinct a order by a desc nulls last) +-- from (values (1),(2),(1),(3),(null),(2)) v(a); + +-- Skip the test below because it requires 4 UDAFs: aggf_trans, aggfns_trans, aggfstr, and aggfns +-- multi-arg aggs, strict/nonstrict, distinct/order by + +-- select aggfstr(a,b,c) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c); +-- select aggfns(a,b,c) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c); + +-- select aggfstr(distinct a,b,c) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,3) i; +-- select aggfns(distinct a,b,c) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,3) i; + +-- select aggfstr(distinct a,b,c order by b) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,3) i; +-- select aggfns(distinct a,b,c order by b) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,3) i; + +-- test specific code paths + +-- [SPARK-28768] Implement more text pattern operators +-- select aggfns(distinct a,a,c order by c using ~<~,a) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,2) i; +-- select aggfns(distinct a,a,c order by c using ~<~) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,2) i; +-- select aggfns(distinct a,a,c order by a) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,2) i; +-- select aggfns(distinct a,b,c order by a,c using ~<~,b) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,2) i; + +-- check node I/O via view creation and usage, also deparsing logic + +-- create view agg_view1 as +-- select aggfns(a,b,c) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c); + +-- select * from agg_view1; +-- select pg_get_viewdef('agg_view1'::regclass); + +-- create or replace view agg_view1 as +-- select aggfns(distinct a,b,c) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,3) i; + +-- select * from agg_view1; +-- select pg_get_viewdef('agg_view1'::regclass); + +-- create or replace view agg_view1 as +-- select aggfns(distinct a,b,c order by b) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,3) i; + +-- select * from agg_view1; +-- select pg_get_viewdef('agg_view1'::regclass); + +-- create or replace view agg_view1 as +-- select aggfns(a,b,c order by b+1) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c); + +-- select * from agg_view1; +-- select pg_get_viewdef('agg_view1'::regclass); + +-- create or replace view agg_view1 as +-- select aggfns(a,a,c order by b) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c); + +-- select * from agg_view1; +-- select pg_get_viewdef('agg_view1'::regclass); + +-- create or replace view agg_view1 as +-- select aggfns(a,b,c order by c using ~<~) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c); + +-- select * from agg_view1; +-- select pg_get_viewdef('agg_view1'::regclass); + +-- create or replace view agg_view1 as +-- select aggfns(distinct a,b,c order by a,c using ~<~,b) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,2) i; + +-- select * from agg_view1; +-- select pg_get_viewdef('agg_view1'::regclass); + +-- drop view agg_view1; + +-- incorrect DISTINCT usage errors + +-- select aggfns(distinct a,b,c order by i) +-- from (values (1,1,'foo')) v(a,b,c), generate_series(1,2) i; +-- select aggfns(distinct a,b,c order by a,b+1) +-- from (values (1,1,'foo')) v(a,b,c), generate_series(1,2) i; +-- select aggfns(distinct a,b,c order by a,b,i,c) +-- from (values (1,1,'foo')) v(a,b,c), generate_series(1,2) i; +-- select aggfns(distinct a,a,c order by a,b) +-- from (values (1,1,'foo')) v(a,b,c), generate_series(1,2) i; + +-- [SPARK-27978] Add built-in Aggregate Functions: string_agg +-- string_agg tests +-- select string_agg(a,',') from (values('aaaa'),('bbbb'),('cccc')) g(a); +-- select string_agg(a,',') from (values('aaaa'),(null),('bbbb'),('cccc')) g(a); +-- select string_agg(a,'AB') from (values(null),(null),('bbbb'),('cccc')) g(a); +-- select string_agg(a,',') from (values(null),(null)) g(a); + +-- check some implicit casting cases, as per bug #5564 +-- select string_agg(distinct f1, ',' order by f1) from varchar_tbl; -- ok +-- select string_agg(distinct f1::text, ',' order by f1) from varchar_tbl; -- not ok +-- select string_agg(distinct f1, ',' order by f1::text) from varchar_tbl; -- not ok +-- select string_agg(distinct f1::text, ',' order by f1::text) from varchar_tbl; -- ok + +-- [SPARK-28121] decode can not accept 'hex' as charset +-- string_agg bytea tests +-- CREATE TABLE bytea_test_table(v BINARY) USING parquet; + +-- select string_agg(v, '') from bytea_test_table; + +-- insert into bytea_test_table values(decode('ff','hex')); + +-- select string_agg(v, '') from bytea_test_table; + +-- insert into bytea_test_table values(decode('aa','hex')); + +-- select string_agg(v, '') from bytea_test_table; +-- select string_agg(v, NULL) from bytea_test_table; +-- select string_agg(v, decode('ee', 'hex')) from bytea_test_table; + +-- drop table bytea_test_table; + +-- FILTER tests + +-- select min(unique1) filter (where unique1 > 100) from tenk1; + +-- select sum(1/ten) filter (where ten > 0) from tenk1; + +-- select ten, sum(distinct four) filter (where four::text ~ '123') from onek a +-- group by ten; + +-- select ten, sum(distinct four) filter (where four > 10) from onek a +-- group by ten +-- having exists (select 1 from onek b where sum(distinct a.four) = b.four); + +-- [SPARK-28682] ANSI SQL: Collation Support +-- select max(foo COLLATE "C") filter (where (bar collate "POSIX") > '0') +-- from (values ('a', 'b')) AS v(foo,bar); + +-- outer reference in FILTER (PostgreSQL extension) +select udf((select udf(count(*)) + from (values (1)) t0(inner_c))) as col +from (values (2),(3)) t1(outer_c); -- inner query is aggregation query +-- select (select count(*) filter (where outer_c <> 0) +-- from (values (1)) t0(inner_c)) +-- from (values (2),(3)) t1(outer_c); -- outer query is aggregation query +-- select (select count(inner_c) filter (where outer_c <> 0) +-- from (values (1)) t0(inner_c)) +-- from (values (2),(3)) t1(outer_c); -- inner query is aggregation query +-- select +-- (select max((select i.unique2 from tenk1 i where i.unique1 = o.unique1)) +-- filter (where o.unique1 < 10)) +-- from tenk1 o; -- outer query is aggregation query + +-- subquery in FILTER clause (PostgreSQL extension) +-- select sum(unique1) FILTER (WHERE +-- unique1 IN (SELECT unique1 FROM onek where unique1 < 100)) FROM tenk1; + +-- exercise lots of aggregate parts with FILTER +-- select aggfns(distinct a,b,c order by a,c using ~<~,b) filter (where a > 1) +-- from (values (1,3,'foo'),(0,null,null),(2,2,'bar'),(3,1,'baz')) v(a,b,c), +-- generate_series(1,2) i; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part4.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part4.sql new file mode 100644 index 000000000000..dd14a7db0db1 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part4.sql @@ -0,0 +1,421 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- AGGREGATES [Part 4] +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L607-L997 + +-- This test file was converted from postgreSQL/aggregates_part4.sql. + +-- [SPARK-27980] Ordered-Set Aggregate Functions +-- ordered-set aggregates + +-- select p, percentile_cont(p) within group (order by x::float8) +-- from generate_series(1,5) x, +-- (values (0::float8),(0.1),(0.25),(0.4),(0.5),(0.6),(0.75),(0.9),(1)) v(p) +-- group by p order by p; + +-- select p, percentile_cont(p order by p) within group (order by x) -- error +-- from generate_series(1,5) x, +-- (values (0::float8),(0.1),(0.25),(0.4),(0.5),(0.6),(0.75),(0.9),(1)) v(p) +-- group by p order by p; + +-- select p, sum() within group (order by x::float8) -- error +-- from generate_series(1,5) x, +-- (values (0::float8),(0.1),(0.25),(0.4),(0.5),(0.6),(0.75),(0.9),(1)) v(p) +-- group by p order by p; + +-- select p, percentile_cont(p,p) -- error +-- from generate_series(1,5) x, +-- (values (0::float8),(0.1),(0.25),(0.4),(0.5),(0.6),(0.75),(0.9),(1)) v(p) +-- group by p order by p; + +select percentile_cont(0.5) within group (order by b) from aggtest; +select percentile_cont(0.5) within group (order by b), sum(b) from aggtest; +select percentile_cont(0.5) within group (order by thousand) from tenk1; +select percentile_disc(0.5) within group (order by thousand) from tenk1; +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- select rank(3) within group (order by x) +-- from (values (1),(1),(2),(2),(3),(3),(4)) v(x); +-- select cume_dist(3) within group (order by x) +-- from (values (1),(1),(2),(2),(3),(3),(4)) v(x); +-- select percent_rank(3) within group (order by x) +-- from (values (1),(1),(2),(2),(3),(3),(4),(5)) v(x); +-- select dense_rank(3) within group (order by x) +-- from (values (1),(1),(2),(2),(3),(3),(4)) v(x); + +-- [SPARK-27980] Ordered-Set Aggregate Functions +-- select percentile_disc(array[0,0.1,0.25,0.5,0.75,0.9,1]) within group (order by thousand) +-- from tenk1; +-- select percentile_cont(array[0,0.25,0.5,0.75,1]) within group (order by thousand) +-- from tenk1; +-- select percentile_disc(array[[null,1,0.5],[0.75,0.25,null]]) within group (order by thousand) +-- from tenk1; +-- select percentile_cont(array[0,1,0.25,0.75,0.5,1,0.3,0.32,0.35,0.38,0.4]) within group (order by x) +-- from generate_series(1,6) x; + +-- [SPARK-27980] Ordered-Set Aggregate Functions +-- [SPARK-28382] Array Functions: unnest +-- select ten, mode() within group (order by string4) from tenk1 group by ten; + +-- select percentile_disc(array[0.25,0.5,0.75]) within group (order by x) +-- from unnest('{fred,jim,fred,jack,jill,fred,jill,jim,jim,sheila,jim,sheila}'::text[]) u(x); + +-- [SPARK-28669] System Information Functions +-- check collation propagates up in suitable cases: +-- select pg_collation_for(percentile_disc(1) within group (order by x collate "POSIX")) +-- from (values ('fred'),('jim')) v(x); + +-- test_rank and test_percentile_disc function created by create_aggregate.sql +-- ordered-set aggs created with CREATE AGGREGATE +-- select test_rank(3) within group (order by x) +-- from (values (1),(1),(2),(2),(3),(3),(4)) v(x); +-- select test_percentile_disc(0.5) within group (order by thousand) from tenk1; + +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- ordered-set aggs can't use ungrouped vars in direct args: +-- select rank(x) within group (order by x) from generate_series(1,5) x; + +-- [SPARK-27980] Ordered-Set Aggregate Functions +-- outer-level agg can't use a grouped arg of a lower level, either: +-- select array(select percentile_disc(a) within group (order by x) +-- from (values (0.3),(0.7)) v(a) group by a) +-- from generate_series(1,5) g(x); + +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- agg in the direct args is a grouping violation, too: +--select rank(sum(x)) within group (order by x) from generate_series(1,5) x; + +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- hypothetical-set type unification and argument-count failures: +-- select rank(3) within group (order by x) from (values ('fred'),('jim')) v(x); +-- select rank(3) within group (order by stringu1,stringu2) from tenk1; +-- select rank('fred') within group (order by x) from generate_series(1,5) x; +-- select rank('adam'::text collate "C") within group (order by x collate "POSIX") +-- from (values ('fred'),('jim')) v(x); +-- hypothetical-set type unification successes: +-- select rank('adam'::varchar) within group (order by x) from (values ('fred'),('jim')) v(x); +-- select rank('3') within group (order by x) from generate_series(1,5) x; + +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- divide by zero check +-- select percent_rank(0) within group (order by x) from generate_series(1,0) x; + +-- [SPARK-27980] Ordered-Set Aggregate Functions +-- deparse and multiple features: +-- create view aggordview1 as +-- select ten, +-- percentile_disc(0.5) within group (order by thousand) as p50, +-- percentile_disc(0.5) within group (order by thousand) filter (where hundred=1) as px, +-- rank(5,'AZZZZ',50) within group (order by hundred, string4 desc, hundred) +-- from tenk1 +-- group by ten order by ten; + +-- select pg_get_viewdef('aggordview1'); +-- select * from aggordview1 order by ten; +-- drop view aggordview1; + +-- least_agg created by create_aggregate.sql +-- variadic aggregates +-- select least_agg(q1,q2) from int8_tbl; +-- select least_agg(variadic array[q1,q2]) from int8_tbl; + + +-- Skip these tests because we do not support create type +-- test aggregates with common transition functions share the same states +-- begin work; + +-- create type avg_state as (total bigint, count bigint); + +-- create or replace function avg_transfn(state avg_state, n int) returns avg_state as +-- $$ +-- declare new_state avg_state; +-- begin +-- raise notice 'avg_transfn called with %', n; +-- if state is null then +-- if n is not null then +-- new_state.total := n; +-- new_state.count := 1; +-- return new_state; +-- end if; +-- return null; +-- elsif n is not null then +-- state.total := state.total + n; +-- state.count := state.count + 1; +-- return state; +-- end if; +-- +-- return null; +-- end +-- $$ language plpgsql; + +-- create function avg_finalfn(state avg_state) returns int4 as +-- $$ +-- begin +-- if state is null then +-- return NULL; +-- else +-- return state.total / state.count; +-- end if; +-- end +-- $$ language plpgsql; + +-- create function sum_finalfn(state avg_state) returns int4 as +-- $$ +-- begin +-- if state is null then +-- return NULL; +-- else +-- return state.total; +-- end if; +-- end +-- $$ language plpgsql; + +-- create aggregate my_avg(int4) +-- ( +-- stype = avg_state, +-- sfunc = avg_transfn, +-- finalfunc = avg_finalfn +-- ); +-- +-- create aggregate my_sum(int4) +-- ( +-- stype = avg_state, +-- sfunc = avg_transfn, +-- finalfunc = sum_finalfn +-- ); + +-- aggregate state should be shared as aggs are the same. +-- select my_avg(one),my_avg(one) from (values(1),(3)) t(one); + +-- aggregate state should be shared as transfn is the same for both aggs. +-- select my_avg(one),my_sum(one) from (values(1),(3)) t(one); + +-- same as previous one, but with DISTINCT, which requires sorting the input. +-- select my_avg(distinct one),my_sum(distinct one) from (values(1),(3),(1)) t(one); + +-- shouldn't share states due to the distinctness not matching. +-- select my_avg(distinct one),my_sum(one) from (values(1),(3)) t(one); + +-- shouldn't share states due to the filter clause not matching. +-- select my_avg(one) filter (where one > 1),my_sum(one) from (values(1),(3)) t(one); + +-- this should not share the state due to different input columns. +-- select my_avg(one),my_sum(two) from (values(1,2),(3,4)) t(one,two); + +-- [SPARK-27980] Ordered-Set Aggregate Functions +-- exercise cases where OSAs share state +-- select +-- percentile_cont(0.5) within group (order by a), +-- percentile_disc(0.5) within group (order by a) +-- from (values(1::float8),(3),(5),(7)) t(a); + +-- select +-- percentile_cont(0.25) within group (order by a), +-- percentile_disc(0.5) within group (order by a) +-- from (values(1::float8),(3),(5),(7)) t(a); + +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- these can't share state currently +-- select +-- rank(4) within group (order by a), +-- dense_rank(4) within group (order by a) +-- from (values(1),(3),(5),(7)) t(a); + +-- test that aggs with the same sfunc and initcond share the same agg state +-- create aggregate my_sum_init(int4) +-- ( +-- stype = avg_state, +-- sfunc = avg_transfn, +-- finalfunc = sum_finalfn, +-- initcond = '(10,0)' +-- ); + +-- create aggregate my_avg_init(int4) +-- ( +-- stype = avg_state, +-- sfunc = avg_transfn, +-- finalfunc = avg_finalfn, +-- initcond = '(10,0)' +-- ); + +-- create aggregate my_avg_init2(int4) +-- ( +-- stype = avg_state, +-- sfunc = avg_transfn, +-- finalfunc = avg_finalfn, +-- initcond = '(4,0)' +-- ); + +-- state should be shared if INITCONDs are matching +-- select my_sum_init(one),my_avg_init(one) from (values(1),(3)) t(one); + +-- Varying INITCONDs should cause the states not to be shared. +-- select my_sum_init(one),my_avg_init2(one) from (values(1),(3)) t(one); + +-- rollback; + +-- test aggregate state sharing to ensure it works if one aggregate has a +-- finalfn and the other one has none. +-- begin work; + +-- create or replace function sum_transfn(state int4, n int4) returns int4 as +-- $$ +-- declare new_state int4; +-- begin +-- raise notice 'sum_transfn called with %', n; +-- if state is null then +-- if n is not null then +-- new_state := n; +-- return new_state; +-- end if; +-- return null; +-- elsif n is not null then +-- state := state + n; +-- return state; +-- end if; +-- +-- return null; +-- end +-- $$ language plpgsql; + +-- create function halfsum_finalfn(state int4) returns int4 as +-- $$ +-- begin +-- if state is null then +-- return NULL; +-- else +-- return state / 2; +-- end if; +-- end +-- $$ language plpgsql; + +-- create aggregate my_sum(int4) +-- ( +-- stype = int4, +-- sfunc = sum_transfn +-- ); + +-- create aggregate my_half_sum(int4) +-- ( +-- stype = int4, +-- sfunc = sum_transfn, +-- finalfunc = halfsum_finalfn +-- ); + +-- Agg state should be shared even though my_sum has no finalfn +-- select my_sum(one),my_half_sum(one) from (values(1),(2),(3),(4)) t(one); + +-- rollback; + + +-- test that the aggregate transition logic correctly handles +-- transition / combine functions returning NULL + +-- First test the case of a normal transition function returning NULL +-- BEGIN; +-- CREATE FUNCTION balkifnull(int8, int4) +-- RETURNS int8 +-- STRICT +-- LANGUAGE plpgsql AS $$ +-- BEGIN +-- IF $1 IS NULL THEN +-- RAISE 'erroneously called with NULL argument'; +-- END IF; +-- RETURN NULL; +-- END$$; + +-- CREATE AGGREGATE balk(int4) +-- ( +-- SFUNC = balkifnull(int8, int4), +-- STYPE = int8, +-- PARALLEL = SAFE, +-- INITCOND = '0' +-- ); + +-- SELECT balk(hundred) FROM tenk1; + +-- ROLLBACK; + +-- Secondly test the case of a parallel aggregate combiner function +-- returning NULL. For that use normal transition function, but a +-- combiner function returning NULL. +-- BEGIN ISOLATION LEVEL REPEATABLE READ; +-- CREATE FUNCTION balkifnull(int8, int8) +-- RETURNS int8 +-- PARALLEL SAFE +-- STRICT +-- LANGUAGE plpgsql AS $$ +-- BEGIN +-- IF $1 IS NULL THEN +-- RAISE 'erroneously called with NULL argument'; +-- END IF; +-- RETURN NULL; +-- END$$; + +-- CREATE AGGREGATE balk(int4) +-- ( +-- SFUNC = int4_sum(int8, int4), +-- STYPE = int8, +-- COMBINEFUNC = balkifnull(int8, int8), +-- PARALLEL = SAFE, +-- INITCOND = '0' +-- ); + +-- force use of parallelism +-- ALTER TABLE tenk1 set (parallel_workers = 4); +-- SET LOCAL parallel_setup_cost=0; +-- SET LOCAL max_parallel_workers_per_gather=4; + +-- EXPLAIN (COSTS OFF) SELECT balk(hundred) FROM tenk1; +-- SELECT balk(hundred) FROM tenk1; + +-- ROLLBACK; + +-- test coverage for aggregate combine/serial/deserial functions +-- BEGIN ISOLATION LEVEL REPEATABLE READ; + +-- SET parallel_setup_cost = 0; +-- SET parallel_tuple_cost = 0; +-- SET min_parallel_table_scan_size = 0; +-- SET max_parallel_workers_per_gather = 4; +-- SET enable_indexonlyscan = off; + +-- [SPARK-28663] Aggregate Functions for Statistics +-- variance(int4) covers numeric_poly_combine +-- sum(int8) covers int8_avg_combine +-- regr_count(float8, float8) covers int8inc_float8_float8 and aggregates with > 1 arg +-- EXPLAIN (COSTS OFF, VERBOSE) +-- SELECT variance(unique1::int4), sum(unique1::int8), regr_count(unique1::float8, unique1::float8) FROM tenk1; + +-- SELECT variance(unique1::int4), sum(unique1::int8), regr_count(unique1::float8, unique1::float8) FROM tenk1; + +-- ROLLBACK; + +-- [SPARK-28661] Hypothetical-Set Aggregate Functions +-- test coverage for dense_rank +-- SELECT dense_rank(x) WITHIN GROUP (ORDER BY x) FROM (VALUES (1),(1),(2),(2),(3),(3)) v(x) GROUP BY (x) ORDER BY 1; + + +-- [SPARK-28664] ORDER BY in aggregate function +-- Ensure that the STRICT checks for aggregates does not take NULLness +-- of ORDER BY columns into account. See bug report around +-- 2a505161-2727-2473-7c46-591ed108ac52@email.cz +-- SELECT min(x ORDER BY y) FROM (VALUES(1, NULL)) AS d(x,y); +-- SELECT min(x ORDER BY y) FROM (VALUES(1, 2)) AS d(x,y); + +-- [SPARK-28382] Array Functions: unnest +-- check collation-sensitive matching between grouping expressions +-- select v||'a', case v||'a' when 'aa' then 1 else 0 end, count(*) +-- from unnest(array['a','b']) u(v) +-- group by v||'a' order by 1; +-- select v||'a', case when v||'a' = 'aa' then 1 else 0 end, count(*) +-- from unnest(array['a','b']) u(v) +-- group by v||'a' order by 1; + +-- Make sure that generation of HashAggregate for uniqification purposes +-- does not lead to array overflow due to unexpected duplicate hash keys +-- see CAFeeJoKKu0u+A_A9R9316djW-YW3-+Gtgvy3ju655qRHR3jtdA@mail.gmail.com +-- explain (costs off) +-- select 1 from tenk1 +-- where (hundred, thousand) in (select twothousand, twothousand from onek); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-case.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-case.sql new file mode 100644 index 000000000000..4ee0dba8fa9c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-case.sql @@ -0,0 +1,263 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- CASE +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/case.sql +-- Test the CASE statement +-- +-- This test file was converted from postgreSQL/case.sql. + +CREATE TABLE CASE_TBL ( + i integer, + f double +) USING parquet; + +CREATE TABLE CASE2_TBL ( + i integer, + j integer +) USING parquet; + +INSERT INTO CASE_TBL VALUES (1, 10.1); +INSERT INTO CASE_TBL VALUES (2, 20.2); +INSERT INTO CASE_TBL VALUES (3, -30.3); +INSERT INTO CASE_TBL VALUES (4, NULL); + +INSERT INTO CASE2_TBL VALUES (1, -1); +INSERT INTO CASE2_TBL VALUES (2, -2); +INSERT INTO CASE2_TBL VALUES (3, -3); +INSERT INTO CASE2_TBL VALUES (2, -4); +INSERT INTO CASE2_TBL VALUES (1, NULL); +INSERT INTO CASE2_TBL VALUES (NULL, -6); + +-- +-- Simplest examples without tables +-- + +SELECT '3' AS `One`, + CASE + WHEN udf(1 < 2) THEN 3 + END AS `Simple WHEN`; + +SELECT '' AS `One`, + CASE + WHEN 1 > 2 THEN udf(3) + END AS `Simple default`; + +SELECT '3' AS `One`, + CASE + WHEN udf(1) < 2 THEN udf(3) + ELSE udf(4) + END AS `Simple ELSE`; + +SELECT udf('4') AS `One`, + CASE + WHEN 1 > 2 THEN 3 + ELSE 4 + END AS `ELSE default`; + +SELECT udf('6') AS `One`, + CASE + WHEN udf(1 > 2) THEN 3 + WHEN udf(4) < 5 THEN 6 + ELSE 7 + END AS `Two WHEN with default`; + +SELECT '7' AS `None`, + CASE WHEN rand() < udf(0) THEN 1 + END AS `NULL on no matches`; + +-- Constant-expression folding shouldn't evaluate unreachable subexpressions +SELECT CASE WHEN udf(1=0) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END; +SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END; + +-- SPARK-39122: Python UDF does not follow the conditional expression evaluation order +-- SELECT CASE WHEN i > 100 THEN udf(1/0) ELSE udf(0) END FROM case_tbl; + +-- Test for cases involving untyped literals in test expression +SELECT CASE 'a' WHEN 'a' THEN udf(1) ELSE udf(2) END; + +-- +-- Examples of targets involving tables +-- + +SELECT '' AS `Five`, + CASE + WHEN i >= 3 THEN i + END AS `>= 3 or Null` + FROM CASE_TBL; + +SELECT '' AS `Five`, + CASE WHEN i >= 3 THEN (i + i) + ELSE i + END AS `Simplest Math` + FROM CASE_TBL; + +SELECT '' AS `Five`, i AS `Value`, + CASE WHEN (i < 0) THEN 'small' + WHEN (i = 0) THEN 'zero' + WHEN (i = 1) THEN 'one' + WHEN (i = 2) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL; + +SELECT '' AS `Five`, + CASE WHEN ((i < 0) or (i < 0)) THEN 'small' + WHEN ((i = 0) or (i = 0)) THEN 'zero' + WHEN ((i = 1) or (i = 1)) THEN 'one' + WHEN ((i = 2) or (i = 2)) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL; + +-- +-- Examples of qualifications involving tables +-- + +-- +-- NULLIF() and COALESCE() +-- Shorthand forms for typical CASE constructs +-- defined in the SQL standard. +-- + +SELECT * FROM CASE_TBL WHERE udf(COALESCE(f,i)) = 4; + +SELECT * FROM CASE_TBL WHERE udf(NULLIF(f,i)) = 2; + +SELECT udf(COALESCE(a.f, b.i, b.j)) + FROM CASE_TBL a, CASE2_TBL b; + +SELECT * + FROM CASE_TBL a, CASE2_TBL b + WHERE udf(COALESCE(a.f, b.i, b.j)) = 2; + +SELECT udf('') AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, + NULLIF(b.i, 4) AS `NULLIF(b.i,4)` + FROM CASE_TBL a, CASE2_TBL b; + +SELECT '' AS `Two`, * + FROM CASE_TBL a, CASE2_TBL b + WHERE udf(COALESCE(f,b.i) = 2); + +-- We don't support update now. +-- +-- Examples of updates involving tables +-- + +-- UPDATE CASE_TBL +-- SET i = CASE WHEN i >= 3 THEN (- i) +-- ELSE (2 * i) END; + +-- SELECT * FROM CASE_TBL; + +-- UPDATE CASE_TBL +-- SET i = CASE WHEN i >= 2 THEN (2 * i) +-- ELSE (3 * i) END; + +-- SELECT * FROM CASE_TBL; + +-- UPDATE CASE_TBL +-- SET i = CASE WHEN b.i >= 2 THEN (2 * j) +-- ELSE (3 * j) END +-- FROM CASE2_TBL b +-- WHERE j = -CASE_TBL.i; + +-- SELECT * FROM CASE_TBL; + +-- +-- Nested CASE expressions +-- + +-- This test exercises a bug caused by aliasing econtext->caseValue_isNull +-- with the isNull argument of the inner CASE's CaseExpr evaluation. After +-- evaluating the vol(null) expression in the inner CASE's second WHEN-clause, +-- the isNull flag for the case test value incorrectly became true, causing +-- the third WHEN-clause not to match. The volatile function calls are needed +-- to prevent constant-folding in the planner, which would hide the bug. + +-- Wrap this in a single transaction so the transient '=' operator doesn't +-- cause problems in concurrent sessions +-- BEGIN; + +-- CREATE FUNCTION vol(text) returns text as +-- 'begin return $1; end' language plpgsql volatile; + +SELECT CASE + (CASE vol('bar') + WHEN udf('foo') THEN 'it was foo!' + WHEN udf(vol(null)) THEN 'null input' + WHEN 'bar' THEN 'it was bar!' END + ) + WHEN udf('it was foo!') THEN 'foo recognized' + WHEN 'it was bar!' THEN udf('bar recognized') + ELSE 'unrecognized' END AS col; + +-- We don't support the features below: +-- 1. CREATE DOMAIN ... +-- 2. CREATE OPERATOR ... +-- 3. CREATE TYPE ... + +-- In this case, we can't inline the SQL function without confusing things. +-- CREATE DOMAIN foodomain AS text; + +-- CREATE FUNCTION volfoo(text) returns foodomain as +-- 'begin return $1::foodomain; end' language plpgsql volatile; + +-- CREATE FUNCTION inline_eq(foodomain, foodomain) returns boolean as +-- 'SELECT CASE $2::text WHEN $1::text THEN true ELSE false END' language sql; + +-- CREATE OPERATOR = (procedure = inline_eq, +-- leftarg = foodomain, rightarg = foodomain); + +-- SELECT CASE volfoo('bar') WHEN 'foo'::foodomain THEN 'is foo' ELSE 'is not foo' END; + +-- ROLLBACK; + +-- Test multiple evaluation of a CASE arg that is a read/write object (#14472) +-- Wrap this in a single transaction so the transient '=' operator doesn't +-- cause problems in concurrent sessions +-- BEGIN; + +-- CREATE DOMAIN arrdomain AS int[]; + +-- CREATE FUNCTION make_ad(int,int) returns arrdomain as +-- 'declare x arrdomain; +-- begin +-- x := array[$1,$2]; +-- return x; +-- end' language plpgsql volatile; + +-- CREATE FUNCTION ad_eq(arrdomain, arrdomain) returns boolean as +-- 'begin return array_eq($1, $2); end' language plpgsql; + +-- CREATE OPERATOR = (procedure = ad_eq, +-- leftarg = arrdomain, rightarg = arrdomain); + +-- SELECT CASE make_ad(1,2) +-- WHEN array[2,4]::arrdomain THEN 'wrong' +-- WHEN array[2,5]::arrdomain THEN 'still wrong' +-- WHEN array[1,2]::arrdomain THEN 'right' +-- END; + +-- ROLLBACK; + +-- Test interaction of CASE with ArrayCoerceExpr (bug #15471) +-- BEGIN; + +-- CREATE TYPE casetestenum AS ENUM ('e', 'f', 'g'); + +-- SELECT +-- CASE 'foo'::text +-- WHEN 'foo' THEN ARRAY['a', 'b', 'c', 'd'] || enum_range(NULL::casetestenum)::text[] +-- ELSE ARRAY['x', 'y'] +-- END; + +-- ROLLBACK; + +-- +-- Clean up +-- + +DROP TABLE CASE_TBL; +DROP TABLE CASE2_TBL; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-join.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-join.sql new file mode 100644 index 000000000000..ff796b4e41fd --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-join.sql @@ -0,0 +1,2090 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- JOIN +-- Test JOIN clauses +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/join.sql +-- +-- This test file was converted from postgreSQL/join.sql. + +-- Disable BroadcastHashJoin optimization to avoid changing result order when we enable AQE +--SET spark.sql.autoBroadcastJoinThreshold = -1 + +CREATE OR REPLACE TEMPORARY VIEW INT2_TBL(f1) AS VALUES + (smallint(trim('0 '))), + (smallint(trim(' 1234 '))), + (smallint(trim(' -1234'))), + (smallint('32767')), + (smallint('-32767')); +CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM + (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) + AS v(f1); +CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM + (VALUES + (123, 456), + (123, 4567890123456789), + (4567890123456789, 123), + (4567890123456789, 4567890123456789), + (4567890123456789, -4567890123456789)) + AS v(q1, q2); +CREATE OR REPLACE TEMPORARY VIEW FLOAT8_TBL AS SELECT * FROM + (VALUES (0.0), (1004.30), (-34.84), + (cast('1.2345678901234e+200' as double)), (cast('1.2345678901234e-200' as double))) + AS v(f1); +CREATE OR REPLACE TEMPORARY VIEW TEXT_TBL AS SELECT * FROM + (VALUES ('doh!'), ('hi de ho neighbor')) + AS v(f1); +CREATE OR REPLACE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1; + +CREATE TABLE J1_TBL ( + i integer, + j integer, + t string +) USING parquet; + +CREATE TABLE J2_TBL ( + i integer, + k integer +) USING parquet; + + +INSERT INTO J1_TBL VALUES (1, 4, 'one'); +INSERT INTO J1_TBL VALUES (2, 3, 'two'); +INSERT INTO J1_TBL VALUES (3, 2, 'three'); +INSERT INTO J1_TBL VALUES (4, 1, 'four'); +INSERT INTO J1_TBL VALUES (5, 0, 'five'); +INSERT INTO J1_TBL VALUES (6, 6, 'six'); +INSERT INTO J1_TBL VALUES (7, 7, 'seven'); +INSERT INTO J1_TBL VALUES (8, 8, 'eight'); +INSERT INTO J1_TBL VALUES (0, NULL, 'zero'); +INSERT INTO J1_TBL VALUES (NULL, NULL, 'null'); +INSERT INTO J1_TBL VALUES (NULL, 0, 'zero'); + +INSERT INTO J2_TBL VALUES (1, -1); +INSERT INTO J2_TBL VALUES (2, 2); +INSERT INTO J2_TBL VALUES (3, -3); +INSERT INTO J2_TBL VALUES (2, 4); +INSERT INTO J2_TBL VALUES (5, -5); +INSERT INTO J2_TBL VALUES (5, -5); +INSERT INTO J2_TBL VALUES (0, NULL); +INSERT INTO J2_TBL VALUES (NULL, NULL); +INSERT INTO J2_TBL VALUES (NULL, 0); + +-- [SPARK-20856] Do not need onerow because it only used for test statement using nested joins +-- useful in some tests below +-- create temp table onerow(); +-- insert into onerow default values; +-- analyze onerow; + + +-- +-- CORRELATION NAMES +-- Make sure that table/column aliases are supported +-- before diving into more complex join syntax. +-- + +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t) + FROM J1_TBL AS tx; + +SELECT udf(udf('')) AS `xxx`, udf(udf(i)), udf(j), udf(t) + FROM J1_TBL tx; + +SELECT udf('') AS `xxx`, a, udf(udf(b)), c + FROM J1_TBL AS t1 (a, b, c); + +SELECT udf('') AS `xxx`, udf(a), udf(b), udf(udf(c)) + FROM J1_TBL t1 (a, b, c); + +SELECT udf('') AS `xxx`, udf(a), b, udf(c), udf(d), e + FROM J1_TBL t1 (a, b, c), J2_TBL t2 (d, e); + +-- [SPARK-28377] Fully support correlation names in the FROM clause +-- SELECT '' AS "xxx", t1.a, t2.e +-- FROM J1_TBL t1 (a, b, c), J2_TBL t2 (d, e) +-- WHERE t1.a = t2.d; + + +-- +-- CROSS JOIN +-- Qualifications are not allowed on cross joins, +-- which degenerate into a standard unqualified inner join. +-- + +SELECT udf('') AS `xxx`, * + FROM J1_TBL CROSS JOIN J2_TBL; + +-- ambiguous column +SELECT udf('') AS `xxx`, udf(i) AS i, udf(k), udf(t) AS t + FROM J1_TBL CROSS JOIN J2_TBL; + +-- resolve previous ambiguity by specifying the table name +SELECT udf('') AS `xxx`, udf(t1.i) AS i, udf(k), udf(t) + FROM J1_TBL t1 CROSS JOIN J2_TBL t2; + +SELECT udf(udf('')) AS `xxx`, udf(udf(ii)) AS ii, udf(udf(tt)) AS tt, udf(udf(kk)) + FROM (J1_TBL CROSS JOIN J2_TBL) + AS tx (ii, jj, tt, ii2, kk); + +-- [SPARK-28377] Fully support correlation names in the FROM clause +-- SELECT '' AS `xxx`, tx.ii, tx.jj, tx.kk +-- FROM (J1_TBL t1 (a, b, c) CROSS JOIN J2_TBL t2 (d, e)) +-- AS tx (ii, jj, tt, ii2, kk); + +SELECT udf('') AS `xxx`, udf(udf(j1_tbl.i)), udf(j), udf(t), udf(a.i), udf(a.k), udf(b.i), udf(b.k) + FROM J1_TBL CROSS JOIN J2_TBL a CROSS JOIN J2_TBL b; + + +-- +-- +-- Inner joins (equi-joins) +-- +-- + +-- +-- Inner joins (equi-joins) with USING clause +-- The USING syntax changes the shape of the resulting table +-- by including a column in the USING clause only once in the result. +-- + +-- Inner equi-join on specified column +SELECT udf('') AS `xxx`, udf(i) AS i, udf(j), udf(t) AS t, udf(k) + FROM J1_TBL INNER JOIN J2_TBL USING (i); + +-- Same as above, slightly different syntax +SELECT udf(udf('')) AS `xxx`, udf(i), udf(j) AS j, udf(t), udf(k) AS k + FROM J1_TBL JOIN J2_TBL USING (i); + +SELECT udf('') AS `xxx`, * + FROM J1_TBL t1 (a, b, c) JOIN J2_TBL t2 (a, d) USING (a) + ORDER BY udf(udf(a)), udf(d); + +-- [SPARK-28377] Fully support correlation names in the FROM clause +-- SELECT '' AS `xxx`, * +-- FROM J1_TBL t1 (a, b, c) JOIN J2_TBL t2 (a, b) USING (b) +-- ORDER BY b, t1.a; + + +-- +-- NATURAL JOIN +-- Inner equi-join on all columns with the same name +-- + +SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL NATURAL JOIN J2_TBL; + +SELECT udf('') AS `xxx`, udf(udf(udf(a))) AS a, udf(b), udf(c), udf(d) + FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (a, d); + +SELECT udf('') AS `xxx`, udf(udf(a)), udf(udf(b)), udf(udf(c)) AS c, udf(udf(udf(d))) AS d + FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (d, a); + +-- [SPARK-28377] Fully support correlation names in the FROM clause +-- mismatch number of columns +-- currently, Postgres will fill in with underlying names +-- SELECT '' AS `xxx`, * +-- FROM J1_TBL t1 (a, b) NATURAL JOIN J2_TBL t2 (a); + + +-- +-- Inner joins (equi-joins) +-- + +SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(udf(J1_TBL.j)), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k) + FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) = J2_TBL.i); + +SELECT udf('') AS `xxx`, udf(udf(J1_TBL.i)), udf(udf(J1_TBL.j)), udf(udf(J1_TBL.t)), J2_TBL.i, J2_TBL.k + FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = udf(J2_TBL.k)); + + +-- +-- Non-equi-joins +-- + +SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(J1_TBL.j), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k) + FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) <= udf(udf(J2_TBL.k))); + + +-- +-- Outer joins +-- Note that OUTER is a noise word +-- + +SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL LEFT OUTER JOIN J2_TBL USING (i) + ORDER BY udf(udf(i)), udf(k), udf(t); + +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL LEFT JOIN J2_TBL USING (i) + ORDER BY udf(i), udf(udf(k)), udf(t); + +SELECT udf('') AS `xxx`, udf(udf(i)), udf(j), udf(t), udf(k) + FROM J1_TBL RIGHT OUTER JOIN J2_TBL USING (i); + +SELECT udf('') AS `xxx`, udf(i), udf(udf(j)), udf(t), udf(k) + FROM J1_TBL RIGHT JOIN J2_TBL USING (i); + +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(udf(t)), udf(k) + FROM J1_TBL FULL OUTER JOIN J2_TBL USING (i) + ORDER BY udf(udf(i)), udf(k), udf(t); + +SELECT udf('') AS `xxx`, udf(i), udf(j), t, udf(udf(k)) + FROM J1_TBL FULL JOIN J2_TBL USING (i) + ORDER BY udf(udf(i)), udf(k), udf(udf(t)); + +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(udf(k)) + FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(k) = 1); + +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(udf(i)) = udf(1)); + +-- +-- semijoin selectivity for <> +-- +-- explain (costs off) +-- select * from int4_tbl i4, tenk1 a +-- where exists(select * from tenk1 b +-- where a.twothousand = b.twothousand and a.fivethous <> b.fivethous) +-- and i4.f1 = a.tenthous; + + +-- +-- More complicated constructs +-- + +-- +-- Multiway full join +-- + +CREATE TABLE t1 (name STRING, n INTEGER) USING parquet; +CREATE TABLE t2 (name STRING, n INTEGER) USING parquet; +CREATE TABLE t3 (name STRING, n INTEGER) USING parquet; + +INSERT INTO t1 VALUES ( 'bb', 11 ); +INSERT INTO t2 VALUES ( 'bb', 12 ); +INSERT INTO t2 VALUES ( 'cc', 22 ); +INSERT INTO t2 VALUES ( 'ee', 42 ); +INSERT INTO t3 VALUES ( 'bb', 13 ); +INSERT INTO t3 VALUES ( 'cc', 23 ); +INSERT INTO t3 VALUES ( 'dd', 33 ); + +SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name); + +-- +-- Test interactions of join syntax and subqueries +-- + +-- Basic cases (we expect planner to pull up the subquery here) +SELECT * FROM +(SELECT udf(name) as name, t2.n FROM t2) as s2 +INNER JOIN +(SELECT udf(udf(name)) as name, t3.n FROM t3) s3 +USING (name); + +SELECT * FROM +(SELECT udf(udf(name)) as name, t2.n FROM t2) as s2 +LEFT JOIN +(SELECT udf(name) as name, t3.n FROM t3) s3 +USING (name); + +SELECT udf(name), udf(udf(s2.n)), udf(s3.n) FROM +(SELECT * FROM t2) as s2 +FULL JOIN +(SELECT * FROM t3) s3 +USING (name); + +-- Cases with non-nullable expressions in subquery results; +-- make sure these go to null as expected +SELECT * FROM +(SELECT udf(udf(name)) as name, udf(n) as s2_n, udf(2) as s2_2 FROM t2) as s2 +NATURAL INNER JOIN +(SELECT udf(name) as name, udf(udf(n)) as s3_n, udf(3) as s3_2 FROM t3) s3; + +SELECT * FROM +(SELECT udf(name) as name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL LEFT JOIN +(SELECT udf(udf(name)) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3; + +SELECT * FROM +(SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL FULL JOIN +(SELECT udf(udf(name)) as name, udf(udf(n)) as s3_n, 3 as s3_2 FROM t3) s3; + +SELECT * FROM +(SELECT udf(udf(name)) as name, udf(n) as s1_n, 1 as s1_1 FROM t1) as s1 +NATURAL INNER JOIN +(SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL INNER JOIN +(SELECT udf(udf(udf(name))) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3; + +SELECT * FROM +(SELECT udf(name) as name, udf(n) as s1_n, udf(udf(1)) as s1_1 FROM t1) as s1 +NATURAL FULL JOIN +(SELECT udf(name) as name, udf(udf(n)) as s2_n, udf(2) as s2_2 FROM t2) as s2 +NATURAL FULL JOIN +(SELECT udf(udf(name)) as name, udf(n) as s3_n, udf(3) as s3_2 FROM t3) s3; + +SELECT name, udf(udf(s1_n)), udf(s2_n), udf(s3_n) FROM +(SELECT name, udf(udf(n)) as s1_n FROM t1) as s1 +NATURAL FULL JOIN + (SELECT * FROM + (SELECT name, udf(n) as s2_n FROM t2) as s2 + NATURAL FULL JOIN + (SELECT name, udf(udf(n)) as s3_n FROM t3) as s3 + ) ss2; + +SELECT * FROM +(SELECT name, n as s1_n FROM t1) as s1 +NATURAL FULL JOIN + (SELECT * FROM + (SELECT name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2 + NATURAL FULL JOIN + (SELECT name, udf(n) as s3_n FROM t3) as s3 + ) ss2; + +-- Constants as join keys can also be problematic +SELECT s1.name, udf(s1_n), s2.name, udf(udf(s2_n)) FROM + (SELECT name, udf(n) as s1_n FROM t1) as s1 +FULL JOIN + (SELECT name, 2 as s2_n FROM t2) as s2 +ON (udf(udf(s1_n)) = udf(s2_n)); + + +-- Test for propagation of nullability constraints into sub-joins + +create or replace temporary view x as select * from + (values (1,11), (2,22), (3,null), (4,44), (5,null)) + as v(x1, x2); + +create or replace temporary view y as select * from + (values (1,111), (2,222), (3,333), (4,null)) + as v(y1, y2); + +select udf(udf(x1)), udf(x2) from x; +select udf(y1), udf(udf(y2)) from y; + +select * from x left join y on (udf(x1) = udf(udf(y1)) and udf(x2) is not null); +select * from x left join y on (udf(udf(x1)) = udf(y1) and udf(y2) is not null); + +select * from (x left join y on (udf(x1) = udf(udf(y1)))) left join x xx(xx1,xx2) +on (udf(udf(x1)) = udf(xx1)); +select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) +on (udf(x1) = xx1 and udf(x2) is not null); +select * from (x left join y on (x1 = udf(y1))) left join x xx(xx1,xx2) +on (udf(x1) = udf(udf(xx1)) and udf(y2) is not null); +select * from (x left join y on (udf(x1) = y1)) left join x xx(xx1,xx2) +on (udf(udf(x1)) = udf(xx1) and udf(udf(xx2)) is not null); +-- these should NOT give the same answers as above +select * from (x left join y on (udf(udf(x1)) = udf(udf(y1)))) left join x xx(xx1,xx2) +on (udf(x1) = udf(xx1)) where (udf(x2) is not null); +select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) +on (udf(x1) = xx1) where (udf(y2) is not null); +select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) +on (x1 = udf(xx1)) where (xx2 is not null); + +-- +-- regression test: check for bug with propagation of implied equality +-- to outside an IN +-- +select udf(udf(count(*))) from tenk1 a where udf(udf(unique1)) in + (select udf(unique1) from tenk1 b join tenk1 c using (unique1) + where udf(udf(b.unique2)) = udf(42)); + +-- +-- regression test: check for failure to generate a plan with multiple +-- degenerate IN clauses +-- +select udf(count(*)) from tenk1 x where + udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and + udf(x.unique1) = 0 and + udf(x.unique1) in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=udf(udf(bb.f1))); + +-- try that with GEQO too +-- begin; +-- set geqo = on; +-- set geqo_threshold = 2; +select udf(udf(count(*))) from tenk1 x where + udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and + udf(x.unique1) = 0 and + udf(udf(x.unique1)) in (select udf(aa.f1) from int4_tbl aa,float8_tbl bb where udf(aa.f1)=udf(udf(bb.f1))); +-- rollback; + +-- Skip this test because table b inherits from table a and we do not support this feature, see inherits.sql +-- +-- regression test: be sure we cope with proven-dummy append rels +-- +-- explain (costs off) +-- select aa, bb, unique1, unique1 +-- from tenk1 right join b on aa = unique1 +-- where bb < bb and bb is null; + +-- select aa, bb, unique1, unique1 +-- from tenk1 right join b on aa = unique1 +-- where bb < bb and bb is null; + +-- +-- regression test: check handling of empty-FROM subquery underneath outer join +-- +-- explain (costs off) +-- select * from int8_tbl i1 left join (int8_tbl i2 join +-- (select 123 as x) ss on i2.q1 = x) on i1.q2 = i2.q2 +-- order by 1, 2; + +select * from int8_tbl i1 left join (int8_tbl i2 join + (select udf(123) as x) ss on udf(udf(i2.q1)) = udf(x)) on udf(udf(i1.q2)) = udf(udf(i2.q2)) +order by udf(udf(1)), 2; + +-- +-- regression test: check a case where join_clause_is_movable_into() gives +-- an imprecise result, causing an assertion failure +-- +select udf(count(*)) +from + (select udf(t3.tenthous) as x1, udf(coalesce(udf(t1.stringu1), udf(t2.stringu1))) as x2 + from tenk1 t1 + left join tenk1 t2 on udf(t1.unique1) = udf(t2.unique1) + join tenk1 t3 on t1.unique2 = udf(t3.unique2)) ss, + tenk1 t4, + tenk1 t5 +where udf(t4.thousand) = udf(t5.unique1) and udf(udf(ss.x1)) = t4.tenthous and udf(ss.x2) = udf(udf(t5.stringu1)); + +-- +-- regression test: check a case where we formerly missed including an EC +-- enforcement clause because it was expected to be handled at scan level +-- +-- explain (costs off) +-- select a.f1, b.f1, t.thousand, t.tenthous from +-- tenk1 t, +-- (select sum(f1)+1 as f1 from int4_tbl i4a) a, +-- (select sum(f1) as f1 from int4_tbl i4b) b +-- where b.f1 = t.thousand and a.f1 = b.f1 and (a.f1+b.f1+999) = t.tenthous; + +select udf(a.f1), udf(b.f1), udf(t.thousand), udf(t.tenthous) from + tenk1 t, + (select udf(udf(sum(udf(f1))+1)) as f1 from int4_tbl i4a) a, + (select udf(sum(udf(f1))) as f1 from int4_tbl i4b) b +where b.f1 = udf(t.thousand) and udf(a.f1) = udf(b.f1) and udf((udf(a.f1)+udf(b.f1)+999)) = udf(udf(t.tenthous)); + +-- +-- check a case where we formerly got confused by conflicting sort orders +-- in redundant merge join path keys +-- +-- explain (costs off) +-- select * from +-- j1_tbl full join +-- (select * from j2_tbl order by j2_tbl.i desc, j2_tbl.k asc) j2_tbl +-- on j1_tbl.i = j2_tbl.i and j1_tbl.i = j2_tbl.k; + +select * from + j1_tbl full join + (select * from j2_tbl order by udf(udf(j2_tbl.i)) desc, udf(j2_tbl.k) asc) j2_tbl + on udf(j1_tbl.i) = udf(j2_tbl.i) and udf(j1_tbl.i) = udf(j2_tbl.k); + +-- +-- a different check for handling of redundant sort keys in merge joins +-- +-- explain (costs off) +-- select count(*) from +-- (select * from tenk1 x order by x.thousand, x.twothousand, x.fivethous) x +-- left join +-- (select * from tenk1 y order by y.unique2) y +-- on x.thousand = y.unique2 and x.twothousand = y.hundred and x.fivethous = y.unique2; + +select udf(count(*)) from + (select * from tenk1 x order by udf(x.thousand), udf(udf(x.twothousand)), x.fivethous) x + left join + (select * from tenk1 y order by udf(y.unique2)) y + on udf(x.thousand) = y.unique2 and x.twothousand = udf(y.hundred) and x.fivethous = y.unique2; + + +-- +-- Clean up +-- + +DROP TABLE t1; +DROP TABLE t2; +DROP TABLE t3; + +DROP TABLE J1_TBL; +DROP TABLE J2_TBL; + +-- Both DELETE and UPDATE allow the specification of additional tables +-- to "join" against to determine which rows should be modified. + +-- CREATE TEMP TABLE t1 (a int, b int); +-- CREATE TEMP TABLE t2 (a int, b int); +-- CREATE TEMP TABLE t3 (x int, y int); + +-- INSERT INTO t1 VALUES (5, 10); +-- INSERT INTO t1 VALUES (15, 20); +-- INSERT INTO t1 VALUES (100, 100); +-- INSERT INTO t1 VALUES (200, 1000); +-- INSERT INTO t2 VALUES (200, 2000); +-- INSERT INTO t3 VALUES (5, 20); +-- INSERT INTO t3 VALUES (6, 7); +-- INSERT INTO t3 VALUES (7, 8); +-- INSERT INTO t3 VALUES (500, 100); + +-- DELETE FROM t3 USING t1 table1 WHERE t3.x = table1.a; +-- SELECT * FROM t3; +-- DELETE FROM t3 USING t1 JOIN t2 USING (a) WHERE t3.x > t1.a; +-- SELECT * FROM t3; +-- DELETE FROM t3 USING t3 t3_other WHERE t3.x = t3_other.x AND t3.y = t3_other.y; +-- SELECT * FROM t3; + +-- Test join against inheritance tree + +-- create temp table t2a () inherits (t2); + +-- insert into t2a values (200, 2001); + +-- select * from t1 left join t2 on (t1.a = t2.a); + +-- Test matching of column name with wrong alias + +-- select t1.x from t1 join t3 on (t1.a = t3.x); + +-- +-- regression test for 8.1 merge right join bug +-- + +create or replace temporary view tt1 as select * from + (values (1, 11), (2, NULL)) + as v(tt1_id, joincol); + +create or replace temporary view tt2 as select * from + (values (21, 11), (22, 11)) + as v(tt2_id, joincol); + +-- set enable_hashjoin to off; +-- set enable_nestloop to off; + +-- these should give the same results + +select tt1.*, tt2.* from tt1 left join tt2 on udf(udf(tt1.joincol)) = udf(tt2.joincol); + +select tt1.*, tt2.* from tt2 right join tt1 on udf(udf(tt1.joincol)) = udf(udf(tt2.joincol)); + +-- reset enable_hashjoin; +-- reset enable_nestloop; + +-- +-- regression test for bug #13908 (hash join with skew tuples & nbatch increase) +-- + +-- set work_mem to '64kB'; +-- set enable_mergejoin to off; + +-- explain (costs off) +-- select count(*) from tenk1 a, tenk1 b +-- where a.hundred = b.thousand and (b.fivethous % 10) < 10; +select udf(count(*)) from tenk1 a, tenk1 b + where udf(a.hundred) = b.thousand and udf(udf((b.fivethous % 10)) < 10); + +-- reset work_mem; +-- reset enable_mergejoin; + +-- +-- regression test for 8.2 bug with improper re-ordering of left joins +-- + +DROP TABLE IF EXISTS tt3; +CREATE TABLE tt3(f1 int, f2 string) USING parquet; +INSERT INTO tt3 SELECT x.id, repeat('xyzzy', 100) FROM range(1,10001) x; +-- create index tt3i on tt3(f1); +-- analyze tt3; + +DROP TABLE IF EXISTS tt4; +CREATE TABLE tt4(f1 int) USING parquet; +INSERT INTO tt4 VALUES (0),(1),(9999); +-- analyze tt4; + +SELECT udf(udf(a.f1)) as f1 +FROM tt4 a +LEFT JOIN ( + SELECT b.f1 + FROM tt3 b LEFT JOIN tt3 c ON udf(b.f1) = udf(c.f1) + WHERE udf(c.f1) IS NULL +) AS d ON udf(a.f1) = d.f1 +WHERE udf(udf(d.f1)) IS NULL; + +-- +-- regression test for proper handling of outer joins within antijoins +-- + +-- create temp table tt4x(c1 int, c2 int, c3 int); + +-- explain (costs off) +-- select * from tt4x t1 +-- where not exists ( +-- select 1 from tt4x t2 +-- left join tt4x t3 on t2.c3 = t3.c1 +-- left join ( select t5.c1 as c1 +-- from tt4x t4 left join tt4x t5 on t4.c2 = t5.c1 +-- ) a1 on t3.c2 = a1.c1 +-- where t1.c1 = t2.c2 +-- ); + +-- +-- regression test for problems of the sort depicted in bug #3494 +-- + +create or replace temporary view tt5 as select * from + (values (1, 10), (1, 11)) + as v(f1, f2); +create or replace temporary view tt6 as select * from + (values (1, 9), (1, 2), (2, 9)) + as v(f1, f2); + +select * from tt5,tt6 where udf(tt5.f1) = udf(tt6.f1) and udf(tt5.f1) = udf(udf(tt5.f2) - udf(tt6.f2)); + +-- +-- regression test for problems of the sort depicted in bug #3588 +-- + +create or replace temporary view xx as select * from + (values (1), (2), (3)) + as v(pkxx); +create or replace temporary view yy as select * from + (values (101, 1), (201, 2), (301, NULL)) + as v(pkyy, pkxx); + +select udf(udf(yy.pkyy)) as yy_pkyy, udf(yy.pkxx) as yy_pkxx, udf(yya.pkyy) as yya_pkyy, + udf(xxa.pkxx) as xxa_pkxx, udf(xxb.pkxx) as xxb_pkxx +from yy + left join (SELECT * FROM yy where pkyy = 101) as yya ON udf(yy.pkyy) = udf(yya.pkyy) + left join xx xxa on udf(yya.pkxx) = udf(udf(xxa.pkxx)) + left join xx xxb on udf(udf(coalesce (xxa.pkxx, 1))) = udf(xxb.pkxx); + +-- +-- regression test for improper pushing of constants across outer-join clauses +-- (as seen in early 8.2.x releases) +-- + +create or replace temporary view zt1 as select * from + (values (53)) + as v(f1); +create or replace temporary view zt2 as select * from + (values (53)) + as v(f2); +create or replace temporary view zt3(f3 int) using parquet; + +select * from + zt2 left join zt3 on (udf(f2) = udf(udf(f3))) + left join zt1 on (udf(udf(f3)) = udf(f1)) +where udf(f2) = 53; + +create temp view zv1 as select *,'dummy' AS junk from zt1; + +select * from + zt2 left join zt3 on (f2 = udf(f3)) + left join zv1 on (udf(f3) = f1) +where udf(udf(f2)) = 53; + +-- +-- regression test for improper extraction of OR indexqual conditions +-- (as seen in early 8.3.x releases) +-- + +select udf(a.unique2), udf(a.ten), udf(b.tenthous), udf(b.unique2), udf(b.hundred) +from tenk1 a left join tenk1 b on a.unique2 = udf(b.tenthous) +where udf(a.unique1) = 42 and + ((udf(b.unique2) is null and udf(a.ten) = 2) or udf(udf(b.hundred)) = udf(udf(3))); + +-- +-- test proper positioning of one-time quals in EXISTS (8.4devel bug) +-- +-- prepare foo(bool) as +-- select count(*) from tenk1 a left join tenk1 b +-- on (a.unique2 = b.unique1 and exists +-- (select 1 from tenk1 c where c.thousand = b.unique2 and $1)); +-- execute foo(true); +-- execute foo(false); + +-- +-- test for sane behavior with noncanonical merge clauses, per bug #4926 +-- + +-- begin; + +-- set enable_mergejoin = 1; +-- set enable_hashjoin = 0; +-- set enable_nestloop = 0; + +create or replace temporary view a (i integer) using parquet; +create or replace temporary view b (x integer, y integer) using parquet; + +select * from a left join b on udf(i) = x and i = udf(y) and udf(x) = udf(i); + +-- rollback; + +-- +-- test handling of merge clauses using record_ops +-- +-- begin; + +-- create type mycomptype as (id int, v bigint); + +-- create temp table tidv (idv mycomptype); +-- create index on tidv (idv); + +-- explain (costs off) +-- select a.idv, b.idv from tidv a, tidv b where a.idv = b.idv; + +-- set enable_mergejoin = 0; + +-- explain (costs off) +-- select a.idv, b.idv from tidv a, tidv b where a.idv = b.idv; + +-- rollback; + +-- +-- test NULL behavior of whole-row Vars, per bug #5025 +-- +select udf(t1.q2), udf(count(t2.q1, t2.q2)) +from int8_tbl t1 left join int8_tbl t2 on (udf(udf(t1.q2)) = t2.q1) +group by udf(t1.q2) order by 1; + +select udf(udf(t1.q2)), udf(count(t2.q1, t2.q2)) +from int8_tbl t1 left join (select * from int8_tbl) t2 on (udf(udf(t1.q2)) = udf(t2.q1)) +group by udf(udf(t1.q2)) order by 1; + +-- [SPARK-28330] Enhance query limit +-- select t1.q2, count(t2.q1, t2.q2) +-- from int8_tbl t1 left join (select * from int8_tbl offset 0) t2 on (t1.q2 = t2.q1) +-- group by t1.q2 order by 1; + +select udf(t1.q2) as q2, udf(udf(count(t2.q1, t2.q2))) +from int8_tbl t1 left join + (select udf(q1) as q1, case when q2=1 then 1 else q2 end as q2 from int8_tbl) t2 + on (udf(t1.q2) = udf(t2.q1)) +group by t1.q2 order by 1; + +-- +-- test incorrect failure to NULL pulled-up subexpressions +-- +-- begin; +create or replace temporary view a as select * from + (values ('p'), ('q')) + as v(code); +create or replace temporary view b as select * from + (values ('p', 1), ('p', 2)) + as v(a, num); +create or replace temporary view c as select * from + (values ('A', 'p'), ('B', 'q'), ('C', null)) + as v(name, a); + +select udf(c.name), udf(ss.code), udf(ss.b_cnt), udf(ss.const) +from c left join + (select a.code, coalesce(b_grp.cnt, 0) as b_cnt, -1 as const + from a left join + (select udf(count(1)) as cnt, b.a as a from b group by b.a) as b_grp + on udf(a.code) = udf(udf(b_grp.a)) + ) as ss + on (udf(udf(c.a)) = udf(ss.code)) +order by c.name; + +-- rollback; + +-- +-- test incorrect handling of placeholders that only appear in targetlists, +-- per bug #6154 +-- +SELECT * FROM +( SELECT 1 as key1 ) sub1 +LEFT JOIN +( SELECT sub3.key3, sub4.value2, COALESCE(sub4.value2, 66) as value3 FROM + ( SELECT 1 as key3 ) sub3 + LEFT JOIN + ( SELECT udf(sub5.key5) as key5, udf(udf(COALESCE(sub6.value1, 1))) as value2 FROM + ( SELECT 1 as key5 ) sub5 + LEFT JOIN + ( SELECT 2 as key6, 42 as value1 ) sub6 + ON sub5.key5 = udf(sub6.key6) + ) sub4 + ON udf(sub4.key5) = sub3.key3 +) sub2 +ON udf(udf(sub1.key1)) = udf(udf(sub2.key3)); + +-- test the path using join aliases, too +SELECT * FROM +( SELECT 1 as key1 ) sub1 +LEFT JOIN +( SELECT udf(sub3.key3) as key3, udf(value2), udf(COALESCE(value2, 66)) as value3 FROM + ( SELECT 1 as key3 ) sub3 + LEFT JOIN + ( SELECT sub5.key5, COALESCE(sub6.value1, 1) as value2 FROM + ( SELECT 1 as key5 ) sub5 + LEFT JOIN + ( SELECT 2 as key6, 42 as value1 ) sub6 + ON udf(udf(sub5.key5)) = sub6.key6 + ) sub4 + ON sub4.key5 = sub3.key3 +) sub2 +ON sub1.key1 = udf(udf(sub2.key3)); + +-- +-- test case where a PlaceHolderVar is used as a nestloop parameter +-- + +-- EXPLAIN (COSTS OFF) +-- SELECT qq, unique1 +-- FROM +-- ( SELECT COALESCE(q1, 0) AS qq FROM int8_tbl a ) AS ss1 +-- FULL OUTER JOIN +-- ( SELECT COALESCE(q2, -1) AS qq FROM int8_tbl b ) AS ss2 +-- USING (qq) +-- INNER JOIN tenk1 c ON qq = unique2; + +SELECT udf(qq), udf(udf(unique1)) + FROM + ( SELECT udf(COALESCE(q1, 0)) AS qq FROM int8_tbl a ) AS ss1 + FULL OUTER JOIN + ( SELECT udf(udf(COALESCE(q2, -1))) AS qq FROM int8_tbl b ) AS ss2 + USING (qq) + INNER JOIN tenk1 c ON udf(qq) = udf(unique2); + +-- +-- nested nestloops can require nested PlaceHolderVars +-- + +create or replace temporary view nt1 as select * from + (values(1,true,true), (2,true,false), (3,false,false)) + as v(id, a1, a2); +create or replace temporary view nt2 as select * from + (values(1,1,true,true), (2,2,true,false), (3,3,false,false)) + as v(id, nt1_id, b1, b2); +create or replace temporary view nt3 as select * from + (values(1,1,true), (2,2,false), (3,3,true)) + as v(id, nt2_id, c1); +-- explain (costs off) +-- select nt3.id +-- from nt3 as nt3 +-- left join +-- (select nt2.*, (nt2.b1 and ss1.a3) AS b3 +-- from nt2 as nt2 +-- left join +-- (select nt1.*, (nt1.id is not null) as a3 from nt1) as ss1 +-- on ss1.id = nt2.nt1_id +-- ) as ss2 +-- on ss2.id = nt3.nt2_id +-- where nt3.id = 1 and ss2.b3; + +select udf(nt3.id) +from nt3 as nt3 + left join + (select nt2.*, (udf(nt2.b1) and udf(ss1.a3)) AS b3 + from nt2 as nt2 + left join + (select nt1.*, (udf(nt1.id) is not null) as a3 from nt1) as ss1 + on ss1.id = udf(udf(nt2.nt1_id)) + ) as ss2 + on udf(ss2.id) = nt3.nt2_id +where udf(nt3.id) = 1 and udf(ss2.b3); + +-- Accessing outer query column is not allowed in LocalLimit +-- +-- test case where a PlaceHolderVar is propagated into a subquery +-- + +-- explain (costs off) +-- select * from +-- int8_tbl t1 left join +-- (select q1 as x, 42 as y from int8_tbl t2) ss +-- on t1.q2 = ss.x +-- where +-- 1 = (select 1 from int8_tbl t3 where ss.y is not null limit 1) +-- order by 1,2; + +-- select * from +-- int8_tbl t1 left join +-- (select q1 as x, 42 as y from int8_tbl t2) ss +-- on t1.q2 = ss.x +-- where +-- 1 = (select 1 from int8_tbl t3 where ss.y is not null limit 1) +-- order by 1,2; + +-- +-- test the corner cases FULL JOIN ON TRUE and FULL JOIN ON FALSE +-- +select * from int4_tbl a full join int4_tbl b on true; +select * from int4_tbl a full join int4_tbl b on false; + +-- +-- test for ability to use a cartesian join when necessary +-- + +-- explain (costs off) +-- select * from +-- tenk1 join int4_tbl on f1 = twothousand, +-- int4(sin(1)) q1, +-- int4(sin(0)) q2 +-- where q1 = thousand or q2 = thousand; + +-- explain (costs off) +-- select * from +-- tenk1 join int4_tbl on f1 = twothousand, +-- int4(sin(1)) q1, +-- int4(sin(0)) q2 +-- where thousand = (q1 + q2); + +-- +-- test ability to generate a suitable plan for a star-schema query +-- + +-- explain (costs off) +-- select * from +-- tenk1, int8_tbl a, int8_tbl b +-- where thousand = a.q1 and tenthous = b.q1 and a.q2 = 1 and b.q2 = 2; + +-- +-- test a corner case in which we shouldn't apply the star-schema optimization +-- + +-- explain (costs off) +-- select t1.unique2, t1.stringu1, t2.unique1, t2.stringu2 from +-- tenk1 t1 +-- inner join int4_tbl i1 +-- left join (select v1.x2, v2.y1, 11 AS d1 +-- from (select 1,0 from onerow) v1(x1,x2) +-- left join (select 3,1 from onerow) v2(y1,y2) +-- on v1.x1 = v2.y2) subq1 +-- on (i1.f1 = subq1.x2) +-- on (t1.unique2 = subq1.d1) +-- left join tenk1 t2 +-- on (subq1.y1 = t2.unique1) +-- where t1.unique2 < 42 and t1.stringu1 > t2.stringu2; + +-- [SPARK-20856] support statement using nested joins +-- select t1.unique2, t1.stringu1, t2.unique1, t2.stringu2 from +-- tenk1 t1 +-- inner join int4_tbl i1 +-- left join (select v1.x2, v2.y1, 11 AS d1 +-- from (select 1,0 from onerow) v1(x1,x2) +-- left join (select 3,1 from onerow) v2(y1,y2) +-- on v1.x1 = v2.y2) subq1 +-- on (i1.f1 = subq1.x2) +-- on (t1.unique2 = subq1.d1) +-- left join tenk1 t2 +-- on (subq1.y1 = t2.unique1) +-- where t1.unique2 < 42 and t1.stringu1 > t2.stringu2; + +-- variant that isn't quite a star-schema case + +-- [SPARK-16452] basic INFORMATION_SCHEMA support +-- select ss1.d1 from +-- tenk1 as t1 +-- inner join tenk1 as t2 +-- on t1.tenthous = t2.ten +-- inner join +-- int8_tbl as i8 +-- left join int4_tbl as i4 +-- inner join (select 64::information_schema.cardinal_number as d1 +-- from tenk1 t3, +-- lateral (select abs(t3.unique1) + random()) ss0(x) +-- where t3.fivethous < 0) as ss1 +-- on i4.f1 = ss1.d1 +-- on i8.q1 = i4.f1 +-- on t1.tenthous = ss1.d1 +-- where t1.unique1 < i4.f1; + +-- this variant is foldable by the remove-useless-RESULT-RTEs code + +-- explain (costs off) +-- select t1.unique2, t1.stringu1, t2.unique1, t2.stringu2 from +-- tenk1 t1 +-- inner join int4_tbl i1 +-- left join (select v1.x2, v2.y1, 11 AS d1 +-- from (values(1,0)) v1(x1,x2) +-- left join (values(3,1)) v2(y1,y2) +-- on v1.x1 = v2.y2) subq1 +-- on (i1.f1 = subq1.x2) +-- on (t1.unique2 = subq1.d1) +-- left join tenk1 t2 +-- on (subq1.y1 = t2.unique1) +-- where t1.unique2 < 42 and t1.stringu1 > t2.stringu2; + +-- [SPARK-20856] support statement using nested joins +-- select t1.unique2, t1.stringu1, t2.unique1, t2.stringu2 from +-- tenk1 t1 +-- inner join int4_tbl i1 +-- left join (select v1.x2, v2.y1, 11 AS d1 +-- from (values(1,0)) v1(x1,x2) +-- left join (values(3,1)) v2(y1,y2) +-- on v1.x1 = v2.y2) subq1 +-- on (i1.f1 = subq1.x2) +-- on (t1.unique2 = subq1.d1) +-- left join tenk1 t2 +-- on (subq1.y1 = t2.unique1) +-- where t1.unique2 < 42 and t1.stringu1 > t2.stringu2; + +-- +-- test extraction of restriction OR clauses from join OR clause +-- (we used to only do this for indexable clauses) +-- + +-- explain (costs off) +-- select * from tenk1 a join tenk1 b on +-- (a.unique1 = 1 and b.unique1 = 2) or (a.unique2 = 3 and b.hundred = 4); +-- explain (costs off) +-- select * from tenk1 a join tenk1 b on +-- (a.unique1 = 1 and b.unique1 = 2) or (a.unique2 = 3 and b.ten = 4); +-- explain (costs off) +-- select * from tenk1 a join tenk1 b on +-- (a.unique1 = 1 and b.unique1 = 2) or +-- ((a.unique2 = 3 or a.unique2 = 7) and b.hundred = 4); + +-- +-- test placement of movable quals in a parameterized join tree +-- + +-- explain (costs off) +-- select * from tenk1 t1 left join +-- (tenk1 t2 join tenk1 t3 on t2.thousand = t3.unique2) +-- on t1.hundred = t2.hundred and t1.ten = t3.ten +-- where t1.unique1 = 1; + +-- explain (costs off) +-- select * from tenk1 t1 left join +-- (tenk1 t2 join tenk1 t3 on t2.thousand = t3.unique2) +-- on t1.hundred = t2.hundred and t1.ten + t2.ten = t3.ten +-- where t1.unique1 = 1; + +-- explain (costs off) +-- select count(*) from +-- tenk1 a join tenk1 b on a.unique1 = b.unique2 +-- left join tenk1 c on a.unique2 = b.unique1 and c.thousand = a.thousand +-- join int4_tbl on b.thousand = f1; + +select udf(count(*)) from + tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2) + left join tenk1 c on udf(a.unique2) = udf(b.unique1) and udf(c.thousand) = udf(udf(a.thousand)) + join int4_tbl on udf(b.thousand) = f1; + +-- explain (costs off) +-- select b.unique1 from +-- tenk1 a join tenk1 b on a.unique1 = b.unique2 +-- left join tenk1 c on b.unique1 = 42 and c.thousand = a.thousand +-- join int4_tbl i1 on b.thousand = f1 +-- right join int4_tbl i2 on i2.f1 = b.tenthous +-- order by 1; + +select udf(b.unique1) from + tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2) + left join tenk1 c on udf(b.unique1) = 42 and c.thousand = udf(a.thousand) + join int4_tbl i1 on udf(b.thousand) = udf(udf(f1)) + right join int4_tbl i2 on udf(udf(i2.f1)) = udf(b.tenthous) + order by udf(1); + +-- explain (costs off) +-- select * from +-- ( +-- select unique1, q1, coalesce(unique1, -1) + q1 as fault +-- from int8_tbl left join tenk1 on (q2 = unique2) +-- ) ss +-- where fault = 122 +-- order by fault; + +select * from +( + select udf(unique1), udf(q1), udf(udf(coalesce(unique1, -1)) + udf(q1)) as fault + from int8_tbl left join tenk1 on (udf(q2) = udf(unique2)) +) ss +where udf(fault) = udf(122) +order by udf(fault); + +-- explain (costs off) +-- select * from +-- (values (1, array[10,20]), (2, array[20,30])) as v1(v1x,v1ys) +-- left join (values (1, 10), (2, 20)) as v2(v2x,v2y) on v2x = v1x +-- left join unnest(v1ys) as u1(u1y) on u1y = v2y; + +-- [SPARK-28382] Array Functions: unnest +-- select * from +-- (values (1, array(10,20)), (2, array(20,30))) as v1(v1x,v1ys) +-- left join (values (1, 10), (2, 20)) as v2(v2x,v2y) on v2x = v1x +-- left join unnest(v1ys) as u1(u1y) on u1y = v2y; + +-- +-- test handling of potential equivalence clauses above outer joins +-- + +-- explain (costs off) +-- select q1, unique2, thousand, hundred +-- from int8_tbl a left join tenk1 b on q1 = unique2 +-- where coalesce(thousand,123) = q1 and q1 = coalesce(hundred,123); + +select udf(q1), udf(unique2), udf(thousand), udf(hundred) + from int8_tbl a left join tenk1 b on udf(q1) = udf(unique2) + where udf(coalesce(thousand,123)) = udf(q1) and udf(q1) = udf(udf(coalesce(hundred,123))); + +-- explain (costs off) +-- select f1, unique2, case when unique2 is null then f1 else 0 end +-- from int4_tbl a left join tenk1 b on f1 = unique2 +-- where (case when unique2 is null then f1 else 0 end) = 0; + +select udf(f1), udf(unique2), case when udf(udf(unique2)) is null then udf(f1) else 0 end + from int4_tbl a left join tenk1 b on udf(f1) = udf(udf(unique2)) + where (case when udf(unique2) is null then udf(f1) else 0 end) = 0; + +-- +-- another case with equivalence clauses above outer joins (bug #8591) +-- + +-- explain (costs off) +-- select a.unique1, b.unique1, c.unique1, coalesce(b.twothousand, a.twothousand) +-- from tenk1 a left join tenk1 b on b.thousand = a.unique1 left join tenk1 c on c.unique2 = coalesce(b.twothousand, a.twothousand) +-- where a.unique2 < 10 and coalesce(b.twothousand, a.twothousand) = 44; + +select udf(a.unique1), udf(b.unique1), udf(c.unique1), udf(coalesce(b.twothousand, a.twothousand)) + from tenk1 a left join tenk1 b on udf(b.thousand) = a.unique1 left join tenk1 c on udf(c.unique2) = udf(coalesce(b.twothousand, a.twothousand)) + where a.unique2 < udf(10) and udf(udf(coalesce(b.twothousand, a.twothousand))) = udf(44); + +-- +-- check handling of join aliases when flattening multiple levels of subquery +-- + +-- explain (verbose, costs off) +-- select foo1.join_key as foo1_id, foo3.join_key AS foo3_id, bug_field from +-- (values (0),(1)) foo1(join_key) +-- left join +-- (select join_key, bug_field from +-- (select ss1.join_key, ss1.bug_field from +-- (select f1 as join_key, 666 as bug_field from int4_tbl i1) ss1 +-- ) foo2 +-- left join +-- (select unique2 as join_key from tenk1 i2) ss2 +-- using (join_key) +-- ) foo3 +-- using (join_key); + + +-- [SPARK-28377] Fully support correlation names in the FROM clause +-- select foo1.join_key as foo1_id, foo3.join_key AS foo3_id, bug_field from +-- (values (0),(1)) foo1(join_key) +-- left join +-- (select join_key, bug_field from +-- (select ss1.join_key, ss1.bug_field from +-- (select f1 as join_key, 666 as bug_field from int4_tbl i1) ss1 +-- ) foo2 +-- left join +-- (select unique2 as join_key from tenk1 i2) ss2 +-- using (join_key) +-- ) foo3 +-- using (join_key); + +-- [SPARK-20856] Support statement using nested joins +-- +-- test successful handling of nested outer joins with degenerate join quals +-- + +-- explain (verbose, costs off) +-- select t1.* from +-- text_tbl t1 +-- left join (select *, '***'::text as d1 from int8_tbl i8b1) b1 +-- left join int8_tbl i8 +-- left join (select *, null::int as d2 from int8_tbl i8b2) b2 +-- on (i8.q1 = b2.q1) +-- on (b2.d2 = b1.q2) +-- on (t1.f1 = b1.d1) +-- left join int4_tbl i4 +-- on (i8.q2 = i4.f1); + +-- select t1.* from +-- text_tbl t1 +-- left join (select *, string('***') as d1 from int8_tbl i8b1) b1 +-- left join int8_tbl i8 +-- left join (select *, int(null) as d2 from int8_tbl i8b2) b2 +-- on (i8.q1 = b2.q1) +-- on (b2.d2 = b1.q2) +-- on (t1.f1 = b1.d1) +-- left join int4_tbl i4 +-- on (i8.q2 = i4.f1); + +-- explain (verbose, costs off) +-- select t1.* from +-- text_tbl t1 +-- left join (select *, '***'::text as d1 from int8_tbl i8b1) b1 +-- left join int8_tbl i8 +-- left join (select *, null::int as d2 from int8_tbl i8b2, int4_tbl i4b2) b2 +-- on (i8.q1 = b2.q1) +-- on (b2.d2 = b1.q2) +-- on (t1.f1 = b1.d1) +-- left join int4_tbl i4 +-- on (i8.q2 = i4.f1); + +-- select t1.* from +-- text_tbl t1 +-- left join (select *, string('***') as d1 from int8_tbl i8b1) b1 +-- left join int8_tbl i8 +-- left join (select *, int(null) as d2 from int8_tbl i8b2, int4_tbl i4b2) b2 +-- on (i8.q1 = b2.q1) +-- on (b2.d2 = b1.q2) +-- on (t1.f1 = b1.d1) +-- left join int4_tbl i4 +-- on (i8.q2 = i4.f1); + +-- explain (verbose, costs off) +-- select t1.* from +-- text_tbl t1 +-- left join (select *, '***'::text as d1 from int8_tbl i8b1) b1 +-- left join int8_tbl i8 +-- left join (select *, null::int as d2 from int8_tbl i8b2, int4_tbl i4b2 +-- where q1 = f1) b2 +-- on (i8.q1 = b2.q1) +-- on (b2.d2 = b1.q2) +-- on (t1.f1 = b1.d1) +-- left join int4_tbl i4 +-- on (i8.q2 = i4.f1); + +-- select t1.* from +-- text_tbl t1 +-- left join (select *, string('***') as d1 from int8_tbl i8b1) b1 +-- left join int8_tbl i8 +-- left join (select *, int(null) as d2 from int8_tbl i8b2, int4_tbl i4b2 +-- where q1 = f1) b2 +-- on (i8.q1 = b2.q1) +-- on (b2.d2 = b1.q2) +-- on (t1.f1 = b1.d1) +-- left join int4_tbl i4 +-- on (i8.q2 = i4.f1); + +-- explain (verbose, costs off) +-- select * from +-- text_tbl t1 +-- inner join int8_tbl i8 +-- on i8.q2 = 456 +-- right join text_tbl t2 +-- on t1.f1 = 'doh!' +-- left join int4_tbl i4 +-- on i8.q1 = i4.f1; + +select * from + text_tbl t1 + inner join int8_tbl i8 + on udf(i8.q2) = udf(udf(456)) + right join text_tbl t2 + on udf(t1.f1) = udf(udf('doh!')) + left join int4_tbl i4 + on udf(udf(i8.q1)) = i4.f1; + +-- [SPARK-27877] ANSI SQL: LATERAL derived table(T491) +-- +-- test for appropriate join order in the presence of lateral references +-- + +-- explain (verbose, costs off) +-- select * from +-- text_tbl t1 +-- left join int8_tbl i8 +-- on i8.q2 = 123, +-- lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss +-- where t1.f1 = ss.f1; + +-- select * from +-- text_tbl t1 +-- left join int8_tbl i8 +-- on i8.q2 = 123, +-- lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss +-- where t1.f1 = ss.f1; + +-- explain (verbose, costs off) +-- select * from +-- text_tbl t1 +-- left join int8_tbl i8 +-- on i8.q2 = 123, +-- lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss1, +-- lateral (select ss1.* from text_tbl t3 limit 1) as ss2 +-- where t1.f1 = ss2.f1; + +-- select * from +-- text_tbl t1 +-- left join int8_tbl i8 +-- on i8.q2 = 123, +-- lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss1, +-- lateral (select ss1.* from text_tbl t3 limit 1) as ss2 +-- where t1.f1 = ss2.f1; + +-- explain (verbose, costs off) +-- select 1 from +-- text_tbl as tt1 +-- inner join text_tbl as tt2 on (tt1.f1 = 'foo') +-- left join text_tbl as tt3 on (tt3.f1 = 'foo') +-- left join text_tbl as tt4 on (tt3.f1 = tt4.f1), +-- lateral (select tt4.f1 as c0 from text_tbl as tt5 limit 1) as ss1 +-- where tt1.f1 = ss1.c0; + +-- select 1 from +-- text_tbl as tt1 +-- inner join text_tbl as tt2 on (tt1.f1 = 'foo') +-- left join text_tbl as tt3 on (tt3.f1 = 'foo') +-- left join text_tbl as tt4 on (tt3.f1 = tt4.f1), +-- lateral (select tt4.f1 as c0 from text_tbl as tt5 limit 1) as ss1 +-- where tt1.f1 = ss1.c0; + +-- +-- check a case in which a PlaceHolderVar forces join order +-- + +-- explain (verbose, costs off) +-- select ss2.* from +-- int4_tbl i41 +-- left join int8_tbl i8 +-- join (select i42.f1 as c1, i43.f1 as c2, 42 as c3 +-- from int4_tbl i42, int4_tbl i43) ss1 +-- on i8.q1 = ss1.c2 +-- on i41.f1 = ss1.c1, +-- lateral (select i41.*, i8.*, ss1.* from text_tbl limit 1) ss2 +-- where ss1.c2 = 0; + +-- select ss2.* from +-- int4_tbl i41 +-- left join int8_tbl i8 +-- join (select i42.f1 as c1, i43.f1 as c2, 42 as c3 +-- from int4_tbl i42, int4_tbl i43) ss1 +-- on i8.q1 = ss1.c2 +-- on i41.f1 = ss1.c1, +-- lateral (select i41.*, i8.*, ss1.* from text_tbl limit 1) ss2 +-- where ss1.c2 = 0; + +-- +-- test successful handling of full join underneath left join (bug #14105) +-- + +-- explain (costs off) +-- select * from +-- (select 1 as id) as xx +-- left join +-- (tenk1 as a1 full join (select 1 as id) as yy on (a1.unique1 = yy.id)) +-- on (xx.id = coalesce(yy.id)); + +select * from + (select udf(udf(1)) as id) as xx + left join + (tenk1 as a1 full join (select udf(1) as id) as yy on (udf(a1.unique1) = udf(yy.id))) + on (xx.id = udf(udf(coalesce(yy.id)))); + +-- +-- test ability to push constants through outer join clauses +-- + +-- explain (costs off) +-- select * from int4_tbl a left join tenk1 b on f1 = unique2 where f1 = 0; + +-- explain (costs off) +-- select * from tenk1 a full join tenk1 b using(unique2) where unique2 = 42; + +-- +-- test that quals attached to an outer join have correct semantics, +-- specifically that they don't re-use expressions computed below the join; +-- we force a mergejoin so that coalesce(b.q1, 1) appears as a join input +-- + +-- set enable_hashjoin to off; +-- set enable_nestloop to off; + +-- explain (verbose, costs off) +-- select a.q2, b.q1 +-- from int8_tbl a left join int8_tbl b on a.q2 = coalesce(b.q1, 1) +-- where coalesce(b.q1, 1) > 0; +select udf(a.q2), udf(b.q1) + from int8_tbl a left join int8_tbl b on udf(a.q2) = coalesce(b.q1, 1) + where udf(udf(coalesce(b.q1, 1)) > 0); + +-- reset enable_hashjoin; +-- reset enable_nestloop; + +-- Skip these test because it only test explain +-- +-- test join removal +-- + +-- begin; + +-- CREATE TEMP TABLE a (id int PRIMARY KEY, b_id int); +-- CREATE TEMP TABLE b (id int PRIMARY KEY, c_id int); +-- CREATE TEMP TABLE c (id int PRIMARY KEY); +-- CREATE TEMP TABLE d (a int, b int); +-- INSERT INTO a VALUES (0, 0), (1, NULL); +-- INSERT INTO b VALUES (0, 0), (1, NULL); +-- INSERT INTO c VALUES (0), (1); +-- INSERT INTO d VALUES (1,3), (2,2), (3,1); + +-- all three cases should be optimizable into a simple seqscan +-- explain (costs off) SELECT a.* FROM a LEFT JOIN b ON a.b_id = b.id; +-- explain (costs off) SELECT b.* FROM b LEFT JOIN c ON b.c_id = c.id; +-- explain (costs off) +-- SELECT a.* FROM a LEFT JOIN (b left join c on b.c_id = c.id) +-- ON (a.b_id = b.id); + +-- check optimization of outer join within another special join +-- explain (costs off) +-- select id from a where id in ( +-- select b.id from b left join c on b.id = c.id +-- ); + +-- check that join removal works for a left join when joining a subquery +-- that is guaranteed to be unique by its GROUP BY clause +-- explain (costs off) +-- select d.* from d left join (select * from b group by b.id, b.c_id) s +-- on d.a = s.id and d.b = s.c_id; + +-- similarly, but keying off a DISTINCT clause +-- explain (costs off) +-- select d.* from d left join (select distinct * from b) s +-- on d.a = s.id and d.b = s.c_id; + +-- join removal is not possible when the GROUP BY contains a column that is +-- not in the join condition. (Note: as of 9.6, we notice that b.id is a +-- primary key and so drop b.c_id from the GROUP BY of the resulting plan; +-- but this happens too late for join removal in the outer plan level.) +-- explain (costs off) +-- select d.* from d left join (select * from b group by b.id, b.c_id) s +-- on d.a = s.id; + +-- similarly, but keying off a DISTINCT clause +-- explain (costs off) +-- select d.* from d left join (select distinct * from b) s +-- on d.a = s.id; + +-- check join removal works when uniqueness of the join condition is enforced +-- by a UNION +-- explain (costs off) +-- select d.* from d left join (select id from a union select id from b) s +-- on d.a = s.id; + +-- check join removal with a cross-type comparison operator +-- explain (costs off) +-- select i8.* from int8_tbl i8 left join (select f1 from int4_tbl group by f1) i4 +-- on i8.q1 = i4.f1; + +-- check join removal with lateral references +-- explain (costs off) +-- select 1 from (select a.id FROM a left join b on a.b_id = b.id) q, +-- lateral generate_series(1, q.id) gs(i) where q.id = gs.i; + +-- rollback; + +create or replace temporary view parent as select * from + (values (1, 10), (2, 20), (3, 30)) + as v(k, pd); +create or replace temporary view child as select * from + (values (1, 100), (4, 400)) + as v(k, cd); + +-- this case is optimizable +select p.* from parent p left join child c on (udf(p.k) = udf(c.k)); +-- explain (costs off) +-- select p.* from parent p left join child c on (p.k = c.k); + +-- this case is not +select p.*, linked from parent p + left join (select c.*, udf(udf(true)) as linked from child c) as ss + on (udf(p.k) = udf(udf(ss.k))); +-- explain (costs off) +-- select p.*, linked from parent p +-- left join (select c.*, true as linked from child c) as ss +-- on (p.k = ss.k); + +-- check for a 9.0rc1 bug: join removal breaks pseudoconstant qual handling +select p.* from + parent p left join child c on (udf(p.k) = c.k) + where p.k = udf(1) and udf(udf(p.k)) = udf(udf(2)); +-- explain (costs off) +-- select p.* from +-- parent p left join child c on (p.k = c.k) +-- where p.k = 1 and p.k = 2; + +select p.* from + (parent p left join child c on (udf(p.k) = c.k)) join parent x on p.k = udf(x.k) + where udf(p.k) = udf(1) and udf(udf(p.k)) = udf(udf(2)); +-- explain (costs off) +-- select p.* from +-- (parent p left join child c on (p.k = c.k)) join parent x on p.k = x.k +-- where p.k = 1 and p.k = 2; + +-- bug 5255: this is not optimizable by join removal +-- begin; + +create or replace temporary view a as select * from + (values (0), (1)) + as v(id); +create or replace temporary view b as select * from + (values (0, 0), (1, NULL)) + as v(id, a_id); + +SELECT * FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(udf(a.id)) IS NULL OR udf(a.id) > 0); +SELECT b.* FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(a.id) IS NULL OR udf(udf(a.id)) > 0); + +-- rollback; + +-- another join removal bug: this is not optimizable, either +-- begin; + +create or replace temporary view innertab as select * from + (values (123L, 42L)) + as v(id, dat1); + +SELECT * FROM + (SELECT udf(1) AS x) ss1 + LEFT JOIN + (SELECT udf(q1), udf(q2), udf(COALESCE(dat1, q1)) AS y + FROM int8_tbl LEFT JOIN innertab ON udf(udf(q2)) = id) ss2 + ON true; + +-- rollback; + +-- another join removal bug: we must clean up correctly when removing a PHV +-- begin; + +-- create temp table uniquetbl (f1 text unique); + +-- explain (costs off) +-- select t1.* from +-- uniquetbl as t1 +-- left join (select *, '***'::text as d1 from uniquetbl) t2 +-- on t1.f1 = t2.f1 +-- left join uniquetbl t3 +-- on t2.d1 = t3.f1; + +-- explain (costs off) +-- select t0.* +-- from +-- text_tbl t0 +-- left join +-- (select case t1.ten when 0 then 'doh!'::text else null::text end as case1, +-- t1.stringu2 +-- from tenk1 t1 +-- join int4_tbl i4 ON i4.f1 = t1.unique2 +-- left join uniquetbl u1 ON u1.f1 = t1.string4) ss +-- on t0.f1 = ss.case1 +-- where ss.stringu2 !~* ss.case1; + +-- [SPARK-27987] Support POSIX Regular Expressions +-- select t0.* +-- from +-- text_tbl t0 +-- left join +-- (select case t1.ten when 0 then 'doh!'::text else null::text end as case1, +-- t1.stringu2 +-- from tenk1 t1 +-- join int4_tbl i4 ON i4.f1 = t1.unique2 +-- left join uniquetbl u1 ON u1.f1 = t1.string4) ss +-- on t0.f1 = ss.case1 +-- where ss.stringu2 !~* ss.case1; + +-- rollback; + +-- bug #8444: we've historically allowed duplicate aliases within aliased JOINs + +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(f1); -- error +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(y.f1); -- error +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on udf(q1) = udf(udf(f1)); -- ok + +-- +-- Test hints given on incorrect column references are useful +-- + +select udf(t1.uunique1) from + tenk1 t1 join tenk2 t2 on t1.two = udf(t2.two); -- error, prefer "t1" suggestion +select udf(udf(t2.uunique1)) from + tenk1 t1 join tenk2 t2 on udf(t1.two) = t2.two; -- error, prefer "t2" suggestion +select udf(uunique1) from + tenk1 t1 join tenk2 t2 on udf(t1.two) = udf(t2.two); -- error, suggest both at once + +-- Skip this test because it is a PostgreSQL specific case +-- +-- Take care to reference the correct RTE +-- + +-- -- select atts.relid::regclass, s.* from pg_stats s join +-- -- pg_attribute a on s.attname = a.attname and s.tablename = +-- -- a.attrelid::regclass::text join (select unnest(indkey) attnum, +-- -- indexrelid from pg_index i) atts on atts.attnum = a.attnum where +-- schemaname != 'pg_catalog'; + +-- [SPARK-27877] ANSI SQL: LATERAL derived table(T491) +-- +-- Test LATERAL +-- + +select unique2, x.* +from tenk1 a, lateral (select * from int4_tbl b where f1 = a.unique1) x; +-- explain (costs off) +-- select unique2, x.* +-- from tenk1 a, lateral (select * from int4_tbl b where f1 = a.unique1) x; +select unique2, x.* +from int4_tbl x, lateral (select unique2 from tenk1 where f1 = unique1) ss; +-- explain (costs off) +-- select unique2, x.* +-- from int4_tbl x, lateral (select unique2 from tenk1 where f1 = unique1) ss; +-- explain (costs off) +-- select unique2, x.* +-- from int4_tbl x cross join lateral (select unique2 from tenk1 where f1 = unique1) ss; +select unique2, x.* +from int4_tbl x left join lateral (select unique1, unique2 from tenk1 where f1 = unique1) ss on true; +-- explain (costs off) +-- select unique2, x.* +-- from int4_tbl x left join lateral (select unique1, unique2 from tenk1 where f1 = unique1) ss on true; + +-- [SPARK-27877] ANSI SQL: LATERAL derived table(T491) +-- check scoping of lateral versus parent references +-- the first of these should return int8_tbl.q2, the second int8_tbl.q1 +-- select *, (select r from (select q1 as q2) x, (select q2 as r) y) from int8_tbl; +-- select *, (select r from (select q1 as q2) x, lateral (select q2 as r) y) from int8_tbl; + +-- lateral with function in FROM +-- select count(*) from tenk1 a, lateral generate_series(1,two) g; +-- explain (costs off) +-- select count(*) from tenk1 a, lateral generate_series(1,two) g; +-- explain (costs off) +-- select count(*) from tenk1 a cross join lateral generate_series(1,two) g; +-- don't need the explicit LATERAL keyword for functions +-- explain (costs off) +-- select count(*) from tenk1 a, generate_series(1,two) g; + +-- lateral with UNION ALL subselect +-- explain (costs off) +-- select * from generate_series(100,200) g, +-- lateral (select * from int8_tbl a where g = q1 union all +-- select * from int8_tbl b where g = q2) ss; +-- select * from generate_series(100,200) g, +-- lateral (select * from int8_tbl a where g = q1 union all +-- select * from int8_tbl b where g = q2) ss; + +-- lateral with VALUES +-- explain (costs off) +-- select count(*) from tenk1 a, +-- tenk1 b join lateral (values(a.unique1)) ss(x) on b.unique2 = ss.x; +-- select count(*) from tenk1 a, +-- tenk1 b join lateral (values(a.unique1)) ss(x) on b.unique2 = ss.x; + +-- lateral with VALUES, no flattening possible +-- explain (costs off) +-- select count(*) from tenk1 a, +-- tenk1 b join lateral (values(a.unique1),(-1)) ss(x) on b.unique2 = ss.x; +-- select count(*) from tenk1 a, +-- tenk1 b join lateral (values(a.unique1),(-1)) ss(x) on b.unique2 = ss.x; + +-- lateral injecting a strange outer join condition +-- explain (costs off) +-- select * from int8_tbl a, +-- int8_tbl x left join lateral (select a.q1 from int4_tbl y) ss(z) +-- on x.q2 = ss.z +-- order by a.q1, a.q2, x.q1, x.q2, ss.z; +-- select * from int8_tbl a, +-- int8_tbl x left join lateral (select a.q1 from int4_tbl y) ss(z) +-- on x.q2 = ss.z +-- order by a.q1, a.q2, x.q1, x.q2, ss.z; + +-- lateral reference to a join alias variable +select * from (select f1/2 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1, + lateral (select x) ss2(y); +-- select * from (select f1 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1, +-- lateral (values(x)) ss2(y); +select * from ((select f1/2 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1) j, + lateral (select x) ss2(y); + +-- lateral references requiring pullup +-- select * from (values(1)) x(lb), +-- lateral generate_series(lb,4) x4; +-- select * from (select f1/1000000000 from int4_tbl) x(lb), +-- lateral generate_series(lb,4) x4; +-- select * from (values(1)) x(lb), +-- lateral (values(lb)) y(lbcopy); +-- select * from (values(1)) x(lb), +-- lateral (select lb from int4_tbl) y(lbcopy); +-- select * from +-- int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1, +-- lateral (values(x.q1,y.q1,y.q2)) v(xq1,yq1,yq2); +select * from + int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1, + lateral (select x.q1,y.q1,y.q2) v(xq1,yq1,yq2); +select x.* from + int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1, + lateral (select x.q1,y.q1,y.q2) v(xq1,yq1,yq2); +-- select v.* from +-- (int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1) +-- left join int4_tbl z on z.f1 = x.q2, +-- lateral (select x.q1,y.q1 union all select x.q2,y.q2) v(vx,vy); +-- select v.* from +-- (int8_tbl x left join (select q1,(select coalesce(q2,0)) q2 from int8_tbl) y on x.q2 = y.q1) +-- left join int4_tbl z on z.f1 = x.q2, +-- lateral (select x.q1,y.q1 union all select x.q2,y.q2) v(vx,vy); +-- select v.* from +-- (int8_tbl x left join (select q1,(select coalesce(q2,0)) q2 from int8_tbl) y on x.q2 = y.q1) +-- left join int4_tbl z on z.f1 = x.q2, +-- lateral (select x.q1,y.q1 from onerow union all select x.q2,y.q2 from onerow) v(vx,vy); + +-- explain (verbose, costs off) +-- select * from +-- int8_tbl a left join +-- lateral (select *, a.q2 as x from int8_tbl b) ss on a.q2 = ss.q1; +select * from + int8_tbl a left join + lateral (select *, a.q2 as x from int8_tbl b) ss on a.q2 = ss.q1; +-- explain (verbose, costs off) +-- select * from +-- int8_tbl a left join +-- lateral (select *, coalesce(a.q2, 42) as x from int8_tbl b) ss on a.q2 = ss.q1; +select * from + int8_tbl a left join + lateral (select *, coalesce(a.q2, 42) as x from int8_tbl b) ss on a.q2 = ss.q1; + +-- lateral can result in join conditions appearing below their +-- real semantic level +-- explain (verbose, costs off) +-- select * from int4_tbl i left join +-- lateral (select * from int2_tbl j where i.f1 = j.f1) k on true; +select * from int4_tbl i left join + lateral (select * from int2_tbl j where i.f1 = j.f1) k on true; +-- explain (verbose, costs off) +-- select * from int4_tbl i left join +-- lateral (select coalesce(i) from int2_tbl j where i.f1 = j.f1) k on true; +-- select * from int4_tbl i left join +-- lateral (select coalesce(i) from int2_tbl j where i.f1 = j.f1) k on true; +-- explain (verbose, costs off) +-- select * from int4_tbl a, +-- lateral ( +-- select * from int4_tbl b left join int8_tbl c on (b.f1 = q1 and a.f1 = q2) +-- ) ss; +-- select * from int4_tbl a, +-- lateral ( +-- select * from int4_tbl b left join int8_tbl c on (b.f1 = q1 and a.f1 = q2) +-- ) ss; + +-- lateral reference in a PlaceHolderVar evaluated at join level +-- explain (verbose, costs off) +-- select * from +-- int8_tbl a left join lateral +-- (select b.q1 as bq1, c.q1 as cq1, least(a.q1,b.q1,c.q1) from +-- int8_tbl b cross join int8_tbl c) ss +-- on a.q2 = ss.bq1; +select * from + int8_tbl a left join lateral + (select b.q1 as bq1, c.q1 as cq1, least(a.q1,b.q1,c.q1) from + int8_tbl b cross join int8_tbl c) ss + on a.q2 = ss.bq1; + +-- case requiring nested PlaceHolderVars +-- explain (verbose, costs off) +-- select * from +-- int8_tbl c left join ( +-- int8_tbl a left join (select q1, coalesce(q2,42) as x from int8_tbl b) ss1 +-- on a.q2 = ss1.q1 +-- cross join +-- lateral (select q1, coalesce(ss1.x,q2) as y from int8_tbl d) ss2 +-- ) on c.q2 = ss2.q1, +-- lateral (select ss2.y offset 0) ss3; + +-- case that breaks the old ph_may_need optimization +-- explain (verbose, costs off) +-- select c.*,a.*,ss1.q1,ss2.q1,ss3.* from +-- int8_tbl c left join ( +-- int8_tbl a left join +-- (select q1, coalesce(q2,f1) as x from int8_tbl b, int4_tbl b2 +-- where q1 < f1) ss1 +-- on a.q2 = ss1.q1 +-- cross join +-- lateral (select q1, coalesce(ss1.x,q2) as y from int8_tbl d) ss2 +-- ) on c.q2 = ss2.q1, +-- lateral (select * from int4_tbl i where ss2.y > f1) ss3; + +-- check processing of postponed quals (bug #9041) +-- explain (verbose, costs off) +-- select * from +-- (select 1 as x offset 0) x cross join (select 2 as y offset 0) y +-- left join lateral ( +-- select * from (select 3 as z offset 0) z where z.z = x.x +-- ) zz on zz.z = y.y; + +-- check dummy rels with lateral references (bug #15694) +-- explain (verbose, costs off) +-- select * from int8_tbl i8 left join lateral +-- (select *, i8.q2 from int4_tbl where false) ss on true; +-- explain (verbose, costs off) +-- select * from int8_tbl i8 left join lateral +-- (select *, i8.q2 from int4_tbl i1, int4_tbl i2 where false) ss on true; + +-- check handling of nested appendrels inside LATERAL +-- select * from +-- ((select 2 as v) union all (select 3 as v)) as q1 +-- cross join lateral +-- ((select * from +-- ((select 4 as v) union all (select 5 as v)) as q3) +-- union all +-- (select q1.v) +-- ) as q2; + +-- check we don't try to do a unique-ified semijoin with LATERAL +-- explain (verbose, costs off) +-- select * from +-- (values (0,9998), (1,1000)) v(id,x), +-- lateral (select f1 from int4_tbl +-- where f1 = any (select unique1 from tenk1 +-- where unique2 = v.x offset 0)) ss; +-- select * from +-- (values (0,9998), (1,1000)) v(id,x), +-- lateral (select f1 from int4_tbl +-- where f1 = any (select unique1 from tenk1 +-- where unique2 = v.x offset 0)) ss; + +-- check proper extParam/allParam handling (this isn't exactly a LATERAL issue, +-- but we can make the test case much more compact with LATERAL) +-- explain (verbose, costs off) +-- select * from (values (0), (1)) v(id), +-- lateral (select * from int8_tbl t1, +-- lateral (select * from +-- (select * from int8_tbl t2 +-- where q1 = any (select q2 from int8_tbl t3 +-- where q2 = (select greatest(t1.q1,t2.q2)) +-- and (select v.id=0)) offset 0) ss2) ss +-- where t1.q1 = ss.q2) ss0; + +-- select * from (values (0), (1)) v(id), +-- lateral (select * from int8_tbl t1, +-- lateral (select * from +-- (select * from int8_tbl t2 +-- where q1 = any (select q2 from int8_tbl t3 +-- where q2 = (select greatest(t1.q1,t2.q2)) +-- and (select v.id=0)) offset 0) ss2) ss +-- where t1.q1 = ss.q2) ss0; + +-- test some error cases where LATERAL should have been used but wasn't +select udf(udf(f1,g)) from int4_tbl a, (select udf(udf(f1)) as g) ss; +select udf(f1,g) from int4_tbl a, (select a.f1 as g) ss; +select udf(udf(f1,g)) from int4_tbl a cross join (select udf(f1) as g) ss; +select udf(f1,g) from int4_tbl a cross join (select udf(udf(a.f1)) as g) ss; +-- SQL:2008 says the left table is in scope but illegal to access here +-- select f1,g from int4_tbl a right join lateral generate_series(0, a.f1) g on true; +-- select f1,g from int4_tbl a full join lateral generate_series(0, a.f1) g on true; +-- check we complain about ambiguous table references +-- select * from +-- int8_tbl x cross join (int4_tbl x cross join lateral (select x.f1) ss); +-- LATERAL can be used to put an aggregate into the FROM clause of its query +-- select 1 from tenk1 a, lateral (select max(a.unique1) from int4_tbl b) ss; + +-- check behavior of LATERAL in UPDATE/DELETE + +-- create temp table xx1 as select f1 as x1, -f1 as x2 from int4_tbl; + +-- error, can't do this: +-- update xx1 set x2 = f1 from (select * from int4_tbl where f1 = x1) ss; +-- update xx1 set x2 = f1 from (select * from int4_tbl where f1 = xx1.x1) ss; +-- can't do it even with LATERAL: +-- update xx1 set x2 = f1 from lateral (select * from int4_tbl where f1 = x1) ss; +-- we might in future allow something like this, but for now it's an error: +-- update xx1 set x2 = f1 from xx1, lateral (select * from int4_tbl where f1 = x1) ss; + +-- also errors: +-- delete from xx1 using (select * from int4_tbl where f1 = x1) ss; +-- delete from xx1 using (select * from int4_tbl where f1 = xx1.x1) ss; +-- delete from xx1 using lateral (select * from int4_tbl where f1 = x1) ss; + +-- [SPARK-25411] Implement range partition in Spark +-- +-- test LATERAL reference propagation down a multi-level inheritance hierarchy +-- produced for a multi-level partitioned table hierarchy. +-- +-- create table join_pt1 (a int, b int, c varchar) partition by range(a); +-- create table join_pt1p1 partition of join_pt1 for values from (0) to (100) partition by range(b); +-- create table join_pt1p2 partition of join_pt1 for values from (100) to (200); +-- create table join_pt1p1p1 partition of join_pt1p1 for values from (0) to (100); +-- insert into join_pt1 values (1, 1, 'x'), (101, 101, 'y'); +-- create table join_ut1 (a int, b int, c varchar); +-- insert into join_ut1 values (101, 101, 'y'), (2, 2, 'z'); +-- explain (verbose, costs off) +-- select t1.b, ss.phv from join_ut1 t1 left join lateral +-- (select t2.a as t2a, t3.a t3a, least(t1.a, t2.a, t3.a) phv +-- from join_pt1 t2 join join_ut1 t3 on t2.a = t3.b) ss +-- on t1.a = ss.t2a order by t1.a; +-- select t1.b, ss.phv from join_ut1 t1 left join lateral +-- (select t2.a as t2a, t3.a t3a, least(t1.a, t2.a, t3.a) phv +-- from join_pt1 t2 join join_ut1 t3 on t2.a = t3.b) ss +-- on t1.a = ss.t2a order by t1.a; +-- +-- drop table join_pt1; +-- drop table join_ut1; +-- +-- test that foreign key join estimation performs sanely for outer joins +-- + +-- begin; + +-- create table fkest (a int, b int, c int unique, primary key(a,b)); +-- create table fkest1 (a int, b int, primary key(a,b)); + +-- insert into fkest select x/10, x%10, x from generate_series(1,1000) x; +-- insert into fkest1 select x/10, x%10 from generate_series(1,1000) x; + +-- alter table fkest1 +-- add constraint fkest1_a_b_fkey foreign key (a,b) references fkest; + +-- analyze fkest; +-- analyze fkest1; + +-- explain (costs off) +-- select * +-- from fkest f +-- left join fkest1 f1 on f.a = f1.a and f.b = f1.b +-- left join fkest1 f2 on f.a = f2.a and f.b = f2.b +-- left join fkest1 f3 on f.a = f3.a and f.b = f3.b +-- where f.c = 1; + +-- rollback; + +-- Skip these test because it only test explain +-- +-- test planner's ability to mark joins as unique +-- + +-- create table j1 (id int primary key); +-- create table j2 (id int primary key); +-- create table j3 (id int); + +-- insert into j1 values(1),(2),(3); +-- insert into j2 values(1),(2),(3); +-- insert into j3 values(1),(1); + +-- analyze j1; +-- analyze j2; +-- analyze j3; + +-- ensure join is properly marked as unique +-- explain (verbose, costs off) +-- select * from j1 inner join j2 on j1.id = j2.id; + +-- ensure join is not unique when not an equi-join +-- explain (verbose, costs off) +-- select * from j1 inner join j2 on j1.id > j2.id; + +-- ensure non-unique rel is not chosen as inner +-- explain (verbose, costs off) +-- select * from j1 inner join j3 on j1.id = j3.id; + +-- ensure left join is marked as unique +-- explain (verbose, costs off) +-- select * from j1 left join j2 on j1.id = j2.id; + +-- ensure right join is marked as unique +-- explain (verbose, costs off) +-- select * from j1 right join j2 on j1.id = j2.id; + +-- ensure full join is marked as unique +-- explain (verbose, costs off) +-- select * from j1 full join j2 on j1.id = j2.id; + +-- a clauseless (cross) join can't be unique +-- explain (verbose, costs off) +-- select * from j1 cross join j2; + +-- ensure a natural join is marked as unique +-- explain (verbose, costs off) +-- select * from j1 natural join j2; + +-- ensure a distinct clause allows the inner to become unique +-- explain (verbose, costs off) +-- select * from j1 +-- inner join (select distinct id from j3) j3 on j1.id = j3.id; + +-- ensure group by clause allows the inner to become unique +-- explain (verbose, costs off) +-- select * from j1 +-- inner join (select id from j3 group by id) j3 on j1.id = j3.id; + +-- drop table if exists j1; +-- drop table if exists j2; +-- drop table if exists j3; + +-- test more complex permutations of unique joins + +CREATE TABLE j1 (id1 int, id2 int) USING parquet; +CREATE TABLE j2 (id1 int, id2 int) USING parquet; +-- create table j3 (id1 int, id2 int) using parquet; + +INSERT INTO j1 values(1,1),(1,2); +INSERT INTO j2 values(1,1); +-- insert into j3 values(1,1); + +-- analyze j1; +-- analyze j2; +-- analyze j3; + +-- ensure there's no unique join when not all columns which are part of the +-- unique index are seen in the join clause +-- explain (verbose, costs off) +-- select * from j1 +-- inner join j2 on j1.id1 = j2.id1; + +-- ensure proper unique detection with multiple join quals +-- explain (verbose, costs off) +-- select * from j1 +-- inner join j2 on j1.id1 = j2.id1 and j1.id2 = j2.id2; + +-- ensure we don't detect the join to be unique when quals are not part of the +-- join condition +-- explain (verbose, costs off) +-- select * from j1 +-- inner join j2 on j1.id1 = j2.id1 where j1.id2 = 1; + +-- as above, but for left joins. +-- explain (verbose, costs off) +-- select * from j1 +-- left join j2 on j1.id1 = j2.id1 where j1.id2 = 1; + +-- validate logic in merge joins which skips mark and restore. +-- it should only do this if all quals which were used to detect the unique +-- are present as join quals, and not plain quals. +-- set enable_nestloop to 0; +-- set enable_hashjoin to 0; +-- set enable_sort to 0; + +-- create indexes that will be preferred over the PKs to perform the join +-- create index j1_id1_idx on j1 (id1) where id1 % 1000 = 1; +-- create index j2_id1_idx on j2 (id1) where id1 % 1000 = 1; + +-- need an additional row in j2, if we want j2_id1_idx to be preferred +INSERT INTO j2 values(1,2); +-- analyze j2; + +-- explain (costs off) select * from j1 +-- inner join j2 on j1.id1 = j2.id1 and j1.id2 = j2.id2 +-- where j1.id1 % 1000 = 1 and j2.id1 % 1000 = 1; + +select * from j1 +inner join j2 on udf(j1.id1) = udf(j2.id1) and udf(udf(j1.id2)) = udf(j2.id2) +where udf(j1.id1) % 1000 = 1 and udf(udf(j2.id1) % 1000) = 1; + +-- reset enable_nestloop; +-- reset enable_hashjoin; +-- reset enable_sort; + +drop table j1; +drop table j2; +-- drop table j3; + +-- Skip these tests because it only test explain +-- check that semijoin inner is not seen as unique for a portion of the outerrel +-- explain (verbose, costs off) +-- select t1.unique1, t2.hundred +-- from onek t1, tenk1 t2 +-- where exists (select 1 from tenk1 t3 +-- where t3.thousand = t1.unique1 and t3.tenthous = t2.hundred) +-- and t1.unique1 < 1; + +-- ... unless it actually is unique +-- create table j3 as select unique1, tenthous from onek; +-- vacuum analyze j3; +-- create unique index on j3(unique1, tenthous); + +-- explain (verbose, costs off) +-- select t1.unique1, t2.hundred +-- from onek t1, tenk1 t2 +-- where exists (select 1 from j3 +-- where j3.unique1 = t1.unique1 and j3.tenthous = t2.hundred) +-- and t1.unique1 < 1; + +-- drop table j3; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-select_having.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-select_having.sql new file mode 100644 index 000000000000..76c0b198aa43 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-select_having.sql @@ -0,0 +1,58 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- SELECT_HAVING +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql +-- +-- This test file was converted from inputs/postgreSQL/select_having.sql +-- TODO: We should add UDFs in GROUP BY clause when [SPARK-28386] and [SPARK-26741] is resolved. + +-- load test data +CREATE TABLE test_having (a int, b int, c string, d string) USING parquet; +INSERT INTO test_having VALUES (0, 1, 'XXXX', 'A'); +INSERT INTO test_having VALUES (1, 2, 'AAAA', 'b'); +INSERT INTO test_having VALUES (2, 2, 'AAAA', 'c'); +INSERT INTO test_having VALUES (3, 3, 'BBBB', 'D'); +INSERT INTO test_having VALUES (4, 3, 'BBBB', 'e'); +INSERT INTO test_having VALUES (5, 3, 'bbbb', 'F'); +INSERT INTO test_having VALUES (6, 4, 'cccc', 'g'); +INSERT INTO test_having VALUES (7, 4, 'cccc', 'h'); +INSERT INTO test_having VALUES (8, 4, 'CCCC', 'I'); +INSERT INTO test_having VALUES (9, 4, 'CCCC', 'j'); + +SELECT udf(b), udf(c) FROM test_having + GROUP BY b, c HAVING udf(count(*)) = 1 ORDER BY udf(b), udf(c); + +-- HAVING is effectively equivalent to WHERE in this case +SELECT udf(b), udf(c) FROM test_having + GROUP BY b, c HAVING udf(b) = 3 ORDER BY udf(b), udf(c); + +-- [SPARK-28386] Cannot resolve ORDER BY columns with GROUP BY and HAVING +-- SELECT lower(c), count(c) FROM test_having +-- GROUP BY lower(c) HAVING count(*) > 2 OR min(a) = max(a) +-- ORDER BY lower(c); + +SELECT udf(c), max(udf(a)) FROM test_having + GROUP BY c HAVING udf(count(*)) > 2 OR udf(min(a)) = udf(max(a)) + ORDER BY c; + +-- test degenerate cases involving HAVING without GROUP BY +-- Per SQL spec, these should generate 0 or 1 row, even without aggregates + +SELECT udf(udf(min(udf(a)))), udf(udf(max(udf(a)))) FROM test_having HAVING udf(udf(min(udf(a)))) = udf(udf(max(udf(a)))); +SELECT udf(min(udf(a))), udf(udf(max(a))) FROM test_having HAVING udf(min(a)) < udf(max(udf(a))); + +-- errors: ungrouped column references +SELECT udf(a) FROM test_having HAVING udf(min(a)) < udf(max(a)); +SELECT 1 AS one FROM test_having HAVING udf(a) > 1; + +-- the really degenerate case: need not scan table at all +SELECT 1 AS one FROM test_having HAVING udf(udf(1) > udf(2)); +SELECT 1 AS one FROM test_having HAVING udf(udf(1) < udf(2)); + +-- [SPARK-33008] Spark SQL throws an exception +-- and just to prove that we aren't scanning the table: +SELECT 1 AS one FROM test_having WHERE 1/udf(a) = 1 HAVING 1 < 2; + +DROP TABLE test_having; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-select_implicit.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-select_implicit.sql new file mode 100755 index 000000000000..1cbd77c6cf86 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/postgreSQL/udf-select_implicit.sql @@ -0,0 +1,165 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- SELECT_IMPLICIT +-- Test cases for queries with ordering terms missing from the target list. +-- This used to be called "junkfilter.sql". +-- The parser uses the term "resjunk" to handle these cases. +-- - thomas 1998-07-09 +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_implicit.sql +-- +-- This test file was converted from postgreSQL/select_implicit.sql + +-- load test data +CREATE TABLE test_missing_target (a int, b int, c string, d string) using parquet; +INSERT INTO test_missing_target VALUES (0, 1, 'XXXX', 'A'); +INSERT INTO test_missing_target VALUES (1, 2, 'ABAB', 'b'); +INSERT INTO test_missing_target VALUES (2, 2, 'ABAB', 'c'); +INSERT INTO test_missing_target VALUES (3, 3, 'BBBB', 'D'); +INSERT INTO test_missing_target VALUES (4, 3, 'BBBB', 'e'); +INSERT INTO test_missing_target VALUES (5, 3, 'bbbb', 'F'); +INSERT INTO test_missing_target VALUES (6, 4, 'cccc', 'g'); +INSERT INTO test_missing_target VALUES (7, 4, 'cccc', 'h'); +INSERT INTO test_missing_target VALUES (8, 4, 'CCCC', 'I'); +INSERT INTO test_missing_target VALUES (9, 4, 'CCCC', 'j'); + + +-- w/ existing GROUP BY target +SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY +udf(test_missing_target.c) +ORDER BY udf(c); + +-- w/o existing GROUP BY target using a relation name in GROUP BY clause +SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(test_missing_target.c) +ORDER BY udf(c); + +-- w/o existing GROUP BY target and w/o existing a different ORDER BY target +-- failure expected +SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(a) ORDER BY udf(b); + +-- w/o existing GROUP BY target and w/o existing same ORDER BY target +SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(b) ORDER BY udf(b); + +-- w/ existing GROUP BY target using a relation name in target +SELECT udf(test_missing_target.b), udf(count(*)) + FROM test_missing_target GROUP BY udf(b) ORDER BY udf(b); + +-- w/o existing GROUP BY target +SELECT udf(c) FROM test_missing_target ORDER BY udf(a); + +-- w/o existing ORDER BY target +SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(b) ORDER BY udf(b) desc; + +-- group using reference number +SELECT udf(count(*)) FROM test_missing_target ORDER BY udf(1) desc; + +-- order using reference number +SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY 1 ORDER BY 1; + +-- group using reference number out of range +-- failure expected +SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY 3; + +-- group w/o existing GROUP BY and ORDER BY target under ambiguous condition +-- failure expected +SELECT udf(count(*)) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY udf(b) ORDER BY udf(b); + +-- order w/ target under ambiguous condition +-- failure NOT expected +SELECT udf(a), udf(a) FROM test_missing_target + ORDER BY udf(a); + +-- order expression w/ target under ambiguous condition +-- failure NOT expected +SELECT udf(udf(a)/2), udf(udf(a)/2) FROM test_missing_target + ORDER BY udf(udf(a)/2); + +-- group expression w/ target under ambiguous condition +-- failure NOT expected +SELECT udf(a/2), udf(a/2) FROM test_missing_target + GROUP BY udf(a/2) ORDER BY udf(a/2); + +-- group w/ existing GROUP BY target under ambiguous condition +SELECT udf(x.b), udf(count(*)) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY udf(x.b) ORDER BY udf(x.b); + +-- group w/o existing GROUP BY target under ambiguous condition +SELECT udf(count(*)) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY udf(x.b) ORDER BY udf(x.b); + +-- [SPARK-28329] SELECT INTO syntax +-- group w/o existing GROUP BY target under ambiguous condition +-- into a table +-- SELECT count(*) INTO TABLE test_missing_target2 +-- FROM test_missing_target x, test_missing_target y +-- WHERE x.a = y.a +-- GROUP BY x.b ORDER BY x.b; +-- SELECT * FROM test_missing_target2; + + +-- Functions and expressions + +-- w/ existing GROUP BY target +SELECT udf(a%2), udf(count(udf(b))) FROM test_missing_target +GROUP BY udf(test_missing_target.a%2) +ORDER BY udf(test_missing_target.a%2); + +-- w/o existing GROUP BY target using a relation name in GROUP BY clause +SELECT udf(count(c)) FROM test_missing_target +GROUP BY udf(lower(test_missing_target.c)) +ORDER BY udf(lower(test_missing_target.c)); + +-- w/o existing GROUP BY target and w/o existing a different ORDER BY target +-- failure expected +SELECT udf(count(udf(a))) FROM test_missing_target GROUP BY udf(a) ORDER BY udf(b); + +-- w/o existing GROUP BY target and w/o existing same ORDER BY target +SELECT udf(count(b)) FROM test_missing_target GROUP BY udf(b/2) ORDER BY udf(b/2); + +-- w/ existing GROUP BY target using a relation name in target +SELECT udf(lower(test_missing_target.c)), udf(count(udf(c))) + FROM test_missing_target GROUP BY udf(lower(c)) ORDER BY udf(lower(c)); + +-- w/o existing GROUP BY target +SELECT udf(a) FROM test_missing_target ORDER BY udf(upper(udf(d))); + +-- w/o existing ORDER BY target +SELECT udf(count(b)) FROM test_missing_target + GROUP BY udf((b + 1) / 2) ORDER BY udf((b + 1) / 2) desc; + +-- group w/o existing GROUP BY and ORDER BY target under ambiguous condition +-- failure expected +SELECT udf(count(udf(x.a))) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY udf(b/2) ORDER BY udf(b/2); + +-- group w/ existing GROUP BY target under ambiguous condition +SELECT udf(x.b/2), udf(count(udf(x.b))) FROM test_missing_target x, +test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY udf(x.b/2) ORDER BY udf(x.b/2); + +-- group w/o existing GROUP BY target under ambiguous condition +-- failure expected due to ambiguous b in count(b) +SELECT udf(count(udf(b))) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY udf(x.b/2); + +-- [SPARK-28329] SELECT INTO syntax +-- group w/o existing GROUP BY target under ambiguous condition +-- into a table +-- SELECT count(x.b) INTO TABLE test_missing_target3 +-- FROM test_missing_target x, test_missing_target y +-- WHERE x.a = y.a +-- GROUP BY x.b/2 ORDER BY x.b/2; +-- SELECT * FROM test_missing_target3; + +-- Cleanup +DROP TABLE test_missing_target; +-- DROP TABLE test_missing_target2; +-- DROP TABLE test_missing_target3; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-count.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-count.sql new file mode 100644 index 000000000000..82058d43f88a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-count.sql @@ -0,0 +1,28 @@ +-- This test file was converted from count.sql +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) +AS testData(a, b); + +-- count with single expression +SELECT + udf(count(*)), udf(count(1)), udf(count(null)), udf(count(a)), udf(count(b)), udf(count(a + b)), udf(count((a, b))) +FROM testData; + +-- distinct count with single expression +SELECT + udf(count(DISTINCT 1)), + udf(count(DISTINCT null)), + udf(count(DISTINCT a)), + udf(count(DISTINCT b)), + udf(count(DISTINCT (a + b))), + udf(count(DISTINCT (a, b))) +FROM testData; + +-- count with multiple expressions +SELECT udf(count(a, b)), udf(count(b, a)), udf(count(testData.*, testData.*)) FROM testData; + +-- distinct count with multiple expressions +SELECT + udf(count(DISTINCT a, b)), udf(count(DISTINCT b, a)), udf(count(DISTINCT *)), udf(count(DISTINCT testData.*, testData.*)) +FROM testData; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-cross-join.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-cross-join.sql new file mode 100644 index 000000000000..9aa6de7e92b8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-cross-join.sql @@ -0,0 +1,37 @@ +-- Cross join detection and error checking is done in JoinSuite since explain output is +-- used in the error message and the ids are not stable. Only positive cases are checked here. +-- This test file was converted from cross-join.sql. + +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1); + +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2); + +-- Cross joins with and without predicates +SELECT * FROM nt1 cross join nt2; +SELECT * FROM nt1 cross join nt2 where udf(nt1.k) = udf(nt2.k); +SELECT * FROM nt1 cross join nt2 on (udf(nt1.k) = udf(nt2.k)); +SELECT * FROM nt1 cross join nt2 where udf(nt1.v1) = "1" and udf(nt2.v2) = "22"; + +SELECT udf(a.key), udf(b.key) FROM +(SELECT udf(k) key FROM nt1 WHERE v1 < 2) a +CROSS JOIN +(SELECT udf(k) key FROM nt2 WHERE v2 = 22) b; + +-- Join reordering +create temporary view A(a, va) as select * from nt1; +create temporary view B(b, vb) as select * from nt1; +create temporary view C(c, vc) as select * from nt1; +create temporary view D(d, vd) as select * from nt1; + +-- Allowed since cross join with C is explicit +select * from ((A join B on (udf(a) = udf(b))) cross join C) join D on (udf(a) = udf(d)); +-- Cross joins with non-equal predicates +SELECT * FROM nt1 CROSS JOIN nt2 ON (udf(nt1.k) > udf(nt2.k)); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-except-all.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-except-all.sql new file mode 100644 index 000000000000..71e05e67698c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-except-all.sql @@ -0,0 +1,162 @@ +-- This test file was converted from except-all.sql. + +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (0), (1), (2), (2), (2), (2), (3), (null), (null) AS tab1(c1); +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1), (2), (2), (3), (5), (5), (null) AS tab2(c1); +CREATE TEMPORARY VIEW tab3 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (2, 3), + (2, 2) + AS tab3(k, v); +CREATE TEMPORARY VIEW tab4 AS SELECT * FROM VALUES + (1, 2), + (2, 3), + (2, 2), + (2, 2), + (2, 20) + AS tab4(k, v); + +-- Basic EXCEPT ALL +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2; + +-- MINUS ALL (synonym for EXCEPT) +SELECT udf(c1) FROM tab1 +MINUS ALL +SELECT udf(c1) FROM tab2; + +-- EXCEPT ALL same table in both branches +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 WHERE udf(c1) IS NOT NULL; + +-- Empty left relation +SELECT udf(c1) FROM tab1 WHERE udf(c1) > 5 +EXCEPT ALL +SELECT udf(c1) FROM tab2; + +-- Empty right relation +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 WHERE udf(c1 > udf(6)); + +-- Type Coerced ExceptAll +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT CAST(udf(1) AS BIGINT); + +-- Error as types of two side are not compatible +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT array(1); + +-- Basic +SELECT udf(k), v FROM tab3 +EXCEPT ALL +SELECT k, udf(v) FROM tab4; + +-- Basic +SELECT k, udf(v) FROM tab4 +EXCEPT ALL +SELECT udf(k), v FROM tab3; + +-- EXCEPT ALL + INTERSECT +SELECT udf(k), udf(v) FROM tab4 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab3 +INTERSECT DISTINCT +SELECT udf(k), udf(v) FROM tab4; + +-- EXCEPT ALL + EXCEPT +SELECT udf(k), v FROM tab4 +EXCEPT ALL +SELECT k, udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), udf(v) FROM tab4; + +-- Chain of set operations +SELECT k, udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab4 +UNION ALL +SELECT udf(k), v FROM tab3 +EXCEPT DISTINCT +SELECT k, udf(v) FROM tab4; + +-- Mismatch on number of columns across both branches +SELECT k FROM tab3 +EXCEPT ALL +SELECT k, v FROM tab4; + +-- Chain of set operations +SELECT udf(k), udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab4 +UNION +SELECT udf(k), udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), udf(v) FROM tab4; + +-- Using MINUS ALL +SELECT udf(k), udf(v) FROM tab3 +MINUS ALL +SELECT k, udf(v) FROM tab4 +UNION +SELECT udf(k), udf(v) FROM tab3 +MINUS DISTINCT +SELECT k, udf(v) FROM tab4; + +-- Chain of set operations +SELECT k, udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), v FROM tab4 +EXCEPT DISTINCT +SELECT k, udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), v FROM tab4; + +-- Join under except all. Should produce empty resultset since both left and right sets +-- are same. +SELECT * +FROM (SELECT tab3.k, + udf(tab4.v) + FROM tab3 + JOIN tab4 + ON udf(tab3.k) = tab4.k) +EXCEPT ALL +SELECT * +FROM (SELECT udf(tab3.k), + tab4.v + FROM tab3 + JOIN tab4 + ON tab3.k = udf(tab4.k)); + +-- Join under except all (2) +SELECT * +FROM (SELECT udf(udf(tab3.k)), + udf(tab4.v) + FROM tab3 + JOIN tab4 + ON udf(udf(tab3.k)) = udf(tab4.k)) +EXCEPT ALL +SELECT * +FROM (SELECT udf(tab4.v) AS k, + udf(udf(tab3.k)) AS v + FROM tab3 + JOIN tab4 + ON udf(tab3.k) = udf(tab4.k)); + +-- Group by under ExceptAll +SELECT udf(v) FROM tab3 GROUP BY v +EXCEPT ALL +SELECT udf(k) FROM tab4 GROUP BY k; + +-- Clean-up +DROP VIEW IF EXISTS tab1; +DROP VIEW IF EXISTS tab2; +DROP VIEW IF EXISTS tab3; +DROP VIEW IF EXISTS tab4; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-except.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-except.sql new file mode 100644 index 000000000000..378b5bbf2df8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-except.sql @@ -0,0 +1,58 @@ +-- This test file was converted from except.sql. +-- Tests different scenarios of except operation +create temporary view t1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", NULL) + as t1(k, v); + +create temporary view t2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5), + ("one", NULL), + (NULL, 5) + as t2(k, v); + + +-- Except operation that will be replaced by left anti join +SELECT udf(k), udf(v) FROM t1 EXCEPT SELECT udf(k), udf(v) FROM t2; + + +-- Except operation that will be replaced by Filter: SPARK-22181 +SELECT * FROM t1 EXCEPT SELECT * FROM t1 where udf(v) <> 1 and v <> udf(2); + + +-- Except operation that will be replaced by Filter: SPARK-22181 +SELECT * FROM t1 where udf(v) <> 1 and v <> udf(22) EXCEPT SELECT * FROM t1 where udf(v) <> 2 and v >= udf(3); + + +-- Except operation that will be replaced by Filter: SPARK-22181 +SELECT t1.* FROM t1, t2 where t1.k = t2.k +EXCEPT +SELECT t1.* FROM t1, t2 where t1.k = t2.k and t1.k != udf('one'); + + +-- Except operation that will be replaced by left anti join +SELECT * FROM t2 where v >= udf(1) and udf(v) <> 22 EXCEPT SELECT * FROM t1; + + +-- Except operation that will be replaced by left anti join +SELECT (SELECT min(udf(k)) FROM t2 WHERE t2.k = t1.k) min_t2 FROM t1 +MINUS +SELECT (SELECT udf(min(k)) FROM t2) abs_min_t2 FROM t1 WHERE t1.k = udf('one'); + + +-- Except operation that will be replaced by left anti join +SELECT t1.k +FROM t1 +WHERE t1.v <= (SELECT udf(max(udf(t2.v))) + FROM t2 + WHERE udf(t2.k) = udf(t1.k)) +MINUS +SELECT t1.k +FROM t1 +WHERE udf(t1.v) >= (SELECT min(udf(t2.v)) + FROM t2 + WHERE t2.k = t1.k); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-group-analytics.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-group-analytics.sql new file mode 100644 index 000000000000..b7dfc011ef11 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-group-analytics.sql @@ -0,0 +1,63 @@ +-- This test file was converted from group-analytics.sql. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData(a, b); + +-- CUBE on overlapping columns +SELECT udf(a + b), b, udf(SUM(a - b)) FROM testData GROUP BY udf(a + b), b WITH CUBE; + +SELECT udf(a), udf(b), SUM(b) FROM testData GROUP BY udf(a), b WITH CUBE; + +-- ROLLUP on overlapping columns +SELECT udf(a + b), b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP; + +SELECT udf(a), b, udf(SUM(b)) FROM testData GROUP BY udf(a), b WITH ROLLUP; + +CREATE OR REPLACE TEMPORARY VIEW courseSales AS SELECT * FROM VALUES +("dotNET", 2012, 10000), ("Java", 2012, 20000), ("dotNET", 2012, 5000), ("dotNET", 2013, 48000), ("Java", 2013, 30000) +AS courseSales(course, year, earnings); + +-- ROLLUP +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY udf(course), year; + +-- CUBE +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, udf(year); + +-- GROUPING SETS +SELECT course, udf(year), SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year); +SELECT course, year, udf(SUM(earnings)) FROM courseSales GROUP BY course, year GROUPING SETS(course); +SELECT udf(course), year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year); + +-- GROUPING SETS with aggregate functions containing groupBy columns +SELECT course, udf(SUM(earnings)) AS sum FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, udf(sum); +SELECT course, SUM(earnings) AS sum, GROUPING_ID(course, earnings) FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY udf(course), sum; + +-- GROUPING/GROUPING_ID +SELECT udf(course), udf(year), GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales +GROUP BY CUBE(course, year); +SELECT course, udf(year), GROUPING(course) FROM courseSales GROUP BY course, udf(year); +SELECT course, udf(year), GROUPING_ID(course, year) FROM courseSales GROUP BY udf(course), year; +SELECT course, year, grouping__id FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, udf(year); + +-- GROUPING/GROUPING_ID in having clause +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) +HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0 ORDER BY course, udf(year); +SELECT course, udf(year) FROM courseSales GROUP BY udf(course), year HAVING GROUPING(course) > 0; +SELECT course, udf(udf(year)) FROM courseSales GROUP BY course, year HAVING GROUPING_ID(course) > 0; +SELECT udf(course), year FROM courseSales GROUP BY CUBE(course, year) HAVING grouping__id > 0; + +-- GROUPING/GROUPING_ID in orderBy clause +SELECT course, year, GROUPING(course), GROUPING(year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, udf(year); +SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, udf(year); +SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GROUPING(course); +SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GROUPING_ID(course); +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, udf(course), year; + +-- Aliases in SELECT could be used in ROLLUP/CUBE/GROUPING SETS +SELECT udf(a + b) AS k1, udf(b) AS k2, SUM(a - b) FROM testData GROUP BY CUBE(k1, k2); +SELECT udf(udf(a + b)) AS k, b, SUM(a - b) FROM testData GROUP BY ROLLUP(k, b); +SELECT udf(a + b), udf(udf(b)) AS k, SUM(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k) diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-group-by.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-group-by.sql new file mode 100644 index 000000000000..0cc57c97b020 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-group-by.sql @@ -0,0 +1,156 @@ +-- This test file was converted from group-by.sql. +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b); + +-- Aggregate with empty GroupBy expressions. +SELECT udf(a), udf(COUNT(b)) FROM testData; +SELECT COUNT(udf(a)), udf(COUNT(b)) FROM testData; + +-- Aggregate with non-empty GroupBy expressions. +SELECT udf(a), COUNT(udf(b)) FROM testData GROUP BY a; +SELECT udf(a), udf(COUNT(udf(b))) FROM testData GROUP BY b; +SELECT COUNT(udf(a)), COUNT(udf(b)) FROM testData GROUP BY udf(a); + +-- Aggregate grouped by literals. +SELECT 'foo', COUNT(udf(a)) FROM testData GROUP BY 1; + +-- Aggregate grouped by literals (whole stage code generation). +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY udf(1); + +-- Aggregate grouped by literals (hash aggregate). +SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1); + +-- Aggregate grouped by literals (sort aggregate). +SELECT 'foo', MAX(STRUCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1); + +-- Aggregate with complex GroupBy expressions. +SELECT udf(a + b), udf(COUNT(b)) FROM testData GROUP BY a + b; +SELECT udf(a + 2), udf(COUNT(b)) FROM testData GROUP BY a + 1; +SELECT udf(a + 1) + 1, udf(COUNT(b)) FROM testData GROUP BY udf(a + 1); + +-- Aggregate with nulls. +SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a)) +FROM testData; + +-- Aggregate with foldable input and multiple distinct groups. +SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY udf(a); + +-- Aliases in SELECT could be used in GROUP BY +SELECT udf(a) AS k, COUNT(udf(b)) FROM testData GROUP BY k; +SELECT a AS k, udf(COUNT(b)) FROM testData GROUP BY k HAVING k > 1; + +-- Aggregate functions cannot be used in GROUP BY +SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k; + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v); +SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY udf(a); + +-- turn off group by aliases +set spark.sql.groupByAliases=false; + +-- Check analysis exceptions +SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k; + +-- Aggregate with empty input and non-empty GroupBy expressions. +SELECT udf(a), COUNT(udf(1)) FROM testData WHERE false GROUP BY udf(a); + +-- Aggregate with empty input and empty GroupBy expressions. +SELECT udf(COUNT(1)) FROM testData WHERE false; +SELECT 1 FROM (SELECT udf(COUNT(1)) FROM testData WHERE false) t; + +-- Aggregate with empty GroupBy expressions and filter on top +SELECT 1 from ( + SELECT 1 AS z, + udf(MIN(a.x)) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z; + +-- SPARK-24369 multiple distinct aggregations having the same argument set +SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y); + +-- SPARK-25708 HAVING without GROUP BY means global aggregate +SELECT udf(1) FROM range(10) HAVING true; + +SELECT udf(udf(1)) FROM range(10) HAVING MAX(id) > 0; + +SELECT udf(id) FROM range(10) HAVING id > 0; + +-- Test data +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v); + +-- empty table +SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0; + +-- all null values +SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4; + +-- aggregates are null Filtering +SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5; + +-- group by +SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k); + +-- having +SELECT udf(k), every(v) FROM test_agg GROUP BY k HAVING every(v) = false; +SELECT udf(k), udf(every(v)) FROM test_agg GROUP BY udf(k) HAVING every(v) IS NULL; + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT udf(k), + udf(Every(v)) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY udf(k); + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT udf(udf(k)), + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY udf(udf(k)); + +-- input type checking Int +SELECT every(udf(1)); + +-- input type checking Short +SELECT some(udf(1S)); + +-- input type checking Long +SELECT any(udf(1L)); + +-- input type checking String +SELECT udf(every("true")); + +-- every/some/any aggregates are supported as windows expression. +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, udf(udf(v)), some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT udf(udf(k)), v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; + +-- Having referencing aggregate expressions is ok. +SELECT udf(count(*)) FROM test_agg HAVING count(*) > 1L; +SELECT k, udf(max(v)) FROM test_agg GROUP BY k HAVING max(v) = true; + +-- Aggrgate expressions can be referenced through an alias +SELECT * FROM (SELECT udf(COUNT(*)) AS cnt FROM test_agg) WHERE cnt > 1L; + +-- Error when aggregate expressions are in where clause directly +SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L; +SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L; +SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-having.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-having.sql new file mode 100644 index 000000000000..ff8573ad7e56 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-having.sql @@ -0,0 +1,20 @@ +-- This test file was converted from having.sql. + +create temporary view hav as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", 5) + as hav(k, v); + +-- having clause +SELECT udf(k) AS k, udf(sum(v)) FROM hav GROUP BY k HAVING udf(sum(v)) > 2; + +-- having condition contains grouping column +SELECT udf(count(udf(k))) FROM hav GROUP BY v + 1 HAVING v + 1 = udf(2); + +-- SPARK-11032: resolve having correctly +SELECT udf(MIN(t.v)) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(udf(COUNT(udf(1))) > 0); + +-- SPARK-20329: make sure we handle timezones correctly +SELECT udf(a + b) FROM VALUES (1L, 2), (3L, 4) AS T(a, b) GROUP BY a + b HAVING a + b > udf(1); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-inline-table.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-inline-table.sql new file mode 100644 index 000000000000..179e528d74ca --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-inline-table.sql @@ -0,0 +1,54 @@ +-- This test file was converted from inline-table.sql. +-- [SPARK-28291] UDFs cannot be evaluated within inline table definition +-- TODO: We should add UDFs in VALUES clause when [SPARK-28291] is resolved. + +-- single row, without table and column alias +select udf(col1), udf(col2) from values ("one", 1); + +-- single row, without column alias +select udf(col1), udf(udf(col2)) from values ("one", 1) as data; + +-- single row +select udf(a), b from values ("one", 1) as data(a, b); + +-- single column multiple rows +select udf(a) from values 1, 2, 3 as data(a); + +-- three rows +select udf(a), b from values ("one", 1), ("two", 2), ("three", null) as data(a, b); + +-- null type +select udf(a), b from values ("one", null), ("two", null) as data(a, b); + +-- int and long coercion +select udf(a), b from values ("one", 1), ("two", 2L) as data(a, b); + +-- foldable expressions +select udf(udf(a)), udf(b) from values ("one", 1 + 0), ("two", 1 + 3L) as data(a, b); + +-- complex types +select udf(a), b from values ("one", array(0, 1)), ("two", array(2, 3)) as data(a, b); + +-- decimal and double coercion +select udf(a), b from values ("one", 2.0), ("two", 3.0D) as data(a, b); + +-- error reporting: nondeterministic function rand +select udf(a), b from values ("one", rand(5)), ("two", 3.0D) as data(a, b); + +-- error reporting: different number of columns +select udf(a), udf(b) from values ("one", 2.0), ("two") as data(a, b); + +-- error reporting: types that are incompatible +select udf(a), udf(b) from values ("one", array(0, 1)), ("two", struct(1, 2)) as data(a, b); + +-- error reporting: number aliases different from number data values +select udf(a), udf(b) from values ("one"), ("two") as data(a, b); + +-- error reporting: unresolved expression +select udf(a), udf(b) from values ("one", random_not_exist_func(1)), ("two", 2) as data(a, b); + +-- error reporting: aggregate expression +select udf(a), udf(b) from values ("one", count(1)), ("two", 2) as data(a, b); + +-- string to timestamp +select udf(a), b from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991-12-06 01:00:00.0'), timestamp('1991-12-06 12:00:00.0'))) as data(a, b); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-inner-join.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-inner-join.sql new file mode 100644 index 000000000000..8bd61b804ab6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-inner-join.sql @@ -0,0 +1,17 @@ +-- This test file was converted from inner-join.sql. +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a); +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a); +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a); +CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a); + +CREATE TEMPORARY VIEW ta AS +SELECT udf(a) AS a, udf('a') AS tag FROM t1 +UNION ALL +SELECT udf(a) AS a, udf('b') AS tag FROM t2; + +CREATE TEMPORARY VIEW tb AS +SELECT udf(a) AS a, udf('a') AS tag FROM t3 +UNION ALL +SELECT udf(a) AS a, udf('b') AS tag FROM t4; + +SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-intersect-all.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-intersect-all.sql new file mode 100644 index 000000000000..028d4c769599 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-intersect-all.sql @@ -0,0 +1,162 @@ +-- This test file was converted from intersect-all.sql. + +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (1, 3), + (2, 3), + (null, null), + (null, null) + AS tab1(k, v); +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (2, 3), + (3, 4), + (null, null), + (null, null) + AS tab2(k, v); + +-- Basic INTERSECT ALL +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT k, udf(v) FROM tab2; + +-- INTERSECT ALL same table in both branches +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab1 WHERE udf(k) = 1; + +-- Empty left relation +SELECT udf(k), udf(v) FROM tab1 WHERE k > udf(2) +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2; + +-- Empty right relation +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab2 WHERE udf(udf(k)) > 3; + +-- Type Coerced INTERSECT ALL +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT CAST(udf(1) AS BIGINT), CAST(udf(2) AS BIGINT); + +-- Error as types of two side are not compatible +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT array(1), udf(2); + +-- Mismatch on number of columns across both branches +SELECT udf(k) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2; + +-- Basic +SELECT udf(k), v FROM tab2 +INTERSECT ALL +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2; + +-- Chain of different `set operations +SELECT udf(k), v FROM tab1 +EXCEPT +SELECT k, udf(v) FROM tab2 +UNION ALL +SELECT k, udf(udf(v)) FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab2 +; + +-- Chain of different `set operations +SELECT udf(k), udf(v) FROM tab1 +EXCEPT +SELECT udf(k), v FROM tab2 +EXCEPT +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(udf(v)) FROM tab2 +; + +-- test use parenthesis to control order of evaluation +( + ( + ( + SELECT udf(k), v FROM tab1 + EXCEPT + SELECT k, udf(v) FROM tab2 + ) + EXCEPT + SELECT udf(k), udf(v) FROM tab1 + ) + INTERSECT ALL + SELECT udf(k), udf(v) FROM tab2 +) +; + +-- Join under intersect all +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(udf(tab1.k)) = tab2.k) +INTERSECT ALL +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(tab1.k) = udf(udf(tab2.k))); + +-- Join under intersect all (2) +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(tab1.k) = udf(tab2.k)) +INTERSECT ALL +SELECT * +FROM (SELECT udf(tab2.v) AS k, + udf(tab1.k) AS v + FROM tab1 + JOIN tab2 + ON tab1.k = udf(tab2.k)); + +-- Group by under intersect all +SELECT udf(v) FROM tab1 GROUP BY v +INTERSECT ALL +SELECT udf(udf(k)) FROM tab2 GROUP BY k; + +-- Test pre spark2.4 behaviour of set operation precedence +-- All the set operators are given equal precedence and are evaluated +-- from left to right as they appear in the query. + +-- Set the property +SET spark.sql.legacy.setopsPrecedence.enabled= true; + +SELECT udf(k), v FROM tab1 +EXCEPT +SELECT k, udf(v) FROM tab2 +UNION ALL +SELECT udf(k), udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(udf(k)), udf(v) FROM tab2; + +SELECT k, udf(v) FROM tab1 +EXCEPT +SELECT udf(k), v FROM tab2 +UNION ALL +SELECT udf(k), udf(v) FROM tab1 +INTERSECT +SELECT udf(k), udf(udf(v)) FROM tab2; + +-- Restore the property +SET spark.sql.legacy.setopsPrecedence.enabled = false; + +-- Clean-up +DROP VIEW IF EXISTS tab1; +DROP VIEW IF EXISTS tab2; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-join-empty-relation.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-join-empty-relation.sql new file mode 100644 index 000000000000..b46206d4530e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-join-empty-relation.sql @@ -0,0 +1,30 @@ +-- This test file was converted from join-empty-relation.sql. + +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a); +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a); + +CREATE TEMPORARY VIEW empty_table as SELECT a FROM t2 WHERE false; + +SELECT udf(t1.a), udf(empty_table.a) FROM t1 INNER JOIN empty_table ON (udf(t1.a) = udf(udf(empty_table.a))); +SELECT udf(t1.a), udf(udf(empty_table.a)) FROM t1 CROSS JOIN empty_table ON (udf(udf(t1.a)) = udf(empty_table.a)); +SELECT udf(udf(t1.a)), empty_table.a FROM t1 LEFT OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)); +SELECT udf(t1.a), udf(empty_table.a) FROM t1 RIGHT OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)); +SELECT udf(t1.a), empty_table.a FROM t1 FULL OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)); +SELECT udf(udf(t1.a)) FROM t1 LEFT SEMI JOIN empty_table ON (udf(t1.a) = udf(udf(empty_table.a))); +SELECT udf(t1.a) FROM t1 LEFT ANTI JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)); + +SELECT udf(empty_table.a), udf(t1.a) FROM empty_table INNER JOIN t1 ON (udf(udf(empty_table.a)) = udf(t1.a)); +SELECT udf(empty_table.a), udf(udf(t1.a)) FROM empty_table CROSS JOIN t1 ON (udf(empty_table.a) = udf(udf(t1.a))); +SELECT udf(udf(empty_table.a)), udf(t1.a) FROM empty_table LEFT OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)); +SELECT empty_table.a, udf(t1.a) FROM empty_table RIGHT OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)); +SELECT empty_table.a, udf(udf(t1.a)) FROM empty_table FULL OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)); +SELECT udf(udf(empty_table.a)) FROM empty_table LEFT SEMI JOIN t1 ON (udf(empty_table.a) = udf(udf(t1.a))); +SELECT empty_table.a FROM empty_table LEFT ANTI JOIN t1 ON (udf(empty_table.a) = udf(t1.a)); + +SELECT udf(empty_table.a) FROM empty_table INNER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(udf(empty_table2.a))); +SELECT udf(udf(empty_table.a)) FROM empty_table CROSS JOIN empty_table AS empty_table2 ON (udf(udf(empty_table.a)) = udf(empty_table2.a)); +SELECT udf(empty_table.a) FROM empty_table LEFT OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)); +SELECT udf(udf(empty_table.a)) FROM empty_table RIGHT OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(udf(empty_table2.a))); +SELECT udf(empty_table.a) FROM empty_table FULL OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)); +SELECT udf(udf(empty_table.a)) FROM empty_table LEFT SEMI JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)); +SELECT udf(empty_table.a) FROM empty_table LEFT ANTI JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-natural-join.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-natural-join.sql new file mode 100644 index 000000000000..7cf080ea1b4e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-natural-join.sql @@ -0,0 +1,22 @@ +-- This test file was converted from natural-join.sql. + +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1); + +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2); + + +SELECT * FROM nt1 natural join nt2 where udf(k) = "one"; + +SELECT * FROM nt1 natural left join nt2 where k <> udf("") order by v1, v2; + +SELECT * FROM nt1 natural right join nt2 where udf(k) <> udf("") order by v1, v2; + +SELECT udf(count(*)) FROM nt1 natural full outer join nt2; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-outer-join.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-outer-join.sql new file mode 100644 index 000000000000..4b09bcb988d2 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-outer-join.sql @@ -0,0 +1,41 @@ +-- This test file was converted from outer-join.sql. + +-- SPARK-17099: Incorrect result when HAVING clause is added to group by query +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(-234), (145), (367), (975), (298) +as t1(int_col1); + +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES +(-769, -244), (-800, -409), (940, 86), (-507, 304), (-367, 158) +as t2(int_col0, int_col1); + +SELECT + (udf(SUM(udf(COALESCE(t1.int_col1, t2.int_col0))))), + (udf(COALESCE(t1.int_col1, t2.int_col0)) * 2) +FROM t1 +RIGHT JOIN t2 + ON udf(t2.int_col0) = udf(t1.int_col1) +GROUP BY udf(GREATEST(COALESCE(udf(t2.int_col1), 109), COALESCE(t1.int_col1, udf(-449)))), + COALESCE(t1.int_col1, t2.int_col0) +HAVING (udf(SUM(COALESCE(udf(t1.int_col1), udf(t2.int_col0))))) + > (udf(COALESCE(t1.int_col1, t2.int_col0)) * 2); + + +-- SPARK-17120: Analyzer incorrectly optimizes plan to empty LocalRelation +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (97) as t1(int_col1); + +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (0) as t2(int_col1); + +-- Set the cross join enabled flag for the LEFT JOIN test since there's no join condition. +-- Ultimately the join should be optimized away. +set spark.sql.crossJoin.enabled = true; +SELECT * +FROM ( +SELECT + udf(COALESCE(udf(t2.int_col1), udf(t1.int_col1))) AS int_col + FROM t1 + LEFT JOIN t2 ON false +) t where (udf(t.int_col)) is not null; +set spark.sql.crossJoin.enabled = false; + + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-pivot.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-pivot.sql new file mode 100644 index 000000000000..93937930de7f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-pivot.sql @@ -0,0 +1,307 @@ +-- This test file was converted from pivot.sql. + +-- Note some test cases have been commented as the current integrated UDFs cannot handle complex types + +create temporary view courseSales as select * from values + ("dotNET", 2012, 10000), + ("Java", 2012, 20000), + ("dotNET", 2012, 5000), + ("dotNET", 2013, 48000), + ("Java", 2013, 30000) + as courseSales(course, year, earnings); + +create temporary view years as select * from values + (2012, 1), + (2013, 2) + as years(y, s); + +create temporary view yearsWithComplexTypes as select * from values + (2012, array(1, 1), map('1', 1), struct(1, 'a')), + (2013, array(2, 2), map('2', 2), struct(2, 'b')) + as yearsWithComplexTypes(y, a, m, s); + +-- pivot courses +SELECT * FROM ( + SELECT udf(year), course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot years with no subquery +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (2012, 2013) +); + +-- pivot courses with multiple aggregations +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)), udf(avg(earnings)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot with no group by column +SELECT * FROM ( + SELECT udf(course) as course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot with no group by column and with multiple aggregations on different columns +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(udf(earnings))), udf(min(year)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot on join query with multiple group by columns +SELECT * FROM ( + SELECT course, year, earnings, udf(s) as s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR s IN (1, 2) +); + +-- pivot on join query with multiple aggregations on different columns +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)), udf(min(s)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot on join query with multiple columns in one aggregation +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings * s)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot with aliases and projection +SELECT 2012_s, 2013_s, 2012_a, 2013_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + udf(sum(e)) s, udf(avg(e)) a + FOR y IN (2012, 2013) +); + +-- pivot with projection and value aliases +SELECT firstYear_s, secondYear_s, firstYear_a, secondYear_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + udf(sum(e)) s, udf(avg(e)) a + FOR y IN (2012 as firstYear, 2013 secondYear) +); + +-- pivot years with non-aggregate function +SELECT * FROM courseSales +PIVOT ( + udf(abs(earnings)) + FOR year IN (2012, 2013) +); + +-- pivot with one of the expressions as non-aggregate function +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)), year + FOR course IN ('dotNET', 'Java') +); + +-- pivot with unresolvable columns +SELECT * FROM ( + SELECT course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR year IN (2012, 2013) +); + +-- pivot with complex aggregate expressions +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(ceil(udf(sum(earnings)))), avg(earnings) + 1 as a1 + FOR course IN ('dotNET', 'Java') +); + +-- pivot with invalid arguments in aggregate expressions +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(udf(avg(earnings))) + FOR course IN ('dotNET', 'Java') +); + +-- pivot on multiple pivot columns +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, year) IN (('dotNET', 2012), ('Java', 2013)) +); + +-- pivot on multiple pivot columns with aliased values +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, s) IN (('dotNET', 2) as c1, ('Java', 1) as c2) +); + +-- pivot on multiple pivot columns with values of wrong data types +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, year) IN ('dotNET', 'Java') +); + +-- pivot with unresolvable values +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (s, 2013) +); + +-- pivot with non-literal values +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (course, 2013) +); + +-- Complex type is not supported in the current UDF. Skipped for now. +-- pivot on join query with columns of complex data types +-- SELECT * FROM ( +-- SELECT course, year, a +-- FROM courseSales +-- JOIN yearsWithComplexTypes ON year = y +--) +--PIVOT ( +-- udf(min(a)) +-- FOR course IN ('dotNET', 'Java') +--); + +-- Complex type is not supported in the current UDF. Skipped for now. +-- pivot on multiple pivot columns with agg columns of complex data types +-- SELECT * FROM ( +-- SELECT course, year, y, a +-- FROM courseSales +-- JOIN yearsWithComplexTypes ON year = y +--) +--PIVOT ( +-- udf(max(a)) +-- FOR (y, course) IN ((2012, 'dotNET'), (2013, 'Java')) +--); + +-- pivot on pivot column of array type +SELECT * FROM ( + SELECT earnings, year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR a IN (array(1, 1), array(2, 2)) +); + +-- pivot on multiple pivot columns containing array type +SELECT * FROM ( + SELECT course, earnings, udf(year) as year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2))) +); + +-- pivot on pivot column of struct type +SELECT * FROM ( + SELECT earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR s IN ((1, 'a'), (2, 'b')) +); + +-- pivot on multiple pivot columns containing struct type +SELECT * FROM ( + SELECT course, earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b'))) +); + +-- pivot on pivot column of map type +SELECT * FROM ( + SELECT earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR m IN (map('1', 1), map('2', 2)) +); + +-- pivot on multiple pivot columns containing map type +SELECT * FROM ( + SELECT course, earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, m) IN (('dotNET', map('1', 1)), ('Java', map('2', 2))) +); + +-- grouping columns output in the same order as input +-- correctly handle pivot columns with different cases +SELECT * FROM ( + SELECT course, earnings, udf("a") as a, udf("z") as z, udf("b") as b, udf("y") as y, + udf("c") as c, udf("x") as x, udf("d") as d, udf("w") as w + FROM courseSales +) +PIVOT ( + udf(sum(Earnings)) + FOR Course IN ('dotNET', 'Java') +); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-special-values.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-special-values.sql new file mode 100644 index 000000000000..9cd15369bb16 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-special-values.sql @@ -0,0 +1,8 @@ +-- This file tests special values such as NaN, Infinity and NULL. + +SELECT udf(x) FROM (VALUES (1), (2), (NULL)) v(x); +SELECT udf(x) FROM (VALUES ('A'), ('B'), (NULL)) v(x); +SELECT udf(x) FROM (VALUES ('NaN'), ('1'), ('2')) v(x); +SELECT udf(x) FROM (VALUES ('Infinity'), ('1'), ('2')) v(x); +SELECT udf(x) FROM (VALUES ('-Infinity'), ('1'), ('2')) v(x); +SELECT udf(x) FROM (VALUES 0.00000001, 0.00000002, 0.00000003) v(x); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-udaf.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-udaf.sql new file mode 100644 index 000000000000..c22d552a0ebe --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-udaf.sql @@ -0,0 +1,18 @@ + -- This test file was converted from udaf.sql. + +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(1), (2), (3), (4) +as t1(int_col1); + +CREATE FUNCTION myDoubleAvg AS 'test.org.apache.spark.sql.MyDoubleAvg'; + +SELECT default.myDoubleAvg(udf(int_col1)) as my_avg, udf(default.myDoubleAvg(udf(int_col1))) as my_avg2, udf(default.myDoubleAvg(int_col1)) as my_avg3 from t1; + +SELECT default.myDoubleAvg(udf(int_col1), udf(3)) as my_avg from t1; + +CREATE FUNCTION udaf1 AS 'test.non.existent.udaf'; + +SELECT default.udaf1(udf(int_col1)) as udaf1, udf(default.udaf1(udf(int_col1))) as udaf2, udf(default.udaf1(int_col1)) as udaf3 from t1; + +DROP FUNCTION myDoubleAvg; +DROP FUNCTION udaf1; \ No newline at end of file diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-union.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-union.sql new file mode 100644 index 000000000000..207bf557acb0 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-union.sql @@ -0,0 +1,56 @@ +-- This test file was converted from union.sql. + +CREATE OR REPLACE TEMPORARY VIEW t1 AS VALUES (1, 'a'), (2, 'b') tbl(c1, c2); +CREATE OR REPLACE TEMPORARY VIEW t2 AS VALUES (1.0, 1), (2.0, 4) tbl(c1, c2); + +-- Simple Union +SELECT udf(c1) as c1, udf(c2) as c2 +FROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 + UNION ALL + SELECT udf(c1) as c1, udf(c2) as c2 FROM t1); + +-- Type Coerced Union +SELECT udf(c1) as c1, udf(c2) as c2 +FROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 WHERE c2 = 'a' + UNION ALL + SELECT udf(c1) as c1, udf(c2) as c2 FROM t2 + UNION ALL + SELECT udf(c1) as c1, udf(c2) as c2 FROM t2); + +-- Regression test for SPARK-18622 +SELECT udf(udf(a)) as a +FROM (SELECT udf(0) a, udf(0) b + UNION ALL + SELECT udf(SUM(1)) a, udf(CAST(0 AS BIGINT)) b + UNION ALL SELECT udf(0) a, udf(0) b) T; + +-- Regression test for SPARK-18841 Push project through union should not be broken by redundant alias removal. +CREATE OR REPLACE TEMPORARY VIEW p1 AS VALUES 1 T(col); +CREATE OR REPLACE TEMPORARY VIEW p2 AS VALUES 1 T(col); +CREATE OR REPLACE TEMPORARY VIEW p3 AS VALUES 1 T(col); +SELECT udf(1) AS x, + udf(col) as col +FROM (SELECT udf(col) AS col + FROM (SELECT udf(p1.col) AS col + FROM p1 CROSS JOIN p2 + UNION ALL + SELECT udf(col) + FROM p3) T1) T2; + +-- SPARK-24012 Union of map and other compatible columns. +SELECT map(1, 2), udf('str') as str +UNION ALL +SELECT map(1, 2, 3, NULL), udf(1); + +-- SPARK-24012 Union of array and other compatible columns. +SELECT array(1, 2), udf('str') as str +UNION ALL +SELECT array(1, 2, 3, NULL), udf(1); + + +-- Clean-up +DROP VIEW IF EXISTS t1; +DROP VIEW IF EXISTS t2; +DROP VIEW IF EXISTS p1; +DROP VIEW IF EXISTS p2; +DROP VIEW IF EXISTS p3; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-window.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-window.sql new file mode 100644 index 000000000000..6cf89fbdb40f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udf/udf-window.sql @@ -0,0 +1,124 @@ +--This test file was converted from window.sql. +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), +(null, null, null, null, null, null), +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) +AS testData(val, val_long, val_double, val_date, val_timestamp, cate); + +-- RowsBetween +SELECT udf(val), cate, count(val) OVER(PARTITION BY cate ORDER BY udf(val) ROWS CURRENT ROW) FROM testData +ORDER BY cate, udf(val); +SELECT udf(val), cate, sum(val) OVER(PARTITION BY cate ORDER BY udf(val) +ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val); +SELECT val_long, udf(cate), sum(val_long) OVER(PARTITION BY cate ORDER BY udf(val_long) +ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY udf(cate), val_long; + +-- RangeBetween +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY val RANGE 1 PRECEDING) FROM testData +ORDER BY cate, udf(val); +SELECT val, udf(cate), sum(val) OVER(PARTITION BY udf(cate) ORDER BY val +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY udf(cate), val; +SELECT val_long, udf(cate), sum(val_long) OVER(PARTITION BY udf(cate) ORDER BY val_long +RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY udf(cate), val_long; +SELECT val_double, udf(cate), sum(val_double) OVER(PARTITION BY udf(cate) ORDER BY val_double +RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING) FROM testData ORDER BY udf(cate), val_double; +SELECT val_date, udf(cate), max(val_date) OVER(PARTITION BY udf(cate) ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING) FROM testData ORDER BY udf(cate), val_date; +SELECT val_timestamp, udf(cate), avg(val_timestamp) OVER(PARTITION BY udf(cate) ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData +ORDER BY udf(cate), val_timestamp; + +-- RangeBetween with reverse OrderBy +SELECT val, udf(cate), sum(val) OVER(PARTITION BY cate ORDER BY val DESC +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; + +-- Invalid window frame +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) +ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val); +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val); +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY udf(val), cate +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val); +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY current_timestamp +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val); +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY val +RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY udf(cate), val; +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY udf(val) +RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val(val); + + +-- Window functions +SELECT udf(val), cate, +max(udf(val)) OVER w AS max, +min(udf(val)) OVER w AS min, +min(udf(val)) OVER w AS min, +count(udf(val)) OVER w AS count, +sum(udf(val)) OVER w AS sum, +avg(udf(val)) OVER w AS avg, +stddev(udf(val)) OVER w AS stddev, +first_value(udf(val)) OVER w AS first_value, +first_value(udf(val), true) OVER w AS first_value_ignore_null, +first_value(udf(val), false) OVER w AS first_value_contain_null, +any_value(udf(val)) OVER w AS any_value, +any_value(udf(val), true) OVER w AS any_value_ignore_null, +any_value(udf(val), false) OVER w AS any_value_contain_null, +last_value(udf(val)) OVER w AS last_value, +last_value(udf(val), true) OVER w AS last_value_ignore_null, +last_value(udf(val), false) OVER w AS last_value_contain_null, +rank() OVER w AS rank, +dense_rank() OVER w AS dense_rank, +cume_dist() OVER w AS cume_dist, +percent_rank() OVER w AS percent_rank, +ntile(2) OVER w AS ntile, +row_number() OVER w AS row_number, +var_pop(udf(val)) OVER w AS var_pop, +var_samp(udf(val)) OVER w AS var_samp, +approx_count_distinct(udf(val)) OVER w AS approx_count_distinct, +covar_pop(udf(val), udf(val_long)) OVER w AS covar_pop, +corr(udf(val), udf(val_long)) OVER w AS corr, +stddev_samp(udf(val)) OVER w AS stddev_samp, +stddev_pop(udf(val)) OVER w AS stddev_pop, +collect_list(udf(val)) OVER w AS collect_list, +collect_set(udf(val)) OVER w AS collect_set, +skewness(udf(val_double)) OVER w AS skewness, +kurtosis(udf(val_double)) OVER w AS kurtosis +FROM testData +WINDOW w AS (PARTITION BY udf(cate) ORDER BY udf(val)) +ORDER BY cate, udf(val); + +-- Null inputs +SELECT udf(val), cate, avg(null) OVER(PARTITION BY cate ORDER BY val) FROM testData ORDER BY cate, val; + +-- OrderBy not specified +SELECT udf(val), cate, row_number() OVER(PARTITION BY cate) FROM testData ORDER BY cate, udf(val); + +-- Over clause is empty +SELECT udf(val), cate, sum(val) OVER(), avg(val) OVER() FROM testData ORDER BY cate, val; + +-- first_value()/last_value()/any_value() over () +SELECT udf(val), cate, +first_value(false) OVER w AS first_value, +first_value(true, true) OVER w AS first_value_ignore_null, +first_value(false, false) OVER w AS first_value_contain_null, +any_value(false) OVER w AS any_value, +any_value(true, true) OVER w AS any_value_ignore_null, +any_value(false, false) OVER w AS any_value_contain_null, +last_value(false) OVER w AS last_value, +last_value(true, true) OVER w AS last_value_ignore_null, +last_value(false, false) OVER w AS last_value_contain_null +FROM testData +WINDOW w AS () +ORDER BY cate, val; + +-- parentheses around window reference +SELECT udf(cate), sum(val) OVER (w) +FROM testData +WHERE val is not null +WINDOW w AS (PARTITION BY cate ORDER BY val); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udtf/udtf.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udtf/udtf.sql new file mode 100644 index 000000000000..a4f598618ab6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/udtf/udtf.sql @@ -0,0 +1,171 @@ +DROP VIEW IF EXISTS t1; +DROP VIEW IF EXISTS t2; +CREATE OR REPLACE TEMPORARY VIEW t1 AS VALUES (0, 1), (1, 2) t(c1, c2); +CREATE OR REPLACE TEMPORARY VIEW t2 AS VALUES (0, 1), (1, 2), (1, 3) t(partition_col, input); + +-- test basic udtf +SELECT * FROM udtf(1, 2); +SELECT * FROM udtf(-1, 0); +SELECT * FROM udtf(0, -1); +SELECT * FROM udtf(0, 0); + +-- test column alias +SELECT a, b FROM udtf(1, 2) t(a, b); + +-- test lateral join +SELECT * FROM t1, LATERAL udtf(c1, c2); +SELECT * FROM t1 LEFT JOIN LATERAL udtf(c1, c2); +SELECT * FROM udtf(1, 2) t(c1, c2), LATERAL udtf(c1, c2); + +-- test non-deterministic input +SELECT * FROM udtf(cast(rand(0) AS int) + 1, 1); + +-- test UDTF calls that take input TABLE arguments +-- As a reminder, the UDTFCountSumLast function returns this analyze result: +-- AnalyzeResult( +-- schema=StructType() +-- .add("count", IntegerType()) +-- .add("total", IntegerType()) +-- .add("last", IntegerType())) +SELECT * FROM UDTFCountSumLast(TABLE(t2) WITH SINGLE PARTITION); +SELECT * FROM UDTFCountSumLast(TABLE(t2) PARTITION BY partition_col ORDER BY input); +SELECT * FROM UDTFCountSumLast(TABLE(t2) PARTITION BY partition_col ORDER BY input DESC); +SELECT * FROM + VALUES (0), (1) AS t(col) + JOIN LATERAL + UDTFCountSumLast(TABLE(t2) PARTITION BY partition_col ORDER BY input DESC); + +-- test UDTF calls that take input TABLE arguments and the 'analyze' method returns required +-- partitioning and/or ordering properties for Catalyst to enforce for the input table +-- As a reminder, the UDTFWithSinglePartition function returns this analyze result: +-- AnalyzeResult( +-- schema=StructType() +-- .add("count", IntegerType()) +-- .add("total", IntegerType()) +-- .add("last", IntegerType()), +-- with_single_partition=True, +-- order_by=[ +-- OrderingColumn("input"), +-- OrderingColumn("partition_col")]) +SELECT * FROM UDTFWithSinglePartition(0, TABLE(t2)); +SELECT * FROM UDTFWithSinglePartition(1, TABLE(t2)); +SELECT * FROM UDTFWithSinglePartition(0, TABLE(t2) WITH SINGLE PARTITION); +SELECT * FROM UDTFWithSinglePartition(0, TABLE(t2) PARTITION BY partition_col); +SELECT * FROM + VALUES (0), (1) AS t(col) + JOIN LATERAL + UDTFWithSinglePartition(0, TABLE(t2) PARTITION BY partition_col); +-- As a reminder, the UDTFPartitionByOrderBy function returns this analyze result: +-- AnalyzeResult( +-- schema=StructType() +-- .add("partition_col", IntegerType()) +-- .add("count", IntegerType()) +-- .add("total", IntegerType()) +-- .add("last", IntegerType()), +-- partition_by=[ +-- PartitioningColumn("partition_col") +-- ], +-- order_by=[ +-- OrderingColumn("input") +-- ]) +SELECT * FROM UDTFPartitionByOrderBy(TABLE(t2)); +SELECT * FROM UDTFPartitionByOrderBy(TABLE(t2) WITH SINGLE PARTITION); +SELECT * FROM UDTFPartitionByOrderBy(TABLE(t2) PARTITION BY partition_col); +SELECT * FROM + VALUES (0), (1) AS t(col) + JOIN LATERAL + UDTFPartitionByOrderBy(TABLE(t2) PARTITION BY partition_col); +SELECT * FROM UDTFPartitionByOrderByComplexExpr(TABLE(t2)); +SELECT * FROM UDTFPartitionByOrderBySelectExpr(TABLE(t2)); +SELECT * FROM UDTFPartitionByOrderBySelectComplexExpr(TABLE(t2)); +SELECT * FROM UDTFPartitionByOrderBySelectExprOnlyPartitionColumn(TABLE(t2)); +SELECT * FROM UDTFInvalidSelectExprParseError(TABLE(t2)); +SELECT * FROM UDTFInvalidSelectExprStringValue(TABLE(t2)); +SELECT * FROM UDTFInvalidComplexSelectExprMissingAlias(TABLE(t2)); +SELECT * FROM UDTFInvalidOrderByAscKeyword(TABLE(t2)); +SELECT * FROM UDTFInvalidOrderByStringList(TABLE(t2)); +-- As a reminder, UDTFInvalidPartitionByAndWithSinglePartition returns this analyze result: +-- AnalyzeResult( +-- schema=StructType() +-- .add("last", IntegerType()), +-- with_single_partition=True, +-- partition_by=[ +-- PartitioningColumn("partition_col") +-- ]) +SELECT * FROM UDTFInvalidPartitionByAndWithSinglePartition(TABLE(t2)); +SELECT * FROM UDTFInvalidPartitionByAndWithSinglePartition(TABLE(t2) WITH SINGLE PARTITION); +SELECT * FROM UDTFInvalidPartitionByAndWithSinglePartition(TABLE(t2) PARTITION BY partition_col); +SELECT * FROM + VALUES (0), (1) AS t(col) + JOIN LATERAL + UDTFInvalidPartitionByAndWithSinglePartition(TABLE(t2) PARTITION BY partition_col); +-- As a reminder, UDTFInvalidOrderByWithoutPartitionBy function returns this analyze result: +-- AnalyzeResult( +-- schema=StructType() +-- .add("last", IntegerType()), +-- order_by=[ +-- OrderingColumn("input") +-- ]) +SELECT * FROM UDTFInvalidOrderByWithoutPartitionBy(TABLE(t2)); +SELECT * FROM UDTFInvalidOrderByWithoutPartitionBy(TABLE(t2) WITH SINGLE PARTITION); +SELECT * FROM UDTFInvalidOrderByWithoutPartitionBy(TABLE(t2) PARTITION BY partition_col); +SELECT * FROM + VALUES (0), (1) AS t(col) + JOIN LATERAL + UDTFInvalidOrderByWithoutPartitionBy(TABLE(t2) PARTITION BY partition_col); +-- The following UDTF calls should fail because the UDTF's 'eval' or 'terminate' method returns None +-- to a non-nullable column, either directly or within an array/struct/map subfield. +SELECT * FROM InvalidEvalReturnsNoneToNonNullableColumnScalarType(TABLE(t2)); +SELECT * FROM InvalidEvalReturnsNoneToNonNullableColumnArrayType(TABLE(t2)); +SELECT * FROM InvalidEvalReturnsNoneToNonNullableColumnArrayElementType(TABLE(t2)); +SELECT * FROM InvalidEvalReturnsNoneToNonNullableColumnStructType(TABLE(t2)); +SELECT * FROM InvalidEvalReturnsNoneToNonNullableColumnMapType(TABLE(t2)); +SELECT * FROM InvalidTerminateReturnsNoneToNonNullableColumnScalarType(TABLE(t2)); +SELECT * FROM InvalidTerminateReturnsNoneToNonNullableColumnArrayType(TABLE(t2)); +SELECT * FROM InvalidTerminateReturnsNoneToNonNullableColumnArrayElementType(TABLE(t2)); +SELECT * FROM InvalidTerminateReturnsNoneToNonNullableColumnStructType(TABLE(t2)); +SELECT * FROM InvalidTerminateReturnsNoneToNonNullableColumnMapType(TABLE(t2)); +-- The following UDTF calls exercise various invalid function definitions and calls to show the +-- error messages. +SELECT * FROM UDTFForwardStateFromAnalyzeWithKwargs(); +SELECT * FROM UDTFForwardStateFromAnalyzeWithKwargs(1, 2); +SELECT * FROM UDTFForwardStateFromAnalyzeWithKwargs(invalid => 2); +SELECT * FROM UDTFForwardStateFromAnalyzeWithKwargs(argument => 1, argument => 2); +SELECT * FROM InvalidAnalyzeMethodWithSinglePartitionNoInputTable(argument => 1); +SELECT * FROM InvalidAnalyzeMethodWithPartitionByNoInputTable(argument => 1); +SELECT * FROM InvalidAnalyzeMethodReturnsNonStructTypeSchema(TABLE(t2)); +SELECT * FROM InvalidAnalyzeMethodWithPartitionByListOfStrings(argument => TABLE(t2)); +SELECT * FROM InvalidForwardStateFromAnalyzeTooManyInitArgs(TABLE(t2)); +SELECT * FROM InvalidNotForwardStateFromAnalyzeTooManyInitArgs(TABLE(t2)); +SELECT * FROM UDTFWithSinglePartition(1); +SELECT * FROM UDTFWithSinglePartition(1, 2, 3); +SELECT * FROM UDTFWithSinglePartition(1, invalid_arg_name => 2); +SELECT * FROM UDTFWithSinglePartition(1, initial_count => 2); +SELECT * FROM UDTFWithSinglePartition(initial_count => 1, initial_count => 2); +SELECT * FROM UDTFInvalidPartitionByOrderByParseError(TABLE(t2)); +-- Exercise the UDTF partitioning bug. +SELECT * FROM UDTFPartitionByIndexingBug( + TABLE( + SELECT + 5 AS unused_col, + 'hi' AS partition_col, + 1.0 AS double_col + + UNION ALL + + SELECT + 4 AS unused_col, + 'hi' AS partition_col, + 1.0 AS double_col + ) +); +-- Exercise a query with both a valid TABLE argument and an invalid unresolved column reference. +-- The 'eval' method of this UDTF would later throw an exception, but that is not relevant here +-- because the analysis of this query should fail before that point. We just want to make sure that +-- this analysis failure returns a reasonable error message. +SELECT * FROM + InvalidEvalReturnsNoneToNonNullableColumnScalarType(TABLE(SELECT 1 AS X), unresolved_column); + +-- cleanup +DROP VIEW t1; +DROP VIEW t2; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/union.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/union.sql new file mode 100644 index 000000000000..ab81cc7bbabb --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/union.sql @@ -0,0 +1,68 @@ +CREATE OR REPLACE TEMPORARY VIEW t1 AS VALUES (1, 'a'), (2, 'b') tbl(c1, c2); +CREATE OR REPLACE TEMPORARY VIEW t2 AS VALUES (1.0, 1), (2.0, 4) tbl(c1, c2); + +-- Simple Union +SELECT * +FROM (SELECT * FROM t1 + UNION ALL + SELECT * FROM t1); + +-- Type Coerced Union +SELECT * +FROM (SELECT * FROM t1 where c1 = 1 + UNION ALL + SELECT * FROM t2 + UNION ALL + SELECT * FROM t2); + +-- Regression test for SPARK-18622 +SELECT a +FROM (SELECT 0 a, 0 b + UNION ALL + SELECT SUM(1) a, CAST(0 AS BIGINT) b + UNION ALL SELECT 0 a, 0 b) T; + +-- Regression test for SPARK-18841 Push project through union should not be broken by redundant alias removal. +CREATE OR REPLACE TEMPORARY VIEW p1 AS VALUES 1 T(col); +CREATE OR REPLACE TEMPORARY VIEW p2 AS VALUES 1 T(col); +CREATE OR REPLACE TEMPORARY VIEW p3 AS VALUES 1 T(col); +SELECT 1 AS x, + col +FROM (SELECT col AS col + FROM (SELECT p1.col AS col + FROM p1 CROSS JOIN p2 + UNION ALL + SELECT col + FROM p3) T1) T2; + +-- SPARK-24012 Union of map and other compatible columns. +SELECT map(1, 2), 'str' +UNION ALL +SELECT map(1, 2, 3, NULL), 1; + +-- SPARK-24012 Union of array and other compatible columns. +SELECT array(1, 2), 'str' +UNION ALL +SELECT array(1, 2, 3, NULL), 1; + +-- SPARK-32638: corrects references when adding aliases in WidenSetOperationTypes +CREATE OR REPLACE TEMPORARY VIEW t3 AS VALUES (decimal(1)) tbl(v); +SELECT t.v FROM ( + SELECT v FROM t3 + UNION ALL + SELECT v + v AS v FROM t3 +) t; + +SELECT SUM(t.v) FROM ( + SELECT v FROM t3 + UNION + SELECT v + v AS v FROM t3 +) t; + +-- Clean-up +DROP VIEW IF EXISTS t1; +DROP VIEW IF EXISTS t2; +DROP VIEW IF EXISTS t3; +DROP VIEW IF EXISTS p1; +DROP VIEW IF EXISTS p2; +DROP VIEW IF EXISTS p3; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/unpivot.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/unpivot.sql new file mode 100644 index 000000000000..08a46a64d165 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/unpivot.sql @@ -0,0 +1,44 @@ +create temporary view courseEarnings as select * from values + ("dotNET", 15000, 48000, 22500), + ("Java", 20000, 30000, NULL) + as courseEarnings(course, `2012`, `2013`, `2014`); + +SELECT * FROM courseEarnings +UNPIVOT ( + earningsYear FOR year IN (`2012`, `2013`, `2014`) +); + +-- NULL values excluded by default, include them explicitly +SELECT * FROM courseEarnings +UNPIVOT INCLUDE NULLS ( + earningsYear FOR year IN (`2012`, `2013`, `2014`) +); + +-- alias for column names +SELECT * FROM courseEarnings +UNPIVOT ( + earningsYear FOR year IN (`2012` as `twenty-twelve`, `2013` as `twenty-thirteen`, `2014` as `twenty-fourteen`) +); + + +create temporary view courseEarningsAndSales as select * from values + ("dotNET", 15000, NULL, 48000, 1, 22500, 1), + ("Java", 20000, 1, 30000, 2, NULL, NULL) + as courseEarningsAndSales(course, earnings2012, sales2012, earnings2013, sales2013, earnings2014, sales2014); + +SELECT * FROM courseEarningsAndSales +UNPIVOT ( + (earnings, sales) FOR year IN ((earnings2012, sales2012), (earnings2013, sales2013), (earnings2014, sales2014)) +); + +-- NULL values excluded by default, include them explicitly +SELECT * FROM courseEarningsAndSales +UNPIVOT INCLUDE NULLS ( + (earnings, sales) FOR year IN ((earnings2012, sales2012), (earnings2013, sales2013), (earnings2014, sales2014)) +); + +-- alias for column names +SELECT * FROM courseEarningsAndSales +UNPIVOT ( + (earnings, sales) FOR year IN ((earnings2012, sales2012) as `2012`, (earnings2013, sales2013) as `2013`, (earnings2014, sales2014) as `2014`) +); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/url-functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/url-functions.sql new file mode 100644 index 000000000000..222473feffe1 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/url-functions.sql @@ -0,0 +1,26 @@ +-- parse_url function +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'HOST'); +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'PATH'); +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'QUERY'); +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'REF'); +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'PROTOCOL'); +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'FILE'); +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'AUTHORITY'); +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'USERINFO'); + +-- url_encode function +select url_encode('https://spark.apache.org'); +select url_encode('inva lid://user:pass@host/file\\;param?query\\;p2'); +select url_encode(null); + +-- url_decode function +select url_decode('https%3A%2F%2Fspark.apache.org'); +select url_decode('http%3A%2F%2spark.apache.org'); +select url_decode('inva lid://user:pass@host/file\\;param?query\\;p2'); +select url_decode(null); + +-- try_url_decode function +select try_url_decode('https%3A%2F%2Fspark.apache.org'); +select try_url_decode('http%3A%2F%2spark.apache.org'); +select try_url_decode('inva lid://user:pass@host/file\\;param?query\\;p2'); +select try_url_decode(null); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/using-join.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/using-join.sql new file mode 100644 index 000000000000..f2657a91910b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/using-join.sql @@ -0,0 +1,95 @@ +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1); + +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5), + ("four", 4) + as nt2(k, v2); + +SELECT * FROM nt1 left outer join nt2 using (k); + +SELECT k FROM nt1 left outer join nt2 using (k); + +SELECT nt1.*, nt2.* FROM nt1 left outer join nt2 using (k); + +SELECT nt1.k, nt2.k FROM nt1 left outer join nt2 using (k); + +SELECT k FROM (SELECT nt2.k FROM nt1 left outer join nt2 using (k)); + +SELECT nt2.k AS key FROM nt1 left outer join nt2 using (k) ORDER BY key; + +SELECT nt1.k, nt2.k FROM nt1 left outer join nt2 using (k) ORDER BY nt2.k; + +SELECT k, nt1.k FROM nt1 left outer join nt2 using (k); + +SELECT k, nt2.k FROM nt1 left outer join nt2 using (k); + +SELECT * FROM nt1 left semi join nt2 using (k); + +SELECT k FROM nt1 left semi join nt2 using (k); + +SELECT nt1.* FROM nt1 left semi join nt2 using (k); + +SELECT nt1.k FROM nt1 left semi join nt2 using (k); + +SELECT k, nt1.k FROM nt1 left semi join nt2 using (k); + +SELECT * FROM nt1 right outer join nt2 using (k); + +SELECT k FROM nt1 right outer join nt2 using (k); + +SELECT nt1.*, nt2.* FROM nt1 right outer join nt2 using (k); + +SELECT nt1.k, nt2.k FROM nt1 right outer join nt2 using (k); + +SELECT k FROM (SELECT nt1.k FROM nt1 right outer join nt2 using (k)); + +SELECT nt1.k AS key FROM nt1 right outer join nt2 using (k) ORDER BY key; + +SELECT k, nt1.k FROM nt1 right outer join nt2 using (k); + +SELECT k, nt2.k FROM nt1 right outer join nt2 using (k); + +SELECT * FROM nt1 full outer join nt2 using (k); + +SELECT k FROM nt1 full outer join nt2 using (k); + +SELECT nt1.*, nt2.* FROM nt1 full outer join nt2 using (k); + +SELECT nt1.k, nt2.k FROM nt1 full outer join nt2 using (k); + +SELECT k FROM (SELECT nt2.k FROM nt1 full outer join nt2 using (k)); + +SELECT nt2.k AS key FROM nt1 full outer join nt2 using (k) ORDER BY key; + +SELECT k, nt1.k FROM nt1 full outer join nt2 using (k); + +SELECT k, nt2.k FROM nt1 full outer join nt2 using (k); + +SELECT * FROM nt1 full outer join nt2 using (k); + +SELECT k FROM nt1 inner join nt2 using (k); + +SELECT nt1.*, nt2.* FROM nt1 inner join nt2 using (k); + +SELECT nt1.k, nt2.k FROM nt1 inner join nt2 using (k); + +SELECT k FROM (SELECT nt2.k FROM nt1 inner join nt2 using (k)); + +SELECT nt2.k AS key FROM nt1 inner join nt2 using (k) ORDER BY key; + +SELECT k, nt1.k FROM nt1 inner join nt2 using (k); + +SELECT k, nt2.k FROM nt1 inner join nt2 using (k); + +WITH + t1 AS (select key from values ('a') t(key)), + t2 AS (select key from values ('a') t(key)) +SELECT t1.key +FROM t1 FULL OUTER JOIN t2 USING (key) +WHERE t1.key NOT LIKE 'bb.%'; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/variant/named-function-arguments.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/variant/named-function-arguments.sql new file mode 100644 index 000000000000..e63e68169a81 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/variant/named-function-arguments.sql @@ -0,0 +1,6 @@ +-- Test for tabled value functions variant_explode and variant_explode_outer +SELECT * FROM variant_explode(input => parse_json('["hello", "world"]')); +SELECT * FROM variant_explode_outer(input => parse_json('{"a": true, "b": 3.14}')); +SELECT * FROM variant_explode(parse_json('["hello", "world"]')), variant_explode(parse_json('{"a": true, "b": 3.14}')); +SELECT * FROM variant_explode(parse_json('{"a": ["hello", "world"], "b": {"x": true, "y": 3.14}}')) AS t, LATERAL variant_explode(t.value); +SELECT num, key, val, 'Spark' FROM variant_explode(parse_json('["hello", "world"]')) AS t(num, key, val); diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/view-schema-binding-config.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/view-schema-binding-config.sql new file mode 100644 index 000000000000..e803254ea642 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/view-schema-binding-config.sql @@ -0,0 +1,172 @@ +-- This test suits check the spark.sql.viewSchemaBindingMode configuration. +-- It can be DISABLED and COMPENSATION + +-- Verify the default binding is true +SET spark.sql.legacy.viewSchemaBindingMode; + +-- 1. Test DISABLED mode. +SET spark.sql.legacy.viewSchemaBindingMode = false; + +-- 1.a Attempts to use the SCHEMA BINDING clause fail with FEATURE_NOT_ENABLED +CREATE OR REPLACE VIEW v WITH SCHEMA BINDING AS SELECT 1; +CREATE OR REPLACE VIEW v WITH SCHEMA COMPENSATION AS SELECT 1; +CREATE OR REPLACE VIEW v WITH SCHEMA TYPE EVOLUTION AS SELECT 1; +CREATE OR REPLACE VIEW v WITH SCHEMA EVOLUTION AS SELECT 1; + +-- 1.b Existing SHOW and DESCRIBE should behave as before Spark 4.0.0 +CREATE OR REPLACE VIEW v AS SELECT 1; +DESCRIBE EXTENDED v; +SHOW TABLE EXTENDED LIKE 'v'; +SHOW CREATE TABLE v; +DROP VIEW IF EXISTS v; + +CREATE OR REPLACE TEMPORARY VIEW v AS SELECT 1; +DESCRIBE EXTENDED v; +SHOW TABLE EXTENDED LIKE 'v'; +DROP VIEW IF EXISTS v; + +-- 1.c Views get invalidated if the types change in an unsafe matter +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT NOT NULL) USING PARQUET; +CREATE OR REPLACE VIEW v AS SELECT * FROM t; +SELECT * FROM v; +-- Baseline: v(c1 INT); +DESCRIBE EXTENDED v; +SHOW CREATE TABLE v; + +-- Widen the column c1 in t +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 BIGINT NOT NULL) USING PARQUET; +-- The view should be invalidated, cannot upcast from BIGINT to INT +SELECT * FROM v; + +-- The view still describes as v(c1 INT); +DESCRIBE EXTENDED v; + +-- 2. Test true mode. In this mode Spark tolerates any supported CAST, not just up cast +SET spark.sql.legacy.viewSchemaBindingMode = true; +SET spark.sql.legacy.viewSchemaCompensation = false; + +-- To verify ANSI_MODE is enforced even if ANSI_MODE is turned off. +SET spark.sql.ansi.enabled = false; + +-- 2.a In BINDING views get invalidated if the type can't cast +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT NOT NULL) USING PARQUET; +CREATE OR REPLACE VIEW v AS SELECT * FROM t; +SELECT * FROM v; +-- Baseline: v(c1 INT); +DESCRIBE EXTENDED v; +SHOW CREATE TABLE v; + +-- Widen the column c1 in t +DROP TABLE t; +CREATE TABLE t(c1 BIGINT NOT NULL) USING PARQUET; +INSERT INTO t VALUES (1); + +-- This fails +SELECT * FROM v; +-- The view still describes as v(c1 BIGINT) +DESCRIBE EXTENDED v; +SHOW CREATE TABLE v; + +-- 2.b Switch to default COMPENSATION +SET spark.sql.legacy.viewSchemaCompensation = true; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT NOT NULL) USING PARQUET; +CREATE OR REPLACE VIEW v AS SELECT * FROM t; +SELECT * FROM v; +-- Baseline: v(c1 INT); +DESCRIBE EXTENDED v; +SHOW CREATE TABLE v; + +-- Widen the column c1 in t +DROP TABLE t; +CREATE TABLE t(c1 BIGINT NOT NULL) USING PARQUET; +INSERT INTO t VALUES (1); + +-- This now succeeds +SELECT * FROM v; +-- The view still describes as v(c1 BIGINT) +DESCRIBE EXTENDED v; +SHOW CREATE TABLE v; + +-- 2.c In COMPENSATION views ignore added columns and change the type +-- Expect the added column to be ignore, but the type will be tolerated, as long as it can cast +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 STRING NOT NULL, c2 INT) USING PARQUET; +INSERT INTO t VALUES ('1', 2); +SELECT * FROM v; +-- The view still describes as v(c1 INT); +DESCRIBE EXTENDED v; + +-- Runtime error if the cast fails +INSERT INTO t VALUES ('a', 2); +SELECT * FROM v; + +-- Compile time error if the cast can't be done +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 MAP, c2 INT) USING PARQUET; + +-- The view should be invalidated, we can't cast a MAP to INT +SELECT * FROM v; + +-- The view still describes as v(c1 INT); +DESCRIBE EXTENDED v; + +-- 2.d Still can't drop a column, though +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT, c2 INT) USING PARQUET; +INSERT INTO t VALUES (1, 2); +CREATE OR REPLACE VIEW v AS SELECT * FROM t; +SELECT * FROM v; + +-- Describes as v(c1 INT, c2 INT) +DESCRIBE EXTENDED v; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT NOT NULL) USING PARQUET; + +-- The view should be invalidated, it lost a column +SELECT * FROM v; +DESCRIBE EXTENDED v; + +-- 2.e Attempt to rename a column +DROP TABLE IF EXISTS t; +CREATE TABLE t(c3 INT NOT NULL, c2 INT) USING PARQUET; +SELECT * FROM v; +DESCRIBE EXTENDED v; + +-- 3. Test the behavior of grandfathered views and temp views +SET spark.sql.legacy.viewSchemaBindingMode = false; +SET spark.sql.legacy.viewSchemaCompensation = false; +CREATE OR REPLACE VIEW v AS SELECT 1; +SET spark.sql.legacy.viewSchemaBindingMode = true; +DESCRIBE EXTENDED v; +SHOW TABLE EXTENDED LIKE 'v'; +SHOW CREATE TABLE v; + +SET spark.sql.legacy.viewSchemaCompensation = true; +DESCRIBE EXTENDED v; +SHOW TABLE EXTENDED LIKE 'v'; +SHOW CREATE TABLE v; + +DROP VIEW IF EXISTS v; + +SET spark.sql.legacy.viewSchemaBindingMode = false; +SET spark.sql.legacy.viewSchemaCompensation = false; +CREATE OR REPLACE TEMPORARY VIEW v AS SELECT 1; +SET spark.sql.legacy.viewSchemaBindingMode = true; +DESCRIBE EXTENDED v; +SHOW TABLE EXTENDED LIKE 'v'; + +SET spark.sql.legacy.viewSchemaCompensation = true; +DESCRIBE EXTENDED v; +SHOW TABLE EXTENDED LIKE 'v'; + +DROP VIEW IF EXISTS v; + +-- 99 Cleanup +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/view-schema-binding.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/view-schema-binding.sql new file mode 100644 index 000000000000..413322db10d2 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/view-schema-binding.sql @@ -0,0 +1,64 @@ +-- This test suite checks that the WITH SCHEMA BINDING clause is correctly implemented + +-- New view with schema binding +-- 1.a BINDING is persisted +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT NOT NULL) USING PARQUET; +CREATE OR REPLACE VIEW v WITH SCHEMA BINDING AS SELECT * FROM t; +SELECT * FROM v; +-- Baseline: v(c1 INT); +DESCRIBE EXTENDED v; + +-- Widen the column c1 in t +DROP TABLE t; +CREATE TABLE t(c1 BIGINT NOT NULL) USING PARQUET; +-- The view should be invalidated, cannot upcast from BIGINT to INT +SELECT * FROM v; + +-- The view still describes as v(c1 INT); +DESCRIBE EXTENDED v; + +-- 1.b In BINDING views get invalidated if a column is lost +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT, c2 INT) USING PARQUET; +CREATE OR REPLACE VIEW v WITH SCHEMA BINDING AS SELECT * FROM t; +SELECT * FROM v; +-- Baseline: v(c1 INT, c2 INT); +DESCRIBE EXTENDED v; + +-- Drop the column c2 from t +DROP TABLE t; +CREATE TABLE t(c1 INT) USING PARQUET; +-- The view should be invalidated, it lost a column +SELECT * FROM v; + +-- The view still describes as v(c1 INT, c2 INT); +DESCRIBE EXTENDED v; + +-- Test ALTER VIEW ... WITH SCHEMA BINDING +SET spark.sql.legacy.viewSchemaCompensation=false; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT NOT NULL) USING PARQUET; +CREATE OR REPLACE VIEW v AS SELECT * FROM t; +SELECT * FROM v; +-- Baseline: v(c1 INT); +-- There is no binding recorded +DESCRIBE EXTENDED v; + +ALTER VIEW v WITH SCHEMA BINDING; +-- Baseline: v(c1 INT); +-- There is SCHEMA BINDING recorded +DESCRIBE EXTENDED v; + +DROP TABLE t; +CREATE TABLE t(c1 BIGINT NOT NULL) USING PARQUET; +-- The view should be invalidated, cannot upcast from BIGINT to INT +SELECT * FROM v; + +-- The view still describes as v(c1 INT); +DESCRIBE EXTENDED v; + +-- 99 Cleanup +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/view-schema-compensation.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/view-schema-compensation.sql new file mode 100644 index 000000000000..21a3ce1e1229 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/view-schema-compensation.sql @@ -0,0 +1,90 @@ +-- This test suite checks the WITH SCHEMA COMPENSATION clause +-- Disable ANSI mode to ensure we are forcing it explicitly in the CASTS +SET spark.sql.ansi.enabled = false; + +-- In COMPENSATION views get invalidated if the type can't cast +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT NOT NULL) USING PARQUET; +CREATE OR REPLACE VIEW v WITH SCHEMA COMPENSATION AS SELECT * FROM t; +SELECT * FROM v; +-- Baseline: v(c1 INT); +DESCRIBE EXTENDED v; + +-- Widen the column c1 in t +DROP TABLE t; +CREATE TABLE t(c1 BIGINT NOT NULL) USING PARQUET; +INSERT INTO t VALUES (1); +SELECT * FROM v; +-- The view still describes as v(c1 BIGINT) +DESCRIBE EXTENDED v; + +-- In COMPENSATION views ignore added a column and change the type +-- Expect the added column to be ignore, but the type will be tolerated, as long as it can cast +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 STRING NOT NULL, c2 INT) USING PARQUET; +INSERT INTO t VALUES ('1', 2); +SELECT * FROM v; +-- The view still describes as v(c1 INT); +DESCRIBE EXTENDED v; + +-- Runtime error if the cast fails +INSERT INTO t VALUES ('a', 2); +SELECT * FROM v; + +-- Compile time error if the cast can't be done +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 MAP, c2 INT) USING PARQUET; + +-- The view should be invalidated, we can't cast a MAP to INT +SELECT * FROM v; + +-- The view still describes as v(c1 INT); +DESCRIBE EXTENDED v; + +-- Still can't drop a column, though +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT, c2 INT) USING PARQUET; +INSERT INTO t VALUES (1, 2); +CREATE OR REPLACE VIEW v AS SELECT * FROM t; +SELECT * FROM v; + +-- Describes as v(c1 INT, c2 INT) +DESCRIBE EXTENDED v; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT NOT NULL) USING PARQUET; + +-- The view should be invalidated, it lost a column +SELECT * FROM v; +DESCRIBE EXTENDED v; + +-- Attempt to rename a column, this fails +DROP TABLE IF EXISTS t; +CREATE TABLE t(c3 INT NOT NULL, c2 INT) USING PARQUET; +SELECT * FROM v; +DESCRIBE EXTENDED v; + +-- Test ALTER VIEW ... WITH SCHEMA ... +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT) USING PARQUET; +INSERT INTO t VALUES(1); +CREATE OR REPLACE VIEW v WITH SCHEMA BINDING AS SELECT * FROM t; +DESCRIBE EXTENDED v; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 STRING) USING PARQUET; +INSERT INTO t VALUES('1'); + +-- This fails, because teh view uses SCHEMA BINDING +SELECT * FROM v; + +-- Now upgrade the view to schema compensation +ALTER VIEW v WITH SCHEMA COMPENSATION; +DESCRIBE EXTENDED v; + +-- Success +SELECT * FROM v; + +-- Cleanup +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/view-schema-evolution.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/view-schema-evolution.sql new file mode 100644 index 000000000000..5ff153acef25 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/view-schema-evolution.sql @@ -0,0 +1,155 @@ +-- This test suite checks the WITH SCHEMA TYPE EVOLUTION clause + +-- In EVOLUTION mode Spark will inherit everything from the query, unless +-- a column list is given. In that case it behaves like TYPE EVOLUTION +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT NOT NULL, c2 INT) USING PARQUET; +INSERT INTO t VALUES (1, 2); +CREATE OR REPLACE VIEW v WITH SCHEMA EVOLUTION AS SELECT * FROM t; +SELECT * FROM v; +DESCRIBE EXTENDED v; + +-- In EVOLUTION views inherit column type changes and name changes +DROP TABLE IF EXISTS t; +CREATE TABLE t(c4 STRING NOT NULL, c5 DOUBLE) USING PARQUET; +INSERT INTO t VALUES ('1', 2.0); +SELECT * FROM v; +-- The view now describes as v(c3 STRING, c4 DOUBLE) +DESCRIBE EXTENDED v; + +-- In EVOLUTION new columns are inherited +DROP TABLE IF EXISTS t; +CREATE TABLE t(c4 STRING, c5 DOUBLE, c6 DATE) USING PARQUET; +INSERT INTO t VALUES ('1', 2.0, DATE'2022-01-01'); +SELECT * FROM v; +-- The view describes as v(c4 STRING, c5 DOUBLE, c6 DATE) +DESCRIBE EXTENDED v; + +-- We can even drop columns +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT, c2 INT) USING PARQUET; +INSERT INTO t VALUES (1, 2); +CREATE OR REPLACE VIEW v WITH SCHEMA EVOLUTION AS SELECT * FROM t; +SELECT * FROM v; +-- Describes as v(c1 INT, c2 INT) +DESCRIBE EXTENDED v; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT) USING PARQUET; +SELECT * FROM v; +-- The view describes as v(c1 INT) +DESCRIBE EXTENDED v; + +-- If a column list is given it behaves like TYPE EVOLUTION +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT NOT NULL, c2 INT) USING PARQUET; +INSERT INTO t VALUES (1, 2); +CREATE OR REPLACE VIEW v(a1, a2) WITH SCHEMA EVOLUTION AS SELECT * FROM t; +SELECT * FROM v; +DESCRIBE EXTENDED v; + +-- In EVOLUTION views with explicit column lists still inherit column type changes +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 STRING NOT NULL, c2 DOUBLE) USING PARQUET; +INSERT INTO t VALUES ('1', 2.0); +SELECT * FROM v; +-- The view now describes as v(a1 STRING, a2 DOUBLE) +DESCRIBE EXTENDED v; + +-- In EVOLUTION views with explicit column lists no new columns are inherited +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 STRING, c2 DOUBLE, c3 DATE) USING PARQUET; +INSERT INTO t VALUES ('1', 2.0, DATE'2022-01-01'); +SELECT * FROM v; +-- The view still describes as v(a1 STRING, a2 DOUBLE) +DESCRIBE EXTENDED v; + +-- In EVOLUTION views with explicit column lists can't drop a column +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT, c2 INT) USING PARQUET; +INSERT INTO t VALUES (1, 2); +CREATE OR REPLACE VIEW v(a1, a2) WITH SCHEMA EVOLUTION AS SELECT * FROM t; +SELECT * FROM v; + +-- Describes as v(a1 INT, a2 INT) +DESCRIBE EXTENDED v; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT) USING PARQUET; + +-- The view should be invalidated, it lost a column +SELECT * FROM v; +DESCRIBE EXTENDED v; + +-- Attempt to rename a column +DROP TABLE IF EXISTS t; +CREATE TABLE t(c3 INT, c2 INT) USING PARQUET; +SELECT * FROM v; +DESCRIBE EXTENDED v; + +-- Test preservation of comments +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT COMMENT 'c1', c2 INT COMMENT 'c2') USING PARQUET; + +-- EVOLUTION, column list, but no comments +CREATE OR REPLACE VIEW v(a1, a2) WITH SCHEMA TYPE EVOLUTION AS SELECT * FROM t; +DESCRIBE EXTENDED v; + +-- EVOLUTION, column list, but no comments +CREATE OR REPLACE VIEW v(a1, a2) WITH SCHEMA EVOLUTION AS SELECT * FROM t; +DESCRIBE EXTENDED v; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 BIGINT COMMENT 'c1 6c', c2 STRING COMMENT 'c2 6c') USING PARQUET; +SELECT * FROM v; +DESCRIBE EXTENDED v; + +-- EVOLUTION, column list with comments +CREATE OR REPLACE VIEW v(a1 COMMENT 'a1', a2 COMMENT 'a2') WITH SCHEMA EVOLUTION AS SELECT * FROM t; +DESCRIBE EXTENDED v; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 BIGINT COMMENT 'c1 6d', c2 STRING COMMENT 'c2 6d') USING PARQUET; +SELECT * FROM v; +DESCRIBE EXTENDED v; + +-- EVOLUTION, no column list +CREATE OR REPLACE VIEW v WITH SCHEMA EVOLUTION AS SELECT * FROM t; +DESCRIBE EXTENDED v; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 BIGINT COMMENT 'c1 6e', c2 STRING COMMENT 'c2 6e') USING PARQUET; +SELECT * FROM v; +DESCRIBE EXTENDED v; + +-- Test error condition where a duplicate column name is produced +DROP TABLE IF EXISTS t1; +CREATE TABLE t1(c1 INT) USING PARQUET; +DROP TABLE IF EXISTS t2; +CREATE TABLE t2(c2 INT) USING PARQUET; +CREATE OR REPLACE VIEW v WITH SCHEMA EVOLUTION AS SELECT * FROM t1, t2; +SELECT * FROM v; +DROP TABLE IF EXISTS t2; +CREATE TABLE t2(c1 INT) USING PARQUET; +-- This should fail with a duplicate column error +SELECT * FROM v; +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +-- Test ALTER VIEW ... WITH SCHEMA EVOLUTION + +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT) USING PARQUET; +INSERT INTO t VALUES(1); +CREATE OR REPLACE VIEW v AS SELECT * FROM t; + +ALTER VIEW v WITH SCHEMA EVOLUTION; +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 STRING, c2 INT) USING PARQUET; +-- No error, extra column +SELECT * FROM v; +DESCRIBE EXTENDED v; + +-- clean up +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/view-schema-type-evolution.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/view-schema-type-evolution.sql new file mode 100644 index 000000000000..c0278f15b641 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/view-schema-type-evolution.sql @@ -0,0 +1,90 @@ +-- This test suite checks the WITH SCHEMA TYPE EVOLUTION clause + +-- In TYPE EVOLUTION mode Spark will inherit the view column types from the query +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT NOT NULL, c2 INT) USING PARQUET; +INSERT INTO t VALUES (1, 2); +CREATE OR REPLACE VIEW v WITH SCHEMA TYPE EVOLUTION AS SELECT * FROM t; +SELECT * FROM v; +DESCRIBE EXTENDED v; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 STRING NOT NULL, c2 DOUBLE) USING PARQUET; +INSERT INTO t VALUES ('1', 2.0); +SELECT * FROM v; +-- The view now describes as v(c1 STRING, c2 DOUBLE) +DESCRIBE EXTENDED v; + +-- In TYPE EVOLUTION no new columns are inherited +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 STRING, c2 DOUBLE, c3 DATE) USING PARQUET; +INSERT INTO t VALUES ('1', 2.0, DATE'2022-01-01'); +SELECT * FROM v; +-- The view still describes as v(c1 STRING, c2 DOUBLE) +DESCRIBE EXTENDED v; + +-- Still can't drop a column +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT, c2 INT) USING PARQUET; +INSERT INTO t VALUES (1, 2); +CREATE OR REPLACE VIEW v WITH SCHEMA TYPE EVOLUTION AS SELECT * FROM t; +SELECT * FROM v; + +-- Describes as v(c1 INT, c2 INT) +DESCRIBE EXTENDED v; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT) USING PARQUET; + +-- The view should be invalid, it lost a column +SELECT * FROM v; +DESCRIBE EXTENDED v; + +-- Attempt to rename a column +DROP TABLE IF EXISTS t; +CREATE TABLE t(c3 INT, c2 INT) USING PARQUET; +SELECT * FROM v; +DESCRIBE EXTENDED v; + +-- Test preservation of comments +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT COMMENT 'c1', c2 INT COMMENT 'c2') USING PARQUET; + +-- Inherit comments from the table, if none are given +CREATE OR REPLACE VIEW v(a1, a2) WITH SCHEMA TYPE EVOLUTION AS SELECT * FROM t; +DESCRIBE EXTENDED v; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 BIGINT COMMENT 'c1 6a', c2 STRING COMMENT 'c2 6a') USING PARQUET; +SELECT * FROM v; +DESCRIBE EXTENDED v; + +-- TYPE EVOLUTION, column list with comments +CREATE OR REPLACE VIEW v(a1 COMMENT 'a1', a2 COMMENT 'a2') WITH SCHEMA TYPE EVOLUTION AS SELECT * FROM t; +DESCRIBE EXTENDED v; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 BIGINT COMMENT 'c1 6b', c2 STRING COMMENT 'c2 6b') USING PARQUET; +SELECT * FROM v; +DESCRIBE EXTENDED v; + +-- Test ALTER VIEW ... WITH SCHEMA TYPE EVOLUTION + +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 INT) USING PARQUET; +INSERT INTO t VALUES(1); +CREATE OR REPLACE VIEW v WITH SCHEMA COMPENSATION AS SELECT * FROM t; + +DROP TABLE IF EXISTS t; +CREATE TABLE t(c1 STRING) USING PARQUET; +INSERT INTO t VALUES('1'); +SELECT * FROM v; +DESCRIBE EXTENDED v; + +ALTER VIEW v WITH SCHEMA TYPE EVOLUTION; +SELECT * FROM v; +DESCRIBE EXTENDED v; + +-- Cleanup +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/view-with-default-collation.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/view-with-default-collation.sql new file mode 100644 index 000000000000..765e5177f3c0 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/view-with-default-collation.sql @@ -0,0 +1,152 @@ +-- CREATE VIEW with DEFAULT COLLATION +DROP VIEW IF EXISTS v; +CREATE VIEW v DEFAULT COLLATION UNICODE AS SELECT 'a'; +SELECT * FROM v; + +DROP VIEW IF EXISTS v; +CREATE VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 'a' AS c1; +SELECT COUNT(*) FROM v WHERE c1 = 'A'; + +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE); +INSERT INTO t VALUES ('a'), ('A'); +CREATE VIEW v DEFAULT COLLATION SR_AI_CI AS SELECT c1 FROM t WHERE 'ć' = 'č'; +SELECT COUNT(*) FROM v; +SELECT COUNT(*) FROM v WHERE c1 = 'A'; + +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE); +INSERT INTO t VALUES ('ć'), ('č'); +CREATE VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t; +SELECT DISTINCT COLLATION(c1) FROM v; +SELECT COUNT(*) FROM v WHERE c1 = 'c'; + +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE); +INSERT INTO t VALUES ('ć'), ('č'); +CREATE VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t; +SELECT DISTINCT COLLATION(c1) FROM v; +SELECT COUNT(*) FROM v WHERE c1 = 'c'; + +DROP VIEW IF EXISTS v; +CREATE VIEW v DEFAULT COLLATION UTF8_LCASE +AS SELECT 'a' AS c1, (SELECT (SELECT CASE 'a' = 'A' WHEN TRUE THEN 'a' ELSE 'b' END) WHERE (SELECT 'b' WHERE 'c' = 'C') = 'B') AS c2, 'c'; +SELECT COUNT(*) FROM v WHERE c1 = 'A'; +SELECT COUNT(*) FROM v WHERE c2 = 'a'; +SELECT COUNT(*) FROM v WHERE c2 = 'b'; +SELECT * FROM v; + +-- CREATE OR REPLACE VIEW with DEFAULT COLLATION +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING, c2 STRING COLLATE UTF8_LCASE); +INSERT INTO t VALUES ('a', 'a'), ('A', 'A'), ('b', 'b'); +CREATE OR REPLACE VIEW v DEFAULT COLLATION sr_ci_ai AS SELECT *, 'ć' AS c3 FROM t; +SELECT DISTINCT COLLATION(c1) FROM v; +SELECT DISTINCT COLLATION(c2) FROM v; +SELECT DISTINCT COLLATION(c3) FROM v; +SELECT COUNT(*) FROM v WHERE c1 = 'A'; +SELECT COUNT(*) FROM v WHERE c2 = 'a'; +SELECT COUNT(*) FROM v WHERE c3 = 'Č'; + +-- ALTER VIEW check default collation +DROP VIEW IF EXISTS v; +CREATE VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 1; +ALTER VIEW v AS SELECT 'a' AS c1, 'b' AS c2; +SELECT COLLATION(c1) FROM v; +SELECT COLLATION(c2) FROM v; +ALTER VIEW v AS SELECT 'c' AS c3 WHERE 'a' = 'A'; +SELECT COLLATION(c3) FROM v; + +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE, c2 STRING, c3 INT); +INSERT INTO t VALUES ('a', 'b', 1); +CREATE VIEW v DEFAULT COLLATION sr_AI_CI AS SELECT 'a' AS c1; +ALTER VIEW v AS + SELECT *, 'c' AS c4, (SELECT (SELECT CASE 'š' = 'S' WHEN TRUE THEN 'd' ELSE 'b' END)) AS c5 + FROM t + WHERE c1 = 'A' AND 'ć' = 'Č'; +SELECT COLLATION(c4) FROM v; +SELECT COLLATION(c5) FROM v; +SELECT c5 FROM v; + +-- CREATE TEMPORARY VIEW with DEFAULT COLLATION +DROP VIEW IF EXISTS v; +CREATE TEMPORARY VIEW v DEFAULT COLLATION UNICODE AS SELECT 'a'; +SELECT * FROM v; + +DROP VIEW IF EXISTS v; +CREATE TEMPORARY VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 'a' AS c1; +SELECT COUNT(*) FROM v WHERE c1 = 'A'; + +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE); +INSERT INTO t VALUES ('a'), ('A'); +CREATE TEMPORARY VIEW v DEFAULT COLLATION SR_AI_CI AS SELECT c1 FROM t WHERE 'ć' = 'č'; +SELECT COUNT(*) FROM v; +SELECT COUNT(*) FROM v WHERE c1 = 'A'; + +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE); +INSERT INTO t VALUES ('ć'), ('č'); +CREATE TEMPORARY VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t; +SELECT DISTINCT COLLATION(c1) FROM v; +SELECT COUNT(*) FROM v WHERE c1 = 'c'; + +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE); +INSERT INTO t VALUES ('ć'), ('č'); +CREATE TEMPORARY VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t; +SELECT DISTINCT COLLATION(c1) FROM v; +SELECT COUNT(*) FROM v WHERE c1 = 'c'; + +DROP VIEW IF EXISTS v; +CREATE TEMPORARY VIEW v DEFAULT COLLATION UTF8_LCASE +AS SELECT 'a' AS c1, (SELECT (SELECT CASE 'a' = 'A' WHEN TRUE THEN 'a' ELSE 'b' END) WHERE (SELECT 'b' WHERE 'c' = 'C') = 'B') AS c2, 'c'; +SELECT COUNT(*) FROM v WHERE c1 = 'A'; +SELECT COUNT(*) FROM v WHERE c2 = 'a'; +SELECT COUNT(*) FROM v WHERE c2 = 'b'; +SELECT * FROM v; + +-- CREATE OR REPLACE TEMPORARY VIEW with DEFAULT COLLATION +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING, c2 STRING COLLATE UTF8_LCASE); +INSERT INTO t VALUES ('a', 'a'), ('A', 'A'), ('b', 'b'); +CREATE OR REPLACE TEMPORARY VIEW v DEFAULT COLLATION sr_ci_ai AS SELECT *, 'ć' AS c3 FROM t; +SELECT DISTINCT COLLATION(c1) FROM v; +SELECT DISTINCT COLLATION(c2) FROM v; +SELECT DISTINCT COLLATION(c3) FROM v; +SELECT COUNT(*) FROM v WHERE c1 = 'A'; +SELECT COUNT(*) FROM v WHERE c2 = 'a'; +SELECT COUNT(*) FROM v WHERE c3 = 'Č'; + +-- ALTER TEMPORARY VIEW check default collation +DROP VIEW IF EXISTS v; +CREATE TEMPORARY VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 1; +ALTER VIEW v AS SELECT 'a' AS c1, 'b' AS c2; +SELECT COLLATION(c1) FROM v; +SELECT COLLATION(c2) FROM v; +ALTER VIEW v AS SELECT 'c' AS c3 WHERE 'a' = 'A'; +SELECT COLLATION(c3) FROM v; + +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE, c2 STRING, c3 INT); +INSERT INTO t VALUES ('a', 'b', 1); +CREATE TEMPORARY VIEW v DEFAULT COLLATION sr_AI_CI AS SELECT 'a' AS c1; +ALTER VIEW v AS + SELECT *, 'c' AS c4, (SELECT (SELECT CASE 'š' = 'S' WHEN TRUE THEN 'd' ELSE 'b' END)) AS c5 + FROM t + WHERE c1 = 'A' AND 'ć' = 'Č'; +SELECT COLLATION(c4) FROM v; +SELECT COLLATION(c5) FROM v; +SELECT c5 FROM v; + +DROP VIEW IF EXISTS v; +DROP TABLE IF EXISTS t; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/window.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/window.sql new file mode 100644 index 000000000000..bec79247f9a6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/window.sql @@ -0,0 +1,485 @@ +-- Test window operator with codegen on and off. +--CONFIG_DIM1 spark.sql.codegen.wholeStage=true +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN +--CONFIG_DIM2 spark.sql.optimizer.windowGroupLimitThreshold=-1 +--CONFIG_DIM2 spark.sql.optimizer.windowGroupLimitThreshold=1000 + +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), +(null, null, null, null, null, null), +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) +AS testData(val, val_long, val_double, val_date, val_timestamp, cate); + +CREATE OR REPLACE TEMPORARY VIEW basic_pays AS SELECT * FROM VALUES +('Diane Murphy','Accounting',8435), +('Mary Patterson','Accounting',9998), +('Jeff Firrelli','Accounting',8992), +('William Patterson','Accounting',8870), +('Gerard Bondur','Accounting',11472), +('Anthony Bow','Accounting',6627), +('Leslie Jennings','IT',8113), +('Leslie Thompson','IT',5186), +('Julie Firrelli','Sales',9181), +('Steve Patterson','Sales',9441), +('Foon Yue Tseng','Sales',6660), +('George Vanauf','Sales',10563), +('Loui Bondur','SCM',10449), +('Gerard Hernandez','SCM',6949), +('Pamela Castillo','SCM',11303), +('Larry Bott','SCM',11798), +('Barry Jones','SCM',10586) +AS basic_pays(employee_name, department, salary); + +CREATE OR REPLACE TEMPORARY VIEW test_ignore_null AS SELECT * FROM VALUES +('a', 0, null), +('a', 1, 'x'), +('b', 2, null), +('c', 3, null), +('a', 4, 'y'), +('b', 5, null), +('a', 6, 'z'), +('a', 7, 'v'), +('a', 8, null) +AS test_ignore_null(content, id, v); + +-- RowsBetween +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val ROWS CURRENT ROW) FROM testData +ORDER BY cate, val; +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val +ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) FROM testData ORDER BY cate, val; +SELECT val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long +ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long; + +-- RangeBetween +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val RANGE 1 PRECEDING) FROM testData +ORDER BY cate, val; +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; +SELECT val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long +RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long; +SELECT val_double, cate, sum(val_double) OVER(PARTITION BY cate ORDER BY val_double +RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING) FROM testData ORDER BY cate, val_double; +SELECT val_date, cate, max(val_date) OVER(PARTITION BY cate ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING) FROM testData ORDER BY cate, val_date; +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData +ORDER BY cate, val_timestamp; +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) +RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData +ORDER BY cate, to_timestamp_ntz(val_timestamp); +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval '1-1' year to month FOLLOWING) FROM testData +ORDER BY cate, val_timestamp; +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) +RANGE BETWEEN CURRENT ROW AND interval '1-1' year to month FOLLOWING) FROM testData +ORDER BY cate, to_timestamp_ntz(val_timestamp); +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING) FROM testData +ORDER BY cate, val_timestamp; +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) +RANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING) FROM testData +ORDER BY cate, to_timestamp_ntz(val_timestamp); +SELECT val_date, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND interval '1-1' year to month FOLLOWING) FROM testData +ORDER BY cate, val_date; +SELECT val_date, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING) FROM testData +ORDER BY cate, val_date; + +-- RangeBetween with reverse OrderBy +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val DESC +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; + +-- Invalid window frame +SELECT val, cate, count(val) OVER(PARTITION BY cate +ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, val; +SELECT val, cate, count(val) OVER(PARTITION BY cate +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val, cate +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY current_timestamp +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY cate, val; +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val; + + +-- Window functions +SELECT val, cate, +max(val) OVER w AS max, +min(val) OVER w AS min, +min(val) OVER w AS min, +count(val) OVER w AS count, +sum(val) OVER w AS sum, +avg(val) OVER w AS avg, +stddev(val) OVER w AS stddev, +first_value(val) OVER w AS first_value, +first_value(val, true) OVER w AS first_value_ignore_null, +first_value(val, false) OVER w AS first_value_contain_null, +any_value(val) OVER w AS any_value, +any_value(val, true) OVER w AS any_value_ignore_null, +any_value(val, false) OVER w AS any_value_contain_null, +last_value(val) OVER w AS last_value, +last_value(val, true) OVER w AS last_value_ignore_null, +last_value(val, false) OVER w AS last_value_contain_null, +rank() OVER w AS rank, +dense_rank() OVER w AS dense_rank, +cume_dist() OVER w AS cume_dist, +percent_rank() OVER w AS percent_rank, +ntile(2) OVER w AS ntile, +row_number() OVER w AS row_number, +var_pop(val) OVER w AS var_pop, +var_samp(val) OVER w AS var_samp, +approx_count_distinct(val) OVER w AS approx_count_distinct, +covar_pop(val, val_long) OVER w AS covar_pop, +corr(val, val_long) OVER w AS corr, +stddev_samp(val) OVER w AS stddev_samp, +stddev_pop(val) OVER w AS stddev_pop, +collect_list(val) OVER w AS collect_list, +collect_set(val) OVER w AS collect_set, +skewness(val_double) OVER w AS skewness, +kurtosis(val_double) OVER w AS kurtosis +FROM testData +WINDOW w AS (PARTITION BY cate ORDER BY val) +ORDER BY cate, val; + +-- Null inputs +SELECT val, cate, avg(null) OVER(PARTITION BY cate ORDER BY val) FROM testData ORDER BY cate, val; + +-- OrderBy not specified +SELECT val, cate, row_number() OVER(PARTITION BY cate) FROM testData ORDER BY cate, val; + +-- Over clause is empty +SELECT val, cate, sum(val) OVER(), avg(val) OVER() FROM testData ORDER BY cate, val; + +-- first_value()/last_value()/any_value() over () +SELECT val, cate, +first_value(false) OVER w AS first_value, +first_value(true, true) OVER w AS first_value_ignore_null, +first_value(false, false) OVER w AS first_value_contain_null, +any_value(false) OVER w AS any_value, +any_value(true, true) OVER w AS any_value_ignore_null, +any_value(false, false) OVER w AS any_value_contain_null, +last_value(false) OVER w AS last_value, +last_value(true, true) OVER w AS last_value_ignore_null, +last_value(false, false) OVER w AS last_value_contain_null +FROM testData +WINDOW w AS () +ORDER BY cate, val; + +-- parentheses around window reference +SELECT cate, sum(val) OVER (w) +FROM testData +WHERE val is not null +WINDOW w AS (PARTITION BY cate ORDER BY val); + +-- with filter predicate +SELECT val, cate, +count(val) FILTER (WHERE val > 1) OVER(PARTITION BY cate) +FROM testData ORDER BY cate, val; + +-- nth_value()/first_value()/any_value() over () +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC) +ORDER BY salary DESC; + +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY salary DESC; + +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY salary DESC; + +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary RANGE BETWEEN 2000 PRECEDING AND 1000 FOLLOWING) +ORDER BY salary; + +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING) +ORDER BY salary DESC; + +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +ORDER BY salary DESC; + +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY salary DESC; + +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY salary DESC; + +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary DESC; + +SELECT + employee_name, + department, + salary, + FIRST_VALUE(employee_name) OVER w highest_salary, + NTH_VALUE(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS ( + PARTITION BY department + ORDER BY salary DESC + RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING +) +ORDER BY department; + +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW + w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING), + w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 2 FOLLOWING) +ORDER BY salary DESC; + +SELECT + content, + id, + v, + lead(v, 0) IGNORE NULLS OVER w lead_0, + lead(v, 1) IGNORE NULLS OVER w lead_1, + lead(v, 2) IGNORE NULLS OVER w lead_2, + lead(v, 3) IGNORE NULLS OVER w lead_3, + lag(v, 0) IGNORE NULLS OVER w lag_0, + lag(v, 1) IGNORE NULLS OVER w lag_1, + lag(v, 2) IGNORE NULLS OVER w lag_2, + lag(v, 3) IGNORE NULLS OVER w lag_3, + lag(v, +3) IGNORE NULLS OVER w lag_plus_3, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id) +ORDER BY id; + +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY id; + +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY id; + +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN 2 PRECEDING AND 2 FOLLOWING) +ORDER BY id; + +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING) +ORDER BY id; + +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +ORDER BY id; + +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY id; + +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY id; + +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY id; + +SELECT + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays; + +SELECT + SUM(salary) OVER w sum_salary +FROM + basic_pays; + +-- Test cases for InferWindowGroupLimit +create or replace temp view t1 (p, o) as values (1, 1), (1, 1), (1, 2), (2, 1), (2, 1), (2, 2); +select * from (select *, dense_rank() over (partition by p order by o) as rnk from t1) where rnk = 1; + +SELECT * FROM (SELECT cate, val, rank() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r = 1; +SELECT * FROM (SELECT cate, val, rank() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r <= 2; +SELECT * FROM (SELECT cate, val, dense_rank() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r = 1; +SELECT * FROM (SELECT cate, val, dense_rank() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r <= 2; +SELECT * FROM (SELECT cate, val, row_number() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r = 1; +SELECT * FROM (SELECT cate, val, row_number() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r <= 2; + +SELECT *, mean(val_double) over (partition BY val ORDER BY val_date RANGE INTERVAL '5' DAY PRECEDING) AS mean FROM testData; +SELECT *, mean(val_double) over (partition BY val ORDER BY val_date RANGE INTERVAL '1 2:3:4.001' DAY TO SECOND PRECEDING) AS mean FROM testData; +SELECT *, mean(val_double) over (partition BY val ORDER BY val_date RANGE DATE '2024-01-01' FOLLOWING) AS mean FROM testData; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/xml-functions.sql b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/xml-functions.sql new file mode 100644 index 000000000000..7e3d21ef7536 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/inputs/xml-functions.sql @@ -0,0 +1,60 @@ +-- to_xml +select to_xml(named_struct('a', 1, 'b', 2), map('indent', '')); +select to_xml(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy', 'indent', '')); +-- Check if errors handled +select to_xml(array(named_struct('a', 1, 'b', 2))); +select to_xml(map('a', 1)); +select to_xml(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE')); +select to_xml(named_struct('a', 1, 'b', 2), map('mode', 1)); +select to_xml(); + +-- from_xml +select from_xml('

1

', 'a INT'); +select from_xml('

', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')); +-- Check if errors handled +select from_xml('

1

', 1); +select from_xml('

1

', 'a InvalidType'); +select from_xml('

1

', 'a INT', named_struct('mode', 'PERMISSIVE')); +select from_xml('

1

', 'a INT', map('mode', 1)); +select from_xml(); + +-- Clean up +DROP VIEW IF EXISTS xmlTable; + +-- from_xml - complex types +select from_xml('

1

', 'struct>'); +select from_xml('

1"2"

', 'struct'); + +-- infer schema of xml literal +select schema_of_xml('

1"2"

'); +select from_xml('

123

', schema_of_xml('

12

')); + +-- from_xml - array type +select from_xml('

12

', 'struct>'); +select from_xml('

1"2"

', 'struct>'); +select from_xml('

1

', 'struct>'); + +select from_xml('

2

', 'struct>'); + +-- from_xml - datetime type +select from_xml('

2012-12-152012-12-15 15:15:15

', 'd date, t timestamp'); +select from_xml( + '

12/15 201212/15 2012 15:15:15}

', + 'd date, t timestamp', + map('dateFormat', 'MM/dd yyyy', 'timestampFormat', 'MM/dd yyyy HH:mm:ss')); +select from_xml( + '

02-29

', + 'd date', + map('dateFormat', 'MM-dd')); +select from_xml( + '

02-29

', + 't timestamp', + map('timestampFormat', 'MM-dd')); + +-- infer schema of xml literal with options +select schema_of_xml(null); +CREATE TEMPORARY VIEW xmlTable(xmlField, a) AS SELECT * FROM VALUES ('

1"2"

', 'a'); +SELECT schema_of_xml(xmlField) FROM xmlTable; + +-- Clean up +DROP VIEW IF EXISTS xmlTable; diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/array.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/array.sql.out new file mode 100644 index 000000000000..b5dabfb47f43 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/array.sql.out @@ -0,0 +1,900 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view data as select * from values + ("one", array(11, 12, 13), array(array(111, 112, 113), array(121, 122, 123))), + ("two", array(21, 22, 23), array(array(211, 212, 213), array(221, 222, 223))) + as data(a, b, c) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from data +-- !query schema +struct,c:array>> +-- !query output +one [11,12,13] [[111,112,113],[121,122,123]] +two [21,22,23] [[211,212,213],[221,222,223]] + + +-- !query +select a, b[0], b[0] + b[1] from data +-- !query schema +struct +-- !query output +one 11 23 +two 21 43 + + +-- !query +select a, c[0][0] + c[0][0 + 1] from data +-- !query schema +struct +-- !query output +one 223 +two 423 + + +-- !query +create temporary view primitive_arrays as select * from values ( + array(true), + array(2Y, 1Y), + array(2S, 1S), + array(2, 1), + array(2L, 1L), + array(9223372036854775809, 9223372036854775808), + array(2.0D, 1.0D), + array(float(2.0), float(1.0)), + array(date '2016-03-14', date '2016-03-13'), + array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000') +) as primitive_arrays( + boolean_array, + tinyint_array, + smallint_array, + int_array, + bigint_array, + decimal_array, + double_array, + float_array, + date_array, + timestamp_array +) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from primitive_arrays +-- !query schema +struct,tinyint_array:array,smallint_array:array,int_array:array,bigint_array:array,decimal_array:array,double_array:array,float_array:array,date_array:array,timestamp_array:array> +-- !query output +[true] [2,1] [2,1] [2,1] [2,1] [9223372036854775809,9223372036854775808] [2.0,1.0] [2.0,1.0] [2016-03-14,2016-03-13] [2016-11-15 20:54:00,2016-11-12 20:54:00] + + +-- !query +select + array_contains(boolean_array, true), array_contains(boolean_array, false), + array_contains(tinyint_array, 2Y), array_contains(tinyint_array, 0Y), + array_contains(smallint_array, 2S), array_contains(smallint_array, 0S), + array_contains(int_array, 2), array_contains(int_array, 0), + array_contains(bigint_array, 2L), array_contains(bigint_array, 0L), + array_contains(decimal_array, 9223372036854775809), array_contains(decimal_array, 1), + array_contains(double_array, 2.0D), array_contains(double_array, 0.0D), + array_contains(float_array, float(2.0)), array_contains(float_array, float(0.0)), + array_contains(date_array, date '2016-03-14'), array_contains(date_array, date '2016-01-01'), + array_contains(timestamp_array, timestamp '2016-11-15 20:54:00.000'), array_contains(timestamp_array, timestamp '2016-01-01 20:54:00.000') +from primitive_arrays +-- !query schema +struct +-- !query output +true false true false true false true false true false true false true false true false true false true false + + +-- !query +select array_contains(b, 11), array_contains(c, array(111, 112, 113)) from data +-- !query schema +struct +-- !query output +false false +true true + + +-- !query +select + sort_array(boolean_array), + sort_array(tinyint_array), + sort_array(smallint_array), + sort_array(int_array), + sort_array(bigint_array), + sort_array(decimal_array), + sort_array(double_array), + sort_array(float_array), + sort_array(date_array), + sort_array(timestamp_array) +from primitive_arrays +-- !query schema +struct,sort_array(tinyint_array, true):array,sort_array(smallint_array, true):array,sort_array(int_array, true):array,sort_array(bigint_array, true):array,sort_array(decimal_array, true):array,sort_array(double_array, true):array,sort_array(float_array, true):array,sort_array(date_array, true):array,sort_array(timestamp_array, true):array> +-- !query output +[true] [1,2] [1,2] [1,2] [1,2] [9223372036854775808,9223372036854775809] [1.0,2.0] [1.0,2.0] [2016-03-13,2016-03-14] [2016-11-12 20:54:00,2016-11-15 20:54:00] + + +-- !query +select sort_array(array('b', 'd'), '1') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"sort_array(array(b, d), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "sort_array(array('b', 'd'), '1')" + } ] +} + + +-- !query +select sort_array(array('b', 'd'), cast(NULL as boolean)) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select + size(boolean_array), + size(tinyint_array), + size(smallint_array), + size(int_array), + size(bigint_array), + size(decimal_array), + size(double_array), + size(float_array), + size(date_array), + size(timestamp_array) +from primitive_arrays +-- !query schema +struct +-- !query output +1 2 2 2 2 2 2 2 2 2 + + +-- !query +select element_at(array(1, 2, 3), 5) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArrayIndexOutOfBoundsException +{ + "errorClass" : "INVALID_ARRAY_INDEX_IN_ELEMENT_AT", + "sqlState" : "22003", + "messageParameters" : { + "arraySize" : "3", + "indexValue" : "5" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "element_at(array(1, 2, 3), 5)" + } ] +} + + +-- !query +select element_at(array(1, 2, 3), -5) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArrayIndexOutOfBoundsException +{ + "errorClass" : "INVALID_ARRAY_INDEX_IN_ELEMENT_AT", + "sqlState" : "22003", + "messageParameters" : { + "arraySize" : "3", + "indexValue" : "-5" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "element_at(array(1, 2, 3), -5)" + } ] +} + + +-- !query +select element_at(array(1, 2, 3), 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_INDEX_OF_ZERO", + "sqlState" : "22003", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "element_at(array(1, 2, 3), 0)" + } ] +} + + +-- !query +select elt(4, '123', '456') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArrayIndexOutOfBoundsException +{ + "errorClass" : "INVALID_ARRAY_INDEX", + "sqlState" : "22003", + "messageParameters" : { + "arraySize" : "2", + "indexValue" : "4" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "elt(4, '123', '456')" + } ] +} + + +-- !query +select elt(0, '123', '456') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArrayIndexOutOfBoundsException +{ + "errorClass" : "INVALID_ARRAY_INDEX", + "sqlState" : "22003", + "messageParameters" : { + "arraySize" : "2", + "indexValue" : "0" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "elt(0, '123', '456')" + } ] +} + + +-- !query +select elt(-1, '123', '456') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArrayIndexOutOfBoundsException +{ + "errorClass" : "INVALID_ARRAY_INDEX", + "sqlState" : "22003", + "messageParameters" : { + "arraySize" : "2", + "indexValue" : "-1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "elt(-1, '123', '456')" + } ] +} + + +-- !query +select elt(null, '123', '456') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select elt(null, '123', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select elt(1, '123', null) +-- !query schema +struct +-- !query output +123 + + +-- !query +select elt(2, '123', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select array(1, 2, 3)[5] +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArrayIndexOutOfBoundsException +{ + "errorClass" : "INVALID_ARRAY_INDEX", + "sqlState" : "22003", + "messageParameters" : { + "arraySize" : "3", + "indexValue" : "5" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "array(1, 2, 3)[5]" + } ] +} + + +-- !query +select array(1, 2, 3)[-1] +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArrayIndexOutOfBoundsException +{ + "errorClass" : "INVALID_ARRAY_INDEX", + "sqlState" : "22003", + "messageParameters" : { + "arraySize" : "3", + "indexValue" : "-1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "array(1, 2, 3)[-1]" + } ] +} + + +-- !query +select array_size(array()) +-- !query schema +struct +-- !query output +0 + + +-- !query +select array_size(array(true)) +-- !query schema +struct +-- !query output +1 + + +-- !query +select array_size(array(2, 1)) +-- !query schema +struct +-- !query output +2 + + +-- !query +select array_size(NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select array_size(map('a', 1, 'b', 2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"map(a, 1, b, 2)\"", + "inputType" : "\"MAP\"", + "paramIndex" : "first", + "requiredType" : "\"ARRAY\"", + "sqlExpr" : "\"array_size(map(a, 1, b, 2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "array_size(map('a', 1, 'b', 2))" + } ] +} + + +-- !query +select size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10))) +-- !query schema +struct +-- !query output +4 + + +-- !query +select size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10))) +-- !query schema +struct +-- !query output +4 + + +-- !query +select size(arrays_zip(array(1, 2, 3), array(4), null, array(7, 8, 9, 10))) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select isnotnull(arrays_zip(array(), array(4), array(7, 8, 9, 10))) +-- !query schema +struct<(arrays_zip(array(), array(4), array(7, 8, 9, 10)) IS NOT NULL):boolean> +-- !query output +true + + +-- !query +select isnotnull(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10))) +-- !query schema +struct<(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10)) IS NOT NULL):boolean> +-- !query output +true + + +-- !query +select isnotnull(arrays_zip(array(1, 2, 3), NULL, array(4), array(7, 8, 9, 10))) +-- !query schema +struct<(arrays_zip(array(1, 2, 3), NULL, array(4), array(7, 8, 9, 10)) IS NOT NULL):boolean> +-- !query output +false + + +-- !query +select get(array(1, 2, 3), 0) +-- !query schema +struct +-- !query output +1 + + +-- !query +select get(array(1, 2, 3), 3) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select get(array(1, 2, 3), null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select get(array(1, 2, 3), -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select array_insert(array(1, 2, 3), 3, 4) +-- !query schema +struct> +-- !query output +[1,2,4,3] + + +-- !query +select array_insert(array(2, 3, 4), 0, 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_INDEX_OF_ZERO", + "sqlState" : "22003", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "array_insert(array(2, 3, 4), 0, 1)" + } ] +} + + +-- !query +select array_insert(array(2, 3, 4), 1, 1) +-- !query schema +struct> +-- !query output +[1,2,3,4] + + +-- !query +select array_insert(array(1, 3, 4), -2, 2) +-- !query schema +struct> +-- !query output +[1,3,2,4] + + +-- !query +select array_insert(array(1, 2, 3), 3, "4") +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "\"ARRAY\"", + "functionName" : "`array_insert`", + "leftType" : "\"ARRAY\"", + "rightType" : "\"STRING\"", + "sqlExpr" : "\"array_insert(array(1, 2, 3), 3, 4)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "array_insert(array(1, 2, 3), 3, \"4\")" + } ] +} + + +-- !query +select array_insert(cast(NULL as ARRAY), 1, 1) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select array_insert(array(1, 2, 3, NULL), cast(NULL as INT), 4) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select array_insert(array(1, 2, 3, NULL), 4, cast(NULL as INT)) +-- !query schema +struct> +-- !query output +[1,2,3,null,null] + + +-- !query +select array_insert(array(2, 3, NULL, 4), 5, 5) +-- !query schema +struct> +-- !query output +[2,3,null,4,5] + + +-- !query +select array_insert(array(2, 3, NULL, 4), -5, 1) +-- !query schema +struct> +-- !query output +[1,2,3,null,4] + + +-- !query +select array_insert(array(1), 2, cast(2 as tinyint)) +-- !query schema +struct> +-- !query output +[1,2] + + +-- !query +set spark.sql.legacy.negativeIndexInArrayInsert=true +-- !query schema +struct +-- !query output +spark.sql.legacy.negativeIndexInArrayInsert true + + +-- !query +select array_insert(array(1, 3, 4), -2, 2) +-- !query schema +struct> +-- !query output +[1,2,3,4] + + +-- !query +select array_insert(array(2, 3, NULL, 4), -5, 1) +-- !query schema +struct> +-- !query output +[1,null,2,3,null,4] + + +-- !query +set spark.sql.legacy.negativeIndexInArrayInsert=false +-- !query schema +struct +-- !query output +spark.sql.legacy.negativeIndexInArrayInsert false + + +-- !query +select array_compact(id) from values (1) as t(id) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"id\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "\"ARRAY\"", + "sqlExpr" : "\"array_compact(id)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "array_compact(id)" + } ] +} + + +-- !query +select array_compact(array("1", null, "2", null)) +-- !query schema +struct> +-- !query output +["1","2"] + + +-- !query +select array_compact(array("a", "b", "c")) +-- !query schema +struct> +-- !query output +["a","b","c"] + + +-- !query +select array_compact(array(1D, null, 2D, null)) +-- !query schema +struct> +-- !query output +[1.0,2.0] + + +-- !query +select array_compact(array(array(1, 2, 3, null), null, array(4, null, 6))) +-- !query schema +struct>> +-- !query output +[[1,2,3,null],[4,null,6]] + + +-- !query +select array_compact(array(null)) +-- !query schema +struct> +-- !query output +[] + + +-- !query +select array_append(array(1, 2, 3), 4) +-- !query schema +struct> +-- !query output +[1,2,3,4] + + +-- !query +select array_append(array('a', 'b', 'c'), 'd') +-- !query schema +struct> +-- !query output +["a","b","c","d"] + + +-- !query +select array_append(array(1, 2, 3, NULL), NULL) +-- !query schema +struct> +-- !query output +[1,2,3,null,null] + + +-- !query +select array_append(array('a', 'b', 'c', NULL), NULL) +-- !query schema +struct> +-- !query output +["a","b","c",null,null] + + +-- !query +select array_append(CAST(null AS ARRAY), 'a') +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select array_append(CAST(null AS ARRAY), CAST(null as String)) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select array_append(array(), 1) +-- !query schema +struct> +-- !query output +[1] + + +-- !query +select array_append(CAST(array() AS ARRAY), CAST(NULL AS String)) +-- !query schema +struct> +-- !query output +[null] + + +-- !query +select array_append(array(CAST(NULL AS String)), CAST(NULL AS String)) +-- !query schema +struct> +-- !query output +[null,null] + + +-- !query +select array_prepend(array(1, 2, 3), 4) +-- !query schema +struct> +-- !query output +[4,1,2,3] + + +-- !query +select array_prepend(array('a', 'b', 'c'), 'd') +-- !query schema +struct> +-- !query output +["d","a","b","c"] + + +-- !query +select array_prepend(array(1, 2, 3, NULL), NULL) +-- !query schema +struct> +-- !query output +[null,1,2,3,null] + + +-- !query +select array_prepend(array('a', 'b', 'c', NULL), NULL) +-- !query schema +struct> +-- !query output +[null,"a","b","c",null] + + +-- !query +select array_prepend(CAST(null AS ARRAY), 'a') +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select array_prepend(CAST(null AS ARRAY), CAST(null as String)) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select array_prepend(array(), 1) +-- !query schema +struct> +-- !query output +[1] + + +-- !query +select array_prepend(CAST(array() AS ARRAY), CAST(NULL AS String)) +-- !query schema +struct> +-- !query output +[null] + + +-- !query +select array_prepend(array(CAST(NULL AS String)), CAST(NULL AS String)) +-- !query schema +struct> +-- !query output +[null,null] + + +-- !query +select array_union(array(0.0, -0.0, DOUBLE("NaN")), array(0.0, -0.0, DOUBLE("NaN"))) +-- !query schema +struct> +-- !query output +[0.0,NaN] + + +-- !query +select array_distinct(array(0.0, -0.0, -0.0, DOUBLE("NaN"), DOUBLE("NaN"))) +-- !query schema +struct> +-- !query output +[0.0,NaN] diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/binary.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/binary.sql.out new file mode 100644 index 000000000000..9571d9130f73 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/binary.sql.out @@ -0,0 +1,51 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT X'' +-- !query schema +struct +-- !query output + + + +-- !query +SELECT X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333' +-- !query schema +struct +-- !query output +Eason Yao 2018-11-17:13:33:33 + + +-- !query +SELECT CAST('Spark' as BINARY) +-- !query schema +struct +-- !query output +Spark + + +-- !query +SELECT array( X'', X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333', CAST('Spark' as BINARY)) +-- !query schema +struct> +-- !query output +[,Eason Yao 2018-11-17:13:33:33,Spark] + + +-- !query +SELECT to_csv(named_struct('n', 1, 'info', X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333')) +-- !query schema +struct +-- !query output +1,Eason Yao 2018-11-17:13:33:33 + + +-- !query +select to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop'))) +-- !query schema +struct +-- !query output + + Eason + 2018 + Kindergarten Cop + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/binary_base64.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/binary_base64.sql.out new file mode 100644 index 000000000000..ef45d059bc81 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/binary_base64.sql.out @@ -0,0 +1,51 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT X'' +-- !query schema +struct +-- !query output + + + +-- !query +SELECT X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333' +-- !query schema +struct +-- !query output +RWFzb24gWWFvIDIwMTgtMTEtMTc6MTM6MzM6MzM + + +-- !query +SELECT CAST('Spark' as BINARY) +-- !query schema +struct +-- !query output +U3Bhcms + + +-- !query +SELECT array( X'', X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333', CAST('Spark' as BINARY)) +-- !query schema +struct> +-- !query output +[,RWFzb24gWWFvIDIwMTgtMTEtMTc6MTM6MzM6MzM,U3Bhcms] + + +-- !query +SELECT to_csv(named_struct('n', 1, 'info', X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333')) +-- !query schema +struct +-- !query output +1,RWFzb24gWWFvIDIwMTgtMTEtMTc6MTM6MzM6MzM + + +-- !query +select to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop'))) +-- !query schema +struct +-- !query output + + RWFzb24 + 2018 + S2luZGVyZ2FydGVuIENvcA + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/binary_basic.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/binary_basic.sql.out new file mode 100644 index 000000000000..0118df765df1 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/binary_basic.sql.out @@ -0,0 +1,51 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT X'' +-- !query schema +struct +-- !query output +[] + + +-- !query +SELECT X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333' +-- !query schema +struct +-- !query output +[69, 97, 115, 111, 110, 32, 89, 97, 111, 32, 50, 48, 49, 56, 45, 49, 49, 45, 49, 55, 58, 49, 51, 58, 51, 51, 58, 51, 51] + + +-- !query +SELECT CAST('Spark' as BINARY) +-- !query schema +struct +-- !query output +[83, 112, 97, 114, 107] + + +-- !query +SELECT array( X'', X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333', CAST('Spark' as BINARY)) +-- !query schema +struct> +-- !query output +[[],[69, 97, 115, 111, 110, 32, 89, 97, 111, 32, 50, 48, 49, 56, 45, 49, 49, 45, 49, 55, 58, 49, 51, 58, 51, 51, 58, 51, 51],[83, 112, 97, 114, 107]] + + +-- !query +SELECT to_csv(named_struct('n', 1, 'info', X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333')) +-- !query schema +struct +-- !query output +1,"[69, 97, 115, 111, 110, 32, 89, 97, 111, 32, 50, 48, 49, 56, 45, 49, 49, 45, 49, 55, 58, 49, 51, 58, 51, 51, 58, 51, 51]" + + +-- !query +select to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop'))) +-- !query schema +struct +-- !query output + + [69, 97, 115, 111, 110] + 2018 + [75, 105, 110, 100, 101, 114, 103, 97, 114, 116, 101, 110, 32, 67, 111, 112] + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/binary_hex.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/binary_hex.sql.out new file mode 100644 index 000000000000..d97f6efae292 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/binary_hex.sql.out @@ -0,0 +1,51 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT X'' +-- !query schema +struct +-- !query output + + + +-- !query +SELECT X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333' +-- !query schema +struct +-- !query output +4561736F6E2059616F20323031382D31312D31373A31333A33333A3333 + + +-- !query +SELECT CAST('Spark' as BINARY) +-- !query schema +struct +-- !query output +537061726B + + +-- !query +SELECT array( X'', X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333', CAST('Spark' as BINARY)) +-- !query schema +struct> +-- !query output +[,4561736F6E2059616F20323031382D31312D31373A31333A33333A3333,537061726B] + + +-- !query +SELECT to_csv(named_struct('n', 1, 'info', X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333')) +-- !query schema +struct +-- !query output +1,4561736F6E2059616F20323031382D31312D31373A31333A33333A3333 + + +-- !query +select to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop'))) +-- !query schema +struct +-- !query output + + 4561736F6E + 2018 + 4B696E64657267617274656E20436F70 + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/binary_hex_discrete.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/binary_hex_discrete.sql.out new file mode 100644 index 000000000000..e0dc049db833 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/binary_hex_discrete.sql.out @@ -0,0 +1,51 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT X'' +-- !query schema +struct +-- !query output +[] + + +-- !query +SELECT X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333' +-- !query schema +struct +-- !query output +[45 61 73 6F 6E 20 59 61 6F 20 32 30 31 38 2D 31 31 2D 31 37 3A 31 33 3A 33 33 3A 33 33] + + +-- !query +SELECT CAST('Spark' as BINARY) +-- !query schema +struct +-- !query output +[53 70 61 72 6B] + + +-- !query +SELECT array( X'', X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333', CAST('Spark' as BINARY)) +-- !query schema +struct> +-- !query output +[[],[45 61 73 6F 6E 20 59 61 6F 20 32 30 31 38 2D 31 31 2D 31 37 3A 31 33 3A 33 33 3A 33 33],[53 70 61 72 6B]] + + +-- !query +SELECT to_csv(named_struct('n', 1, 'info', X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333')) +-- !query schema +struct +-- !query output +1,[45 61 73 6F 6E 20 59 61 6F 20 32 30 31 38 2D 31 31 2D 31 37 3A 31 33 3A 33 33 3A 33 33] + + +-- !query +select to_xml(named_struct('name', binary('Eason'), 'birth', 2018, 'org', binary('Kindergarten Cop'))) +-- !query schema +struct +-- !query output + + [45 61 73 6F 6E] + 2018 + [4B 69 6E 64 65 72 67 61 72 74 65 6E 20 43 6F 70] + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/bitwise.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/bitwise.sql.out new file mode 100644 index 000000000000..7233b0d0ae49 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/bitwise.sql.out @@ -0,0 +1,476 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select bit_count(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select bit_count(true) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count(false) +-- !query schema +struct +-- !query output +0 + + +-- !query +select bit_count(cast(1 as tinyint)) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count(cast(2 as tinyint)) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count(cast(3 as tinyint)) +-- !query schema +struct +-- !query output +2 + + +-- !query +select bit_count(1S) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count(2S) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count(3S) +-- !query schema +struct +-- !query output +2 + + +-- !query +select bit_count(1) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count(2) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count(3) +-- !query schema +struct +-- !query output +2 + + +-- !query +select bit_count(1L) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count(2L) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count(3L) +-- !query schema +struct +-- !query output +2 + + +-- !query +select bit_count(-1L) +-- !query schema +struct +-- !query output +64 + + +-- !query +select bit_count(9223372036854775807L) +-- !query schema +struct +-- !query output +63 + + +-- !query +select bit_count(-9223372036854775808L) +-- !query schema +struct +-- !query output +1 + + +-- !query +select bit_count("bit count") +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"bit count\"", + "inputType" : "\"STRING\"", + "paramIndex" : "first", + "requiredType" : "(\"INTEGRAL\" or \"BOOLEAN\")", + "sqlExpr" : "\"bit_count(bit count)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "bit_count(\"bit count\")" + } ] +} + + +-- !query +select bit_count('a') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"a\"", + "inputType" : "\"STRING\"", + "paramIndex" : "first", + "requiredType" : "(\"INTEGRAL\" or \"BOOLEAN\")", + "sqlExpr" : "\"bit_count(a)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "bit_count('a')" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (2, 3, 4, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT BIT_XOR(b3) AS n1 FROM bitwise_test where 1 = 0 +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT BIT_XOR(b4) AS n1 FROM bitwise_test where b4 is null +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT + BIT_XOR(cast(b1 as tinyint)) AS a4, + BIT_XOR(cast(b2 as smallint)) AS b5, + BIT_XOR(b3) AS c2, + BIT_XOR(b4) AS d2, + BIT_XOR(distinct b4) AS e2 +FROM bitwise_test +-- !query schema +struct +-- !query output +4 5 2 2 2 + + +-- !query +SELECT bit_xor(b3) FROM bitwise_test GROUP BY b1 & 1 +-- !query schema +struct +-- !query output +4 +6 + + +-- !query +SELECT b1, bit_xor(b2) FROM bitwise_test GROUP BY b1 HAVING bit_and(b2) < 7 +-- !query schema +struct +-- !query output +1 1 +2 3 + + +-- !query +SELECT b1, b2, bit_xor(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test +-- !query schema +struct +-- !query output +1 1 1 +2 3 3 +7 7 7 + + +-- !query +select getbit(11L, 3), getbit(11L, 2), getbit(11L, 1), getbit(11L, 0) +-- !query schema +struct +-- !query output +1 0 1 1 + + +-- !query +select getbit(11L, 2 + 1), getbit(11L, 3 - 1), getbit(10L + 1, 1 * 1), getbit(cast(11L / 1 AS long), 1 - 1) +-- !query schema +struct +-- !query output +1 0 1 1 + + +-- !query +select getbit(11L, 63) +-- !query schema +struct +-- !query output +0 + + +-- !query +select getbit(11L, -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.BIT_POSITION_RANGE", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`getbit`", + "invalidValue" : "-1", + "parameter" : "`pos`", + "upper" : "64" + } +} + + +-- !query +select getbit(11L, 64) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.BIT_POSITION_RANGE", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`getbit`", + "invalidValue" : "64", + "parameter" : "`pos`", + "upper" : "64" + } +} + + +-- !query +SELECT 20181117 >> 2 +-- !query schema +struct<(20181117 >> 2):int> +-- !query output +5045279 + + +-- !query +SELECT 20181117 << 2 +-- !query schema +struct<(20181117 << 2):int> +-- !query output +80724468 + + +-- !query +SELECT 20181117 >>> 2 +-- !query schema +struct<(20181117 >>> 2):int> +-- !query output +5045279 + + +-- !query +SELECT 20181117 > > 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'>'", + "hint" : "" + } +} + + +-- !query +SELECT 20181117 < < 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'<'", + "hint" : "" + } +} + + +-- !query +SELECT 20181117 > >> 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'>>'", + "hint" : "" + } +} + + +-- !query +SELECT 20181117 <<< 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'<'", + "hint" : "" + } +} + + +-- !query +SELECT 20181117 >>>> 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'>'", + "hint" : "" + } +} + + +-- !query +select cast(null as array>), 20181117 >> 2 +-- !query schema +struct>,(20181117 >> 2):int> +-- !query output +NULL 5045279 + + +-- !query +select cast(null as array>), 20181117 >>> 2 +-- !query schema +struct>,(20181117 >>> 2):int> +-- !query output +NULL 5045279 + + +-- !query +select cast(null as map>), 20181117 >> 2 +-- !query schema +struct>,(20181117 >> 2):int> +-- !query output +NULL 5045279 + + +-- !query +select 1 << 1 + 2 as plus_over_shift +-- !query schema +struct +-- !query output +8 + + +-- !query +select 2 >> 1 << 1 as left_to_right +-- !query schema +struct +-- !query output +2 + + +-- !query +select 1 & 2 >> 1 as shift_over_ampersand +-- !query schema +struct +-- !query output +1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cast.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cast.sql.out new file mode 100644 index 000000000000..7dd7180165f2 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cast.sql.out @@ -0,0 +1,2046 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT CAST('1.23' AS int) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1.23'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "CAST('1.23' AS int)" + } ] +} + + +-- !query +SELECT CAST('1.23' AS long) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1.23'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "CAST('1.23' AS long)" + } ] +} + + +-- !query +SELECT CAST('-4.56' AS int) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'-4.56'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "CAST('-4.56' AS int)" + } ] +} + + +-- !query +SELECT CAST('-4.56' AS long) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'-4.56'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "CAST('-4.56' AS long)" + } ] +} + + +-- !query +SELECT CAST('abc' AS int) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'abc'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "CAST('abc' AS int)" + } ] +} + + +-- !query +SELECT CAST('abc' AS long) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'abc'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "CAST('abc' AS long)" + } ] +} + + +-- !query +SELECT CAST('abc' AS float) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'abc'", + "sourceType" : "\"STRING\"", + "targetType" : "\"FLOAT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "CAST('abc' AS float)" + } ] +} + + +-- !query +SELECT CAST('abc' AS double) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'abc'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "CAST('abc' AS double)" + } ] +} + + +-- !query +SELECT CAST('1234567890123' AS int) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1234567890123'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "CAST('1234567890123' AS int)" + } ] +} + + +-- !query +SELECT CAST('12345678901234567890123' AS long) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'12345678901234567890123'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "CAST('12345678901234567890123' AS long)" + } ] +} + + +-- !query +SELECT CAST('' AS int) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "''", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "CAST('' AS int)" + } ] +} + + +-- !query +SELECT CAST('' AS long) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "''", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "CAST('' AS long)" + } ] +} + + +-- !query +SELECT CAST('' AS float) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "''", + "sourceType" : "\"STRING\"", + "targetType" : "\"FLOAT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "CAST('' AS float)" + } ] +} + + +-- !query +SELECT CAST('' AS double) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "''", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "CAST('' AS double)" + } ] +} + + +-- !query +SELECT CAST(NULL AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST(NULL AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('123.a' AS int) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'123.a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "CAST('123.a' AS int)" + } ] +} + + +-- !query +SELECT CAST('123.a' AS long) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'123.a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "CAST('123.a' AS long)" + } ] +} + + +-- !query +SELECT CAST('123.a' AS float) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'123.a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"FLOAT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "CAST('123.a' AS float)" + } ] +} + + +-- !query +SELECT CAST('123.a' AS double) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'123.a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "CAST('123.a' AS double)" + } ] +} + + +-- !query +SELECT CAST('-2147483648' AS int) +-- !query schema +struct +-- !query output +-2147483648 + + +-- !query +SELECT CAST('-2147483649' AS int) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'-2147483649'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "CAST('-2147483649' AS int)" + } ] +} + + +-- !query +SELECT CAST('2147483647' AS int) +-- !query schema +struct +-- !query output +2147483647 + + +-- !query +SELECT CAST('2147483648' AS int) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'2147483648'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "CAST('2147483648' AS int)" + } ] +} + + +-- !query +SELECT CAST('-9223372036854775808' AS long) +-- !query schema +struct +-- !query output +-9223372036854775808 + + +-- !query +SELECT CAST('-9223372036854775809' AS long) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'-9223372036854775809'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "CAST('-9223372036854775809' AS long)" + } ] +} + + +-- !query +SELECT CAST('9223372036854775807' AS long) +-- !query schema +struct +-- !query output +9223372036854775807 + + +-- !query +SELECT CAST('9223372036854775808' AS long) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'9223372036854775808'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "CAST('9223372036854775808' AS long)" + } ] +} + + +-- !query +SELECT HEX(CAST('abc' AS binary)) +-- !query schema +struct +-- !query output +616263 + + +-- !query +SELECT HEX(CAST(CAST(123 AS byte) AS binary)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(CAST(123 AS TINYINT) AS BINARY)\"", + "srcType" : "\"TINYINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 44, + "fragment" : "CAST(CAST(123 AS byte) AS binary)" + } ] +} + + +-- !query +SELECT HEX(CAST(CAST(-123 AS byte) AS binary)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(CAST(-123 AS TINYINT) AS BINARY)\"", + "srcType" : "\"TINYINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 45, + "fragment" : "CAST(CAST(-123 AS byte) AS binary)" + } ] +} + + +-- !query +SELECT HEX(CAST(123S AS binary)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(123 AS BINARY)\"", + "srcType" : "\"SMALLINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 31, + "fragment" : "CAST(123S AS binary)" + } ] +} + + +-- !query +SELECT HEX(CAST(-123S AS binary)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(-123 AS BINARY)\"", + "srcType" : "\"SMALLINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 32, + "fragment" : "CAST(-123S AS binary)" + } ] +} + + +-- !query +SELECT HEX(CAST(123 AS binary)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(123 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 30, + "fragment" : "CAST(123 AS binary)" + } ] +} + + +-- !query +SELECT HEX(CAST(-123 AS binary)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(-123 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 31, + "fragment" : "CAST(-123 AS binary)" + } ] +} + + +-- !query +SELECT HEX(CAST(123L AS binary)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(123 AS BINARY)\"", + "srcType" : "\"BIGINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 31, + "fragment" : "CAST(123L AS binary)" + } ] +} + + +-- !query +SELECT HEX(CAST(-123L AS binary)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(-123 AS BINARY)\"", + "srcType" : "\"BIGINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 32, + "fragment" : "CAST(-123L AS binary)" + } ] +} + + +-- !query +DESC FUNCTION boolean +-- !query schema +struct +-- !query output +Class: org.apache.spark.sql.catalyst.expressions.Cast +Function: boolean +Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`. + + +-- !query +DESC FUNCTION EXTENDED boolean +-- !query schema +struct +-- !query output +Class: org.apache.spark.sql.catalyst.expressions.Cast +Extended Usage: + No example/argument for boolean. + + Since: 2.0.1 + +Function: boolean +Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`. + + +-- !query +SELECT CAST('interval 3 month 1 hour' AS interval) +-- !query schema +struct +-- !query output +3 months 1 hours + + +-- !query +SELECT CAST("interval '3-1' year to month" AS interval year to month) +-- !query schema +struct +-- !query output +3-1 + + +-- !query +SELECT CAST("interval '3 00:00:01' day to second" AS interval day to second) +-- !query schema +struct +-- !query output +3 00:00:01.000000000 + + +-- !query +SELECT CAST(interval 3 month 1 hour AS string) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 3 month 1 hour" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 35, + "fragment" : "interval 3 month 1 hour" + } ] +} + + +-- !query +SELECT CAST(interval 3 year 1 month AS string) +-- !query schema +struct +-- !query output +INTERVAL '3-1' YEAR TO MONTH + + +-- !query +SELECT CAST(interval 3 day 1 second AS string) +-- !query schema +struct +-- !query output +INTERVAL '3 00:00:01' DAY TO SECOND + + +-- !query +select cast(' 1' as tinyint) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast(' 1\t' as tinyint) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast(' 1' as smallint) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast(' 1' as INT) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast(' 1' as bigint) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast(' 1' as float) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +select cast(' 1 ' as DOUBLE) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +select cast('1.0 ' as DEC) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast('1中文' as tinyint) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1中文'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TINYINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "cast('1中文' as tinyint)" + } ] +} + + +-- !query +select cast('1中文' as smallint) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1中文'", + "sourceType" : "\"STRING\"", + "targetType" : "\"SMALLINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "cast('1中文' as smallint)" + } ] +} + + +-- !query +select cast('1中文' as INT) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1中文'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "cast('1中文' as INT)" + } ] +} + + +-- !query +select cast('中文1' as bigint) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'中文1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "cast('中文1' as bigint)" + } ] +} + + +-- !query +select cast('1中文' as bigint) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1中文'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "cast('1中文' as bigint)" + } ] +} + + +-- !query +select cast('\t\t true \n\r ' as boolean) +-- !query schema +struct +-- !query output +true + + +-- !query +select cast('\t\n false \t\r' as boolean) +-- !query schema +struct +-- !query output +false + + +-- !query +select cast('\t\n xyz \t\r' as boolean) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'\t\n xyz \t\r'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "cast('\\t\\n xyz \\t\\r' as boolean)" + } ] +} + + +-- !query +select cast('23.45' as decimal(4, 2)) +-- !query schema +struct +-- !query output +23.45 + + +-- !query +select cast('123.45' as decimal(4, 2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "4", + "scale" : "2", + "value" : "123.45" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "cast('123.45' as decimal(4, 2))" + } ] +} + + +-- !query +select cast('xyz' as decimal(4, 2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'xyz'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DECIMAL(4,2)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "cast('xyz' as decimal(4, 2))" + } ] +} + + +-- !query +select cast('2022-01-01' as date) +-- !query schema +struct +-- !query output +2022-01-01 + + +-- !query +select cast('a' as date) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast('a' as date)" + } ] +} + + +-- !query +select cast('2022-01-01 00:00:00' as timestamp) +-- !query schema +struct +-- !query output +2022-01-01 00:00:00 + + +-- !query +select cast('a' as timestamp) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "cast('a' as timestamp)" + } ] +} + + +-- !query +select cast('2022-01-01 00:00:00' as timestamp_ntz) +-- !query schema +struct +-- !query output +2022-01-01 00:00:00 + + +-- !query +select cast('a' as timestamp_ntz) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP_NTZ\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast('a' as timestamp_ntz)" + } ] +} + + +-- !query +select cast(cast('inf' as double) as timestamp) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "Infinity", + "sourceType" : "\"DOUBLE\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast(cast('inf' as double) as timestamp)" + } ] +} + + +-- !query +select cast(cast('inf' as float) as timestamp) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "Infinity", + "sourceType" : "\"DOUBLE\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(cast('inf' as float) as timestamp)" + } ] +} + + +-- !query +select cast(interval '1' year as tinyint) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast(interval '-10-2' year to month as smallint) +-- !query schema +struct +-- !query output +-122 + + +-- !query +select cast(interval '1000' month as int) +-- !query schema +struct +-- !query output +1000 + + +-- !query +select cast(interval -'10.123456' second as tinyint) +-- !query schema +struct +-- !query output +-10 + + +-- !query +select cast(interval '23:59:59' hour to second as smallint) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"INTERVAL HOUR TO SECOND\"", + "targetType" : "\"SMALLINT\"", + "value" : "INTERVAL '23:59:59' HOUR TO SECOND" + } +} + + +-- !query +select cast(interval -'1 02:03:04.123' day to second as int) +-- !query schema +struct +-- !query output +-93784 + + +-- !query +select cast(interval '10' day as bigint) +-- !query schema +struct +-- !query output +10 + + +-- !query +select cast(interval '-1000' month as tinyint) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"INTERVAL MONTH\"", + "targetType" : "\"TINYINT\"", + "value" : "INTERVAL '-1000' MONTH" + } +} + + +-- !query +select cast(interval '1000000' second as smallint) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"INTERVAL SECOND\"", + "targetType" : "\"SMALLINT\"", + "value" : "INTERVAL '1000000' SECOND" + } +} + + +-- !query +select cast(1Y as interval year) +-- !query schema +struct +-- !query output +1-0 + + +-- !query +select cast(-122S as interval year to month) +-- !query schema +struct +-- !query output +-10-2 + + +-- !query +select cast(ym as interval year to month) from values(-122S) as t(ym) +-- !query schema +struct +-- !query output +-10-2 + + +-- !query +select cast(1000 as interval month) +-- !query schema +struct +-- !query output +83-4 + + +-- !query +select cast(-10L as interval second) +-- !query schema +struct +-- !query output +-0 00:00:10.000000000 + + +-- !query +select cast(100Y as interval hour to second) +-- !query schema +struct +-- !query output +0 00:01:40.000000000 + + +-- !query +select cast(dt as interval hour to second) from values(100Y) as t(dt) +-- !query schema +struct +-- !query output +0 00:01:40.000000000 + + +-- !query +select cast(-1000S as interval day to second) +-- !query schema +struct +-- !query output +-0 00:16:40.000000000 + + +-- !query +select cast(10 as interval day) +-- !query schema +struct +-- !query output +10 00:00:00.000000000 + + +-- !query +select cast(2147483647 as interval year) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"INT\"", + "targetType" : "\"INTERVAL YEAR\"", + "value" : "2147483647" + } +} + + +-- !query +select cast(-9223372036854775808L as interval day) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"BIGINT\"", + "targetType" : "\"INTERVAL DAY\"", + "value" : "-9223372036854775808L" + } +} + + +-- !query +select cast(interval '-1' year as decimal(10, 0)) +-- !query schema +struct +-- !query output +-1 + + +-- !query +select cast(interval '1.000001' second as decimal(10, 6)) +-- !query schema +struct +-- !query output +1.000001 + + +-- !query +select cast(interval '08:11:10.001' hour to second as decimal(10, 4)) +-- !query schema +struct +-- !query output +29470.0010 + + +-- !query +select cast(interval '1 01:02:03.1' day to second as decimal(8, 1)) +-- !query schema +struct +-- !query output +90123.1 + + +-- !query +select cast(interval '10.123' second as decimal(4, 2)) +-- !query schema +struct +-- !query output +10.12 + + +-- !query +select cast(interval '10.005' second as decimal(4, 2)) +-- !query schema +struct +-- !query output +10.01 + + +-- !query +select cast(interval '10.123' second as decimal(5, 2)) +-- !query schema +struct +-- !query output +10.12 + + +-- !query +select cast(interval '10.123' second as decimal(1, 0)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "1", + "scale" : "0", + "value" : "10.123000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(interval '10.123' second as decimal(1, 0))" + } ] +} + + +-- !query +select cast(10.123456BD as interval day to second) +-- !query schema +struct +-- !query output +0 00:00:10.123456000 + + +-- !query +select cast(80.654321BD as interval hour to minute) +-- !query schema +struct +-- !query output +0 01:20:00.000000000 + + +-- !query +select cast(-10.123456BD as interval year to month) +-- !query schema +struct +-- !query output +-0-10 + + +-- !query +select cast(10.654321BD as interval month) +-- !query schema +struct +-- !query output +0-11 + + +-- !query +SELECT '1.23' :: int +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1.23'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "'1.23' :: int" + } ] +} + + +-- !query +SELECT 'abc' :: int +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'abc'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "'abc' :: int" + } ] +} + + +-- !query +SELECT '12345678901234567890123' :: long +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'12345678901234567890123'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "'12345678901234567890123' :: long" + } ] +} + + +-- !query +SELECT '' :: int +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "''", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "'' :: int" + } ] +} + + +-- !query +SELECT NULL :: int +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT '123.a' :: int +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'123.a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "'123.a' :: int" + } ] +} + + +-- !query +SELECT '-2147483648' :: int +-- !query schema +struct +-- !query output +-2147483648 + + +-- !query +SELECT HEX('abc' :: binary) +-- !query schema +struct +-- !query output +616263 + + +-- !query +SELECT HEX((123 :: byte) :: binary) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(CAST(123 AS TINYINT) AS BINARY)\"", + "srcType" : "\"TINYINT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 34, + "fragment" : "(123 :: byte) :: binary" + } ] +} + + +-- !query +SELECT 'interval 3 month 1 hour' :: interval +-- !query schema +struct +-- !query output +3 months 1 hours + + +-- !query +SELECT interval 3 day 1 second :: string +-- !query schema +struct +-- !query output +INTERVAL '3 00:00:01' DAY TO SECOND + + +-- !query +select ' 1 ' :: DOUBLE +-- !query schema +struct +-- !query output +1.0 + + +-- !query +select '1.0 ' :: DEC +-- !query schema +struct +-- !query output +1 + + +-- !query +select '\t\t true \n\r ' :: boolean +-- !query schema +struct +-- !query output +true + + +-- !query +select '2022-01-01 00:00:00' :: timestamp +-- !query schema +struct +-- !query output +2022-01-01 00:00:00 + + +-- !query +select interval '-10-2' year to month :: smallint +-- !query schema +struct +-- !query output +-122 + + +-- !query +select -10L :: interval second +-- !query schema +struct +-- !query output +-0 00:00:10.000000000 + + +-- !query +select interval '08:11:10.001' hour to second :: decimal(10, 4) +-- !query schema +struct +-- !query output +29470.0010 + + +-- !query +select 10.123456BD :: interval day to second +-- !query schema +struct +-- !query output +0 00:00:10.123456000 + + +-- !query +SELECT '1.23' :: int :: long +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1.23'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "'1.23' :: int" + } ] +} + + +-- !query +SELECT '2147483648' :: long :: int +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"BIGINT\"", + "targetType" : "\"INT\"", + "value" : "2147483648L" + } +} + + +-- !query +SELECT CAST('2147483648' :: long AS int) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"BIGINT\"", + "targetType" : "\"INT\"", + "value" : "2147483648L" + } +} + + +-- !query +SELECT map(1, '123', 2, '456')[1] :: int +-- !query schema +struct +-- !query output +123 + + +-- !query +SELECT '2147483648' :: BINT +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_DATATYPE", + "sqlState" : "0A000", + "messageParameters" : { + "typeName" : "\"BINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 27, + "fragment" : "BINT" + } ] +} + + +-- !query +SELECT '2147483648' :: SELECT +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_DATATYPE", + "sqlState" : "0A000", + "messageParameters" : { + "typeName" : "\"SELECT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 29, + "fragment" : "SELECT" + } ] +} + + +-- !query +SELECT FALSE IS NOT NULL :: string +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'::'", + "hint" : "" + } +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/ceil-floor-with-scale-param.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/ceil-floor-with-scale-param.sql.out new file mode 100644 index 000000000000..86f54665ad01 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/ceil-floor-with-scale-param.sql.out @@ -0,0 +1,321 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT CEIL(2.5, 0) +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT CEIL(3.5, 0) +-- !query schema +struct +-- !query output +4 + + +-- !query +SELECT CEIL(-2.5, 0) +-- !query schema +struct +-- !query output +-2 + + +-- !query +SELECT CEIL(-3.5, 0) +-- !query schema +struct +-- !query output +-3 + + +-- !query +SELECT CEIL(-0.35, 1) +-- !query schema +struct +-- !query output +-0.3 + + +-- !query +SELECT CEIL(-35, -1) +-- !query schema +struct +-- !query output +-30 + + +-- !query +SELECT CEIL(-0.1, 0) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT CEIL(5, 0) +-- !query schema +struct +-- !query output +5 + + +-- !query +SELECT CEIL(3.14115, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT CEIL(9.9, 0) +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT CEIL(CAST(99 AS DECIMAL(2, 0)), -1) +-- !query schema +struct +-- !query output +100 + + +-- !query +SELECT CEIL(2.5, null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NON_FOLDABLE_ARGUMENT", + "sqlState" : "42K08", + "messageParameters" : { + "funcName" : "`ceil`", + "paramName" : "`scale`", + "paramType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "CEIL(2.5, null)" + } ] +} + + +-- !query +SELECT CEIL(2.5, 'a') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NON_FOLDABLE_ARGUMENT", + "sqlState" : "42K08", + "messageParameters" : { + "funcName" : "`ceil`", + "paramName" : "`scale`", + "paramType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "CEIL(2.5, 'a')" + } ] +} + + +-- !query +SELECT CEIL(2.5, 0, 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "2", + "functionName" : "`ceil`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "CEIL(2.5, 0, 0)" + } ] +} + + +-- !query +SELECT FLOOR(2.5, 0) +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT FLOOR(3.5, 0) +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT FLOOR(-2.5, 0) +-- !query schema +struct +-- !query output +-3 + + +-- !query +SELECT FLOOR(-3.5, 0) +-- !query schema +struct +-- !query output +-4 + + +-- !query +SELECT FLOOR(-0.35, 1) +-- !query schema +struct +-- !query output +-0.4 + + +-- !query +SELECT FLOOR(-35, -1) +-- !query schema +struct +-- !query output +-40 + + +-- !query +SELECT FLOOR(-0.1, 0) +-- !query schema +struct +-- !query output +-1 + + +-- !query +SELECT FLOOR(5, 0) +-- !query schema +struct +-- !query output +5 + + +-- !query +SELECT FLOOR(3.14115, -3) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT FLOOR(-9.9, 0) +-- !query schema +struct +-- !query output +-10 + + +-- !query +SELECT FLOOR(CAST(-99 AS DECIMAL(2, 0)), -1) +-- !query schema +struct +-- !query output +-100 + + +-- !query +SELECT FLOOR(2.5, null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NON_FOLDABLE_ARGUMENT", + "sqlState" : "42K08", + "messageParameters" : { + "funcName" : "`floor`", + "paramName" : "`scale`", + "paramType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "FLOOR(2.5, null)" + } ] +} + + +-- !query +SELECT FLOOR(2.5, 'a') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NON_FOLDABLE_ARGUMENT", + "sqlState" : "42K08", + "messageParameters" : { + "funcName" : "`floor`", + "paramName" : "`scale`", + "paramType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "FLOOR(2.5, 'a')" + } ] +} + + +-- !query +SELECT FLOOR(2.5, 0, 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "2", + "functionName" : "`floor`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "FLOOR(2.5, 0, 0)" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/change-column.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/change-column.sql.out new file mode 100644 index 000000000000..a6110543159a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/change-column.sql.out @@ -0,0 +1,341 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE test_change(a INT, b STRING, c INT) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC test_change +-- !query schema +struct +-- !query output +a int +b string +c int + + +-- !query +ALTER TABLE test_change CHANGE a +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0035", + "messageParameters" : { + "message" : "ALTER TABLE table CHANGE COLUMN requires a TYPE, a SET/DROP, a COMMENT, or a FIRST/AFTER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 32, + "fragment" : "ALTER TABLE test_change CHANGE a" + } ] +} + + +-- !query +DESC test_change +-- !query schema +struct +-- !query output +a int +b string +c int + + +-- !query +ALTER TABLE test_change RENAME COLUMN a TO a1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "RENAME COLUMN", + "tableName" : "`spark_catalog`.`default`.`test_change`" + } +} + + +-- !query +DESC test_change +-- !query schema +struct +-- !query output +a int +b string +c int + + +-- !query +ALTER TABLE test_change CHANGE a TYPE STRING +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_SUPPORTED_CHANGE_COLUMN", + "sqlState" : "0A000", + "messageParameters" : { + "newName" : "`a`", + "newType" : "\"STRING\"", + "originName" : "`a`", + "originType" : "\"INT\"", + "table" : "`spark_catalog`.`default`.`test_change`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 44, + "fragment" : "ALTER TABLE test_change CHANGE a TYPE STRING" + } ] +} + + +-- !query +DESC test_change +-- !query schema +struct +-- !query output +a int +b string +c int + + +-- !query +ALTER TABLE test_change CHANGE a AFTER b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "ALTER COLUMN ... FIRST | AFTER", + "tableName" : "`spark_catalog`.`default`.`test_change`" + } +} + + +-- !query +ALTER TABLE test_change CHANGE b FIRST +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "ALTER COLUMN ... FIRST | AFTER", + "tableName" : "`spark_catalog`.`default`.`test_change`" + } +} + + +-- !query +DESC test_change +-- !query schema +struct +-- !query output +a int +b string +c int + + +-- !query +ALTER TABLE test_change CHANGE a COMMENT 'this is column a' +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER TABLE test_change CHANGE b COMMENT '#*02?`' +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER TABLE test_change CHANGE c COMMENT '' +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC test_change +-- !query schema +struct +-- !query output +a int this is column a +b string #*02?` +c int + + +-- !query +ALTER TABLE test_change CHANGE a TYPE INT +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER TABLE test_change CHANGE a COMMENT 'this is column a' +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC test_change +-- !query schema +struct +-- !query output +a int this is column a +b string #*02?` +c int + + +-- !query +ALTER TABLE test_change CHANGE invalid_col TYPE INT +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`invalid_col`", + "proposal" : "`a`, `b`, `c`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 51, + "fragment" : "ALTER TABLE test_change CHANGE invalid_col TYPE INT" + } ] +} + + +-- !query +DESC test_change +-- !query schema +struct +-- !query output +a int this is column a +b string #*02?` +c int + + +-- !query +ALTER TABLE test_change CHANGE A COMMENT 'case insensitivity' +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC test_change +-- !query schema +struct +-- !query output +a int case insensitivity +b string #*02?` +c int + + +-- !query +CREATE TEMPORARY VIEW temp_view(a, b) AS SELECT 1, "one" +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER TABLE temp_view CHANGE a TYPE INT +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + "sqlState" : "42809", + "messageParameters" : { + "operation" : "ALTER TABLE ... CHANGE COLUMN", + "viewName" : "`temp_view`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 21, + "fragment" : "temp_view" + } ] +} + + +-- !query +CREATE GLOBAL TEMPORARY VIEW global_temp_view(a, b) AS SELECT 1, "one" +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER TABLE global_temp.global_temp_view CHANGE a TYPE INT +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE", + "sqlState" : "42809", + "messageParameters" : { + "operation" : "ALTER TABLE ... CHANGE COLUMN", + "viewName" : "`global_temp`.`global_temp_view`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 40, + "fragment" : "global_temp.global_temp_view" + } ] +} + + +-- !query +DROP TABLE test_change +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW global_temp.global_temp_view +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/charvarchar.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/charvarchar.sql.out new file mode 100644 index 000000000000..bbb58936f5b8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/charvarchar.sql.out @@ -0,0 +1,1273 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create table char_tbl(c char(5), v varchar(6)) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_tbl +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_tbl +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_tbl + + +-- !query +desc formatted char_tbl c +-- !query schema +struct +-- !query output +col_name c +data_type char(5) +comment NULL +min NULL +max NULL +num_nulls NULL +distinct_count NULL +avg_col_len NULL +max_col_len NULL +histogram NULL + + +-- !query +show create table char_tbl +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.default.char_tbl ( + c CHAR(5), + v VARCHAR(6)) +USING parquet + + +-- !query +create table char_tbl2 using parquet as select * from char_tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +show create table char_tbl2 +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.default.char_tbl2 ( + c CHAR(5), + v VARCHAR(6)) +USING parquet + + +-- !query +desc formatted char_tbl2 +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_tbl2 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_tbl2 + + +-- !query +desc formatted char_tbl2 c +-- !query schema +struct +-- !query output +col_name c +data_type char(5) +comment NULL +min NULL +max NULL +num_nulls NULL +distinct_count NULL +avg_col_len NULL +max_col_len NULL +histogram NULL + + +-- !query +create table char_tbl3 like char_tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_tbl3 +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_tbl3 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_tbl3 + + +-- !query +desc formatted char_tbl3 c +-- !query schema +struct +-- !query output +col_name c +data_type char(5) +comment NULL +min NULL +max NULL +num_nulls NULL +distinct_count NULL +avg_col_len NULL +max_col_len NULL +histogram NULL + + +-- !query +show create table char_tbl3 +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.default.char_tbl3 ( + c CHAR(5), + v VARCHAR(6)) +USING parquet + + +-- !query +create view char_view as select * from char_tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_view +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_view +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text select * from char_tbl +View Original Text select * from char_tbl +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c`, `v`] + + +-- !query +desc formatted char_view c +-- !query schema +struct +-- !query output +col_name c +data_type char(5) +comment NULL +min NULL +max NULL +num_nulls NULL +distinct_count NULL +avg_col_len NULL +max_col_len NULL +histogram NULL + + +-- !query +show create table char_view +-- !query schema +struct +-- !query output +CREATE VIEW default.char_view ( + c, + v) +WITH SCHEMA COMPENSATION +AS select * from char_tbl + + +-- !query +alter table char_tbl rename to char_tbl1 +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_tbl1 +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_tbl1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_tbl1 + + +-- !query +alter table char_tbl1 change column c type char(6) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_SUPPORTED_CHANGE_COLUMN", + "sqlState" : "0A000", + "messageParameters" : { + "newName" : "`c`", + "newType" : "\"CHAR(6)\"", + "originName" : "`c`", + "originType" : "\"CHAR(5)\"", + "table" : "`spark_catalog`.`default`.`char_tbl1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 50, + "fragment" : "alter table char_tbl1 change column c type char(6)" + } ] +} + + +-- !query +alter table char_tbl1 change column c type char(5) +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_tbl1 +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_tbl1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_tbl1 + + +-- !query +alter table char_tbl1 add columns (d char(5)) +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_tbl1 +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) +d char(5) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_tbl1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_tbl1 + + +-- !query +alter view char_view as select * from char_tbl2 +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_view +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_view +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text select * from char_tbl2 +View Original Text select * from char_tbl2 +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c`, `v`] + + +-- !query +alter table char_tbl1 SET TBLPROPERTIES('yes'='no') +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_tbl1 +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) +d char(5) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_tbl1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Table Properties [yes=no] +Location [not included in comparison]/{warehouse_dir}/char_tbl1 + + +-- !query +alter view char_view SET TBLPROPERTIES('yes'='no') +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_view +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_view +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text select * from char_tbl2 +View Original Text select * from char_tbl2 +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c`, `v`] +Table Properties [yes=no] + + +-- !query +alter table char_tbl1 UNSET TBLPROPERTIES('yes') +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_tbl1 +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) +d char(5) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_tbl1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_tbl1 + + +-- !query +alter view char_view UNSET TBLPROPERTIES('yes') +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_view +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_view +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text select * from char_tbl2 +View Original Text select * from char_tbl2 +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c`, `v`] + + +-- !query +alter table char_tbl1 SET SERDEPROPERTIES('yes'='no') +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_tbl1 +-- !query schema +struct +-- !query output +c char(5) +v varchar(6) +d char(5) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_tbl1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_tbl1 +Storage Properties [yes=no] + + +-- !query +create table char_part(c1 char(5), c2 char(2), v1 varchar(6), v2 varchar(2)) using parquet partitioned by (v2, c2) +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_part +-- !query schema +struct +-- !query output +c1 char(5) +v1 varchar(6) +v2 varchar(2) +c2 char(2) +# Partition Information +# col_name data_type comment +v2 varchar(2) +c2 char(2) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_part +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_part +Partition Provider Catalog + + +-- !query +alter table char_part change column c1 comment 'char comment' +-- !query schema +struct<> +-- !query output + + + +-- !query +alter table char_part change column v1 comment 'varchar comment' +-- !query schema +struct<> +-- !query output + + + +-- !query +alter table char_part add partition (v2='ke', c2='nt') location 'loc1' +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_part +-- !query schema +struct +-- !query output +c1 char(5) char comment +v1 varchar(6) varchar comment +v2 varchar(2) +c2 char(2) +# Partition Information +# col_name data_type comment +v2 varchar(2) +c2 char(2) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_part +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_part +Partition Provider Catalog + + +-- !query +alter table char_part partition (v2='ke') rename to partition (v2='nt') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1232", + "messageParameters" : { + "partitionColumnNames" : "v2, c2", + "specKeys" : "v2", + "tableName" : "`spark_catalog`.`default`.`char_part`" + } +} + + +-- !query +desc formatted char_part +-- !query schema +struct +-- !query output +c1 char(5) char comment +v1 varchar(6) varchar comment +v2 varchar(2) +c2 char(2) +# Partition Information +# col_name data_type comment +v2 varchar(2) +c2 char(2) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_part +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_part +Partition Provider Catalog + + +-- !query +alter table char_part partition (v2='ke', c2='nt') set location 'loc2' +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_part +-- !query schema +struct +-- !query output +c1 char(5) char comment +v1 varchar(6) varchar comment +v2 varchar(2) +c2 char(2) +# Partition Information +# col_name data_type comment +v2 varchar(2) +c2 char(2) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_part +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_part +Partition Provider Catalog + + +-- !query +MSCK REPAIR TABLE char_part +-- !query schema +struct<> +-- !query output + + + +-- !query +desc formatted char_part +-- !query schema +struct +-- !query output +c1 char(5) char comment +v1 varchar(6) varchar comment +v2 varchar(2) +c2 char(2) +# Partition Information +# col_name data_type comment +v2 varchar(2) +c2 char(2) + +# Detailed Table Information +Catalog spark_catalog +Database default +Table char_part +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/char_part +Partition Provider Catalog + + +-- !query +create temporary view str_view as select c, v from values + (null, null), + (null, 'S'), + ('N', 'N '), + ('Ne', 'Sp'), + ('Net ', 'Spa '), + ('NetE', 'Spar'), + ('NetEa ', 'Spark '), + ('NetEas ', 'Spark'), + ('NetEase', 'Spark-') t(c, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table char_tbl4(c7 char(7), c8 char(8), v varchar(6), s string) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into char_tbl4 select c, c, v, c from str_view +-- !query schema +struct<> +-- !query output + + + +-- !query +select c7, c8, v, s from char_tbl4 +-- !query schema +struct +-- !query output +N N N N +NULL NULL NULL NULL +NULL NULL S NULL +Ne Ne Sp Ne +Net Net Spa Net +NetE NetE Spar NetE +NetEa NetEa Spark NetEa +NetEas NetEas Spark NetEas +NetEase NetEase Spark- NetEase + + +-- !query +select c7, c8, v, s from char_tbl4 where c7 = c8 +-- !query schema +struct +-- !query output +N N N N +Ne Ne Sp Ne +Net Net Spa Net +NetE NetE Spar NetE +NetEa NetEa Spark NetEa +NetEas NetEas Spark NetEas +NetEase NetEase Spark- NetEase + + +-- !query +select c7, c8, v, s from char_tbl4 where c7 = v +-- !query schema +struct +-- !query output + + + +-- !query +select c7, c8, v, s from char_tbl4 where c7 = s +-- !query schema +struct +-- !query output +NetEas NetEas Spark NetEas +NetEase NetEase Spark- NetEase + + +-- !query +select c7, c8, v, s from char_tbl4 where c7 = 'NetEase ' +-- !query schema +struct +-- !query output +NetEase NetEase Spark- NetEase + + +-- !query +select c7, c8, v, s from char_tbl4 where v = 'Spark ' +-- !query schema +struct +-- !query output +NetEa NetEa Spark NetEa + + +-- !query +select c7, c8, v, s from char_tbl4 order by c7 +-- !query schema +struct +-- !query output +NULL NULL NULL NULL +NULL NULL S NULL +N N N N +Ne Ne Sp Ne +Net Net Spa Net +NetE NetE Spar NetE +NetEa NetEa Spark NetEa +NetEas NetEas Spark NetEas +NetEase NetEase Spark- NetEase + + +-- !query +select c7, c8, v, s from char_tbl4 order by v +-- !query schema +struct +-- !query output +NULL NULL NULL NULL +N N N N +NULL NULL S NULL +Ne Ne Sp Ne +Net Net Spa Net +NetE NetE Spar NetE +NetEas NetEas Spark NetEas +NetEa NetEa Spark NetEa +NetEase NetEase Spark- NetEase + + +-- !query +select ascii(c7), ascii(c8), ascii(v), ascii(s) from char_tbl4 +-- !query schema +struct +-- !query output +78 78 78 78 +78 78 83 78 +78 78 83 78 +78 78 83 78 +78 78 83 78 +78 78 83 78 +78 78 83 78 +NULL NULL 83 NULL +NULL NULL NULL NULL + + +-- !query +select base64(c7), base64(c8), base64(v), ascii(s) from char_tbl4 +-- !query schema +struct +-- !query output +NULL NULL NULL NULL +NULL NULL Uw== NULL +TiAgICAgIA== TiAgICAgICA= TiA= 78 +TmUgICAgIA== TmUgICAgICA= U3A= 78 +TmV0ICAgIA== TmV0ICAgICA= U3BhICA= 78 +TmV0RSAgIA== TmV0RSAgICA= U3Bhcg== 78 +TmV0RWEgIA== TmV0RWEgICA= U3Bhcmsg 78 +TmV0RWFzIA== TmV0RWFzICA= U3Bhcms= 78 +TmV0RWFzZQ== TmV0RWFzZSA= U3Bhcmst 78 + + +-- !query +select bit_length(c7), bit_length(c8), bit_length(v), bit_length(s) from char_tbl4 +-- !query schema +struct +-- !query output +56 64 16 16 +56 64 16 8 +56 64 32 32 +56 64 40 40 +56 64 40 56 +56 64 48 48 +56 64 48 56 +NULL NULL 8 NULL +NULL NULL NULL NULL + + +-- !query +select char_length(c7), char_length(c8), char_length(v), char_length(s) from char_tbl4 +-- !query schema +struct +-- !query output +7 8 2 1 +7 8 2 2 +7 8 4 4 +7 8 5 5 +7 8 5 7 +7 8 6 6 +7 8 6 7 +NULL NULL 1 NULL +NULL NULL NULL NULL + + +-- !query +select octet_length(c7), octet_length(c8), octet_length(v), octet_length(s) from char_tbl4 +-- !query schema +struct +-- !query output +7 8 2 1 +7 8 2 2 +7 8 4 4 +7 8 5 5 +7 8 5 7 +7 8 6 6 +7 8 6 7 +NULL NULL 1 NULL +NULL NULL NULL NULL + + +-- !query +select concat_ws('|', c7, c8), concat_ws('|', c7, v), concat_ws('|', c7, s), concat_ws('|', v, s) from char_tbl4 +-- !query schema +struct +-- !query output + + S S +N |N N |N N |N N |N +Ne |Ne Ne |Sp Ne |Ne Sp|Ne +Net |Net Net |Spa Net |Net Spa |Net +NetE |NetE NetE |Spar NetE |NetE Spar|NetE +NetEa |NetEa NetEa |Spark NetEa |NetEa Spark |NetEa +NetEas |NetEas NetEas |Spark NetEas |NetEas Spark|NetEas +NetEase|NetEase NetEase|Spark- NetEase|NetEase Spark-|NetEase + + +-- !query +select concat(c7, c8), concat(c7, v), concat(c7, s), concat(v, s) from char_tbl4 +-- !query schema +struct +-- !query output +N N N N N N N N +NULL NULL NULL NULL +NULL NULL NULL NULL +Ne Ne Ne Sp Ne Ne SpNe +Net Net Net Spa Net Net Spa Net +NetE NetE NetE Spar NetE NetE SparNetE +NetEa NetEa NetEa Spark NetEa NetEa Spark NetEa +NetEas NetEas NetEas Spark NetEas NetEas SparkNetEas +NetEaseNetEase NetEaseSpark- NetEaseNetEase Spark-NetEase + + +-- !query +select like(c7, 'Ne _'), like(c8, 'Ne _') from char_tbl4 +-- !query schema +struct +-- !query output +NULL NULL +NULL NULL +false false +false false +false false +false false +false false +false false +false true + + +-- !query +select like(v, 'Spark_') from char_tbl4 +-- !query schema +struct +-- !query output +NULL +false +false +false +false +false +false +true +true + + +-- !query +select c7 = c8, upper(c7) = upper(c8), lower(c7) = lower(c8) from char_tbl4 where s = 'NetEase' +-- !query schema +struct<(c7 = c8):boolean,(upper(c7) = upper(c8)):boolean,(lower(c7) = lower(c8)):boolean> +-- !query output +true false false + + +-- !query +select c7 = s, upper(c7) = upper(s), lower(c7) = lower(s) from char_tbl4 where s = 'NetEase' +-- !query schema +struct<(c7 = s):boolean,(upper(c7) = upper(s)):boolean,(lower(c7) = lower(s)):boolean> +-- !query output +true true true + + +-- !query +select c7 = 'NetEase', upper(c7) = upper('NetEase'), lower(c7) = lower('NetEase') from char_tbl4 where s = 'NetEase' +-- !query schema +struct<(c7 = NetEase):boolean,(upper(c7) = upper(NetEase)):boolean,(lower(c7) = lower(NetEase)):boolean> +-- !query output +true true true + + +-- !query +select printf('Hey, %s%s%s%s', c7, c8, v, s) from char_tbl4 +-- !query schema +struct +-- !query output +Hey, N N N N +Hey, Ne Ne SpNe +Hey, Net Net Spa Net +Hey, NetE NetE SparNetE +Hey, NetEa NetEa Spark NetEa +Hey, NetEas NetEas SparkNetEas +Hey, NetEaseNetEase Spark-NetEase +Hey, nullnullSnull +Hey, nullnullnullnull + + +-- !query +select repeat(c7, 2), repeat(c8, 2), repeat(v, 2), repeat(s, 2) from char_tbl4 +-- !query schema +struct +-- !query output +N N N N N N NN +NULL NULL NULL NULL +NULL NULL SS NULL +Ne Ne Ne Ne SpSp NeNe +Net Net Net Net Spa Spa Net Net +NetE NetE NetE NetE SparSpar NetENetE +NetEa NetEa NetEa NetEa Spark Spark NetEa NetEa +NetEas NetEas NetEas NetEas SparkSpark NetEas NetEas +NetEaseNetEase NetEase NetEase Spark-Spark- NetEaseNetEase + + +-- !query +select replace(c7, 'Net', 'Apache'), replace(c8, 'Net', 'Apache'), replace(v, 'Spark', 'Kyuubi'), replace(s, 'Net', 'Apache') from char_tbl4 +-- !query schema +struct +-- !query output +Apache Apache Spa Apache +ApacheE ApacheE Spar ApacheE +ApacheEa ApacheEa Kyuubi ApacheEa +ApacheEas ApacheEas Kyuubi ApacheEas +ApacheEase ApacheEase Kyuubi- ApacheEase +N N N N +NULL NULL NULL NULL +NULL NULL S NULL +Ne Ne Sp Ne + + +-- !query +select rpad(c7, 10), rpad(c8, 5), rpad(v, 5), rpad(s, 5) from char_tbl4 +-- !query schema +struct +-- !query output +N N N N +NULL NULL NULL NULL +NULL NULL S NULL +Ne Ne Sp Ne +Net Net Spa Net +NetE NetE Spar NetE +NetEa NetEa Spark NetEa +NetEas NetEa Spark NetEa +NetEase NetEa Spark NetEa + + +-- !query +select rtrim(c7), rtrim(c8), rtrim(v), rtrim(s) from char_tbl4 +-- !query schema +struct +-- !query output +N N N N +NULL NULL NULL NULL +NULL NULL S NULL +Ne Ne Sp Ne +Net Net Spa Net +NetE NetE Spar NetE +NetEa NetEa Spark NetEa +NetEas NetEas Spark NetEas +NetEase NetEase Spark- NetEase + + +-- !query +select split(c7, 'e'), split(c8, 'e'), split(v, 'a'), split(s, 'e') from char_tbl4 +-- !query schema +struct,split(c8, e, -1):array,split(v, a, -1):array,split(s, e, -1):array> +-- !query output +NULL NULL NULL NULL +NULL NULL ["S"] NULL +["N "] ["N "] ["N "] ["N"] +["N"," "] ["N"," "] ["Sp"] ["N",""] +["N","t "] ["N","t "] ["Sp"," "] ["N","t "] +["N","tE "] ["N","tE "] ["Sp","r"] ["N","tE"] +["N","tEa "] ["N","tEa "] ["Sp","rk "] ["N","tEa "] +["N","tEas "] ["N","tEas "] ["Sp","rk"] ["N","tEas "] +["N","tEas",""] ["N","tEas"," "] ["Sp","rk-"] ["N","tEas",""] + + +-- !query +select substring(c7, 2), substring(c8, 2), substring(v, 3), substring(s, 2) from char_tbl4 +-- !query schema +struct +-- !query output + +NULL NULL NULL +NULL NULL NULL NULL +e e e +et et a et +etE etE ar etE +etEa etEa ark etEa +etEas etEas ark etEas +etEase etEase ark- etEase + + +-- !query +select left(c7, 2), left(c8, 2), left(v, 3), left(s, 2) from char_tbl4 +-- !query schema +struct +-- !query output +N N N N +NULL NULL NULL NULL +NULL NULL S NULL +Ne Ne Sp Ne +Ne Ne Spa Ne +Ne Ne Spa Ne +Ne Ne Spa Ne +Ne Ne Spa Ne +Ne Ne Spa Ne + + +-- !query +select right(c7, 2), right(c8, 2), right(v, 3), right(s, 2) from char_tbl4 +-- !query schema +struct +-- !query output + N N + Sp Ne + a + par tE + rk a +NULL NULL NULL NULL +NULL NULL S NULL +s ark s +se e rk- se + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +select typeof(c7), typeof(c8), typeof(v), typeof(s) from char_tbl4 limit 1 +-- !query schema +struct +-- !query output +string string string string + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query + +select cast(c7 as char(1)), cast(c8 as char(10)), cast(v as char(1)), cast(v as varchar(1)), cast(s as char(5)) from char_tbl4 +-- !query schema +struct +-- !query output +N N N N N +NULL NULL NULL NULL NULL +NULL NULL S S NULL +Ne Ne Sp Sp Ne +Net Net Spa Spa Net +NetE NetE Spar Spar NetE +NetEa NetEa Spark Spark NetEa +NetEas NetEas Spark Spark NetEas +NetEase NetEase Spark- Spark- NetEase + + +-- !query +drop table char_tbl1 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table char_tbl2 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table char_tbl3 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table char_tbl4 +-- !query schema +struct<> +-- !query output + + + +-- !query +select ascii('§'), ascii('÷'), ascii('×10') +-- !query schema +struct +-- !query output +167 247 215 + + +-- !query +select chr(167), chr(247), chr(215) +-- !query schema +struct +-- !query output +§ ÷ × + + +-- !query +SELECT to_varchar(78.12, '$99.99') +-- !query schema +struct +-- !query output +$78.12 + + +-- !query +SELECT to_varchar(111.11, '99.9') +-- !query schema +struct +-- !query output +##.# + + +-- !query +SELECT to_varchar(12454.8, '99,999.9S') +-- !query schema +struct +-- !query output +12,454.8+ diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/collations.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/collations.sql.out new file mode 100644 index 000000000000..2055ec0c2c80 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/collations.sql.out @@ -0,0 +1,5834 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create table t1(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t1 values('aaa', 'aaa') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t1 values('AAA', 'AAA') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t1 values('bbb', 'bbb') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t1 values('BBB', 'BBB') +-- !query schema +struct<> +-- !query output + + + +-- !query +describe table t1 +-- !query schema +struct +-- !query output +utf8_binary string +utf8_lcase string collate UTF8_LCASE + + +-- !query +select count(*) from t1 group by utf8_binary +-- !query schema +struct +-- !query output +1 +1 +1 +1 + + +-- !query +select count(*) from t1 group by utf8_lcase +-- !query schema +struct +-- !query output +2 +2 + + +-- !query +select * from t1 where utf8_binary = 'aaa' +-- !query schema +struct +-- !query output +aaa aaa + + +-- !query +select * from t1 where utf8_lcase = 'aaa' collate utf8_lcase +-- !query schema +struct +-- !query output +AAA AAA +aaa aaa + + +-- !query +select * from t1 where utf8_binary < 'bbb' +-- !query schema +struct +-- !query output +AAA AAA +BBB BBB +aaa aaa + + +-- !query +select * from t1 where utf8_lcase < 'bbb' collate utf8_lcase +-- !query schema +struct +-- !query output +AAA AAA +aaa aaa + + +-- !query +select l.utf8_binary, r.utf8_lcase from t1 l join t1 r on l.utf8_lcase = r.utf8_lcase +-- !query schema +struct +-- !query output +AAA AAA +AAA aaa +BBB BBB +BBB bbb +aaa AAA +aaa aaa +bbb BBB +bbb bbb + + +-- !query +create table t2(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t2 values('aaa', 'aaa') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t2 values('bbb', 'bbb') +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from t1 anti join t2 on t1.utf8_lcase = t2.utf8_lcase +-- !query schema +struct +-- !query output + + + +-- !query +drop table t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate utf8_lcase from values ('aaa'), ('bbb') +-- !query schema +struct +-- !query output +zzz + + +-- !query +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate utf8_lcase from values ('aaa'), ('bbb') +-- !query schema +struct +-- !query output +aaa +bbb +zzz +zzz + + +-- !query +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate utf8_lcase from values ('aaa'), ('bbb') +-- !query schema +struct +-- !query output +aaa +bbb +zzz + + +-- !query +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate utf8_lcase from values ('aaa'), ('bbb') +-- !query schema +struct +-- !query output +AAA +BBB +ZZZ +aaa +aaa +bbb +bbb +zzz + + +-- !query +select col1 collate utf8_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate utf8_lcase from values ('aaa'), ('bbb') +-- !query schema +struct +-- !query output +aaa +bbb + + +-- !query +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate unicode_ci from values ('aaa'), ('bbb') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"STRING COLLATE UNICODE_CI\"", + "dataType2" : "\"STRING COLLATE UTF8_LCASE\"", + "hint" : "", + "operator" : "EXCEPT", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 162, + "fragment" : "select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate unicode_ci from values ('aaa'), ('bbb')" + } ] +} + + +-- !query +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate unicode_ci from values ('aaa'), ('bbb') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"STRING COLLATE UNICODE_CI\"", + "dataType2" : "\"STRING COLLATE UTF8_LCASE\"", + "hint" : "", + "operator" : "EXCEPT ALL", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 166, + "fragment" : "select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate unicode_ci from values ('aaa'), ('bbb')" + } ] +} + + +-- !query +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate unicode_ci from values ('aaa'), ('bbb') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"STRING COLLATE UNICODE_CI\"", + "dataType2" : "\"STRING COLLATE UTF8_LCASE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 161, + "fragment" : "select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate unicode_ci from values ('aaa'), ('bbb')" + } ] +} + + +-- !query +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate unicode_ci from values ('aaa'), ('bbb') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"STRING COLLATE UNICODE_CI\"", + "dataType2" : "\"STRING COLLATE UTF8_LCASE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 165, + "fragment" : "select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate unicode_ci from values ('aaa'), ('bbb')" + } ] +} + + +-- !query +select col1 collate utf8_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate unicode_ci from values ('aaa'), ('bbb') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"STRING COLLATE UNICODE_CI\"", + "dataType2" : "\"STRING COLLATE UTF8_LCASE\"", + "hint" : "", + "operator" : "INTERSECT", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 156, + "fragment" : "select col1 collate utf8_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate unicode_ci from values ('aaa'), ('bbb')" + } ] +} + + +-- !query +create table t1 (c1 struct) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t1 values (named_struct('utf8_binary', 'aaa', 'utf8_lcase', 'aaa')) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t1 values (named_struct('utf8_binary', 'AAA', 'utf8_lcase', 'AAA')) +-- !query schema +struct<> +-- !query output + + + +-- !query +select count(*) from t1 group by c1.utf8_binary +-- !query schema +struct +-- !query output +1 +1 + + +-- !query +select count(*) from t1 group by c1.utf8_lcase +-- !query schema +struct +-- !query output +2 + + +-- !query +drop table t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +select array_contains(ARRAY('aaa' collate utf8_lcase),'AAA' collate utf8_lcase) +-- !query schema +struct +-- !query output +true + + +-- !query +select array_position(ARRAY('aaa' collate utf8_lcase, 'bbb' collate utf8_lcase),'BBB' collate utf8_lcase) +-- !query schema +struct +-- !query output +2 + + +-- !query +select nullif('aaa' COLLATE utf8_lcase, 'AAA' COLLATE utf8_lcase) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select least('aaa' COLLATE utf8_lcase, 'AAA' collate utf8_lcase, 'a' collate utf8_lcase) +-- !query schema +struct +-- !query output +a + + +-- !query +select arrays_overlap(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)) +-- !query schema +struct +-- !query output +true + + +-- !query +select array_distinct(array('aaa' collate utf8_lcase, 'AAA' collate utf8_lcase)) +-- !query schema +struct> +-- !query output +["aaa"] + + +-- !query +select array_union(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)) +-- !query schema +struct> +-- !query output +["aaa"] + + +-- !query +select array_intersect(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)) +-- !query schema +struct> +-- !query output +["aaa"] + + +-- !query +select array_except(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)) +-- !query schema +struct> +-- !query output +[] + + +-- !query +select 'a' collate unicode < 'A' +-- !query schema +struct<(collate(a, unicode) < 'A' collate UNICODE):boolean> +-- !query output +true + + +-- !query +select 'a' collate unicode_ci = 'A' +-- !query schema +struct<(collate(a, unicode_ci) = 'A' collate UNICODE_CI):boolean> +-- !query output +true + + +-- !query +select 'a' collate unicode_ai = 'å' +-- !query schema +struct<(collate(a, unicode_ai) = 'å' collate UNICODE_AI):boolean> +-- !query output +true + + +-- !query +select 'a' collate unicode_ci_ai = 'Å' +-- !query schema +struct<(collate(a, unicode_ci_ai) = 'Å' collate UNICODE_CI_AI):boolean> +-- !query output +true + + +-- !query +select 'a' collate en < 'A' +-- !query schema +struct<(collate(a, en) < 'A' collate en):boolean> +-- !query output +true + + +-- !query +select 'a' collate en_ci = 'A' +-- !query schema +struct<(collate(a, en_ci) = 'A' collate en_CI):boolean> +-- !query output +true + + +-- !query +select 'a' collate en_ai = 'å' +-- !query schema +struct<(collate(a, en_ai) = 'å' collate en_AI):boolean> +-- !query output +true + + +-- !query +select 'a' collate en_ci_ai = 'Å' +-- !query schema +struct<(collate(a, en_ci_ai) = 'Å' collate en_CI_AI):boolean> +-- !query output +true + + +-- !query +select 'Kypper' collate sv < 'Köpfe' +-- !query schema +struct<(collate(Kypper, sv) < 'Köpfe' collate sv):boolean> +-- !query output +true + + +-- !query +select 'Kypper' collate de > 'Köpfe' +-- !query schema +struct<(collate(Kypper, de) > 'Köpfe' collate de):boolean> +-- !query output +true + + +-- !query +select 'I' collate tr_ci = 'ı' +-- !query schema +struct<(collate(I, tr_ci) = 'ı' collate tr_CI):boolean> +-- !query output +true + + +-- !query +create table t4 (text string collate utf8_binary, pairDelim string collate utf8_lcase, keyValueDelim string collate utf8_binary) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t4 values('a:1,b:2,c:3', ',', ':') +-- !query schema +struct<> +-- !query output + + + +-- !query +select str_to_map(text, pairDelim, keyValueDelim) from t4 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INDETERMINATE_COLLATION", + "sqlState" : "42P22" +} + + +-- !query +select str_to_map(text collate utf8_binary, pairDelim collate utf8_lcase, keyValueDelim collate utf8_binary) from t4 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\"" + } +} + + +-- !query +select str_to_map(text collate utf8_binary, pairDelim collate utf8_binary, keyValueDelim collate utf8_binary) from t4 +-- !query schema +struct> +-- !query output +{"a":"1","b":"2","c":"3"} + + +-- !query +select str_to_map(text collate unicode_ai, pairDelim collate unicode_ai, keyValueDelim collate unicode_ai) from t4 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"collate(text, unicode_ai)\"", + "inputType" : "\"STRING COLLATE UNICODE_AI\"", + "paramIndex" : "first", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"str_to_map(collate(text, unicode_ai), collate(pairDelim, unicode_ai), collate(keyValueDelim, unicode_ai))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 106, + "fragment" : "str_to_map(text collate unicode_ai, pairDelim collate unicode_ai, keyValueDelim collate unicode_ai)" + } ] +} + + +-- !query +drop table t4 +-- !query schema +struct<> +-- !query output + + + +-- !query +create table t5(s string, utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('Spark', 'Spark', 'SQL') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaAAaA') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaA') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('aaAaAAaA', 'aaAaAAaA', 'aaAaaAaAaaAaaAaAaaAaaAaA') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('bbAbaAbA', 'bbAbAAbA', 'a') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('İo', 'İo', 'İo') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('İo', 'İo', 'İo ') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('İo', 'İo ', 'İo') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('İo', 'İo', 'i̇o') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('efd2', 'efd2', 'efd2') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('Hello, world! Nice day.', 'Hello, world! Nice day.', 'Hello, world! Nice day.') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('Something else. Nothing here.', 'Something else. Nothing here.', 'Something else. Nothing here.') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('kitten', 'kitten', 'sitTing') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('abc', 'abc', 'abc') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t5 values ('abcdcba', 'abcdcba', 'aBcDCbA') +-- !query schema +struct<> +-- !query output + + + +-- !query +create table t6(ascii long) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t6 values (97) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t6 values (66) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table t7(ascii double) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t7 values (97.52143) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t7 values (66.421) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table t8(format string collate utf8_binary, utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t8 values ('%s%s', 'abCdE', 'abCdE') +-- !query schema +struct<> +-- !query output + + + +-- !query +create table t9(num long) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t9 values (97) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t9 values (66) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table t10(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t10 values ('aaAaAAaA', 'aaAaaAaA') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t10 values ('efd2', 'efd2') +-- !query schema +struct<> +-- !query output + + + +-- !query +select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +SQL SQL +Something else. Nothing here. Something else. Nothing here. +a a +aBcDCbA aBcDCbA +aaAaAAaA aaAaAAaA +aaAaaAaA aaAaaAaA +aaAaaAaAaaAaaAaAaaAaaAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +efd2 efd2 +i̇o i̇o +sitTing sitTing +İo İo +İo İo +İo İo + + +-- !query +select concat_ws(' ', utf8_binary, utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo İo +İo i̇o +İo İo +İo İo + + +-- !query +select concat_ws(' ' collate utf8_binary, utf8_binary, 'SQL' collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\"" + } +} + + +-- !query +select concat_ws(' ' collate utf8_lcase, utf8_binary, 'SQL' collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. SQL +Something else. Nothing here. SQL +Spark SQL +aaAaAAaA SQL +aaAaAAaA SQL +aaAaAAaA SQL +abc SQL +abcdcba SQL +bbAbAAbA SQL +efd2 SQL +kitten SQL +İo SQL +İo SQL +İo SQL +İo SQL + + +-- !query +select concat_ws(',', utf8_lcase, 'word'), concat_ws(',', utf8_binary, 'word') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day.,word Hello, world! Nice day.,word +SQL,word Spark,word +Something else. Nothing here.,word Something else. Nothing here.,word +a,word bbAbAAbA,word +aBcDCbA,word abcdcba,word +aaAaAAaA,word aaAaAAaA,word +aaAaaAaA,word aaAaAAaA,word +aaAaaAaAaaAaaAaAaaAaaAaA,word aaAaAAaA,word +abc,word abc,word +efd2,word efd2,word +i̇o,word İo,word +sitTing,word kitten,word +İo ,word İo,word +İo,word İo ,word +İo,word İo,word + + +-- !query +select concat_ws(',', utf8_lcase, 'word' collate utf8_binary), concat_ws(',', utf8_binary, 'word' collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day.,word Hello, world! Nice day.,word +SQL,word Spark,word +Something else. Nothing here.,word Something else. Nothing here.,word +a,word bbAbAAbA,word +aBcDCbA,word abcdcba,word +aaAaAAaA,word aaAaAAaA,word +aaAaaAaA,word aaAaAAaA,word +aaAaaAaAaaAaaAaAaaAaaAaA,word aaAaAAaA,word +abc,word abc,word +efd2,word efd2,word +i̇o,word İo,word +sitTing,word kitten,word +İo ,word İo,word +İo,word İo ,word +İo,word İo,word + + +-- !query +select elt(2, s, utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. +Something else. Nothing here. +Spark +aaAaAAaA +aaAaAAaA +aaAaAAaA +abc +abcdcba +bbAbAAbA +efd2 +kitten +İo +İo +İo +İo + + +-- !query +select elt(2, utf8_binary, utf8_lcase, s) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. +SQL +Something else. Nothing here. +a +aBcDCbA +aaAaAAaA +aaAaaAaA +aaAaaAaAaaAaaAaAaaAaaAaA +abc +efd2 +i̇o +sitTing +İo +İo +İo + + +-- !query +select elt(1, utf8_binary collate utf8_binary, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\"" + } +} + + +-- !query +select elt(1, utf8_binary collate utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. +Something else. Nothing here. +Spark +aaAaAAaA +aaAaAAaA +aaAaAAaA +abc +abcdcba +bbAbAAbA +efd2 +kitten +İo +İo +İo +İo + + +-- !query +select elt(1, utf8_binary collate utf8_binary, utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. +Something else. Nothing here. +Spark +aaAaAAaA +aaAaAAaA +aaAaAAaA +abc +abcdcba +bbAbAAbA +efd2 +kitten +İo +İo +İo +İo + + +-- !query +select elt(1, utf8_binary, 'word'), elt(1, utf8_lcase, 'word') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select elt(1, utf8_binary, 'word' collate utf8_lcase), elt(1, utf8_lcase, 'word' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select split_part(utf8_binary, utf8_lcase, 3) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INDETERMINATE_COLLATION", + "sqlState" : "42P22" +} + + +-- !query +select split_part(s, utf8_binary, 1) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + + + + +bbAbaAbA +İo + + +-- !query +select split_part(utf8_binary collate utf8_binary, s collate utf8_lcase, 1) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\"" + } +} + + +-- !query +select split_part(utf8_binary, utf8_lcase collate utf8_binary, 2) from t5 +-- !query schema +struct +-- !query output + + + +-- !query +select split_part(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + + + + + +b + + +-- !query +select split_part(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 2) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"collate(utf8_binary, unicode_ai)\"", + "inputType" : "\"STRING COLLATE UNICODE_AI\"", + "paramIndex" : "first", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"split_part(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai), 2)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 83, + "fragment" : "split_part(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 2)" + } ] +} + + +-- !query +select split_part(utf8_binary, 'a', 3), split_part(utf8_lcase, 'a', 3) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + + + +A +A +A + + +-- !query +select split_part(utf8_binary, 'a' collate utf8_lcase, 3), split_part(utf8_lcase, 'a' collate utf8_binary, 3) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + + + + A + A + A + + +-- !query +select split_part(utf8_binary, 'a ' collate utf8_lcase_rtrim, 3), split_part(utf8_lcase, 'a' collate utf8_binary, 3) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + + + + A + A + A + + +-- !query +select contains(utf8_binary, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION", + "sqlState" : "42P22", + "messageParameters" : { + "expr" : "\"contains(utf8_binary, utf8_lcase)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "contains(utf8_binary, utf8_lcase)" + } ] +} + + +-- !query +select contains(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output +false +false +true +true +true +true +true +true +true +true +true +true +true +true +true + + +-- !query +select contains(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\"" + } +} + + +-- !query +select contains(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +false +false +false +false +false +false +false +false +true +true +true +true +true +true +true + + +-- !query +select contains(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +false +false +false +false +true +true +true +true +true +true +true +true +true +true +true + + +-- !query +select contains(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"collate(utf8_binary, unicode_ai)\"", + "inputType" : "\"STRING COLLATE UNICODE_AI\"", + "paramIndex" : "first", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"contains(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "contains(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)" + } ] +} + + +-- !query +select contains(utf8_binary, 'a'), contains(utf8_lcase, 'a') from t5 +-- !query schema +struct +-- !query output +false false +false false +false false +false false +false false +false false +false false +false true +true false +true true +true true +true true +true true +true true +true true + + +-- !query +select contains(utf8_binary, 'AaAA' collate utf8_lcase), contains(utf8_lcase, 'AAa' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +true false +true false +true true + + +-- !query +select contains(utf8_binary, 'AaAA ' collate utf8_lcase_rtrim), contains(utf8_lcase, 'AAa ' collate utf8_binary_rtrim) from t5 +-- !query schema +struct +-- !query output +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +true false +true false +true true + + +-- !query +select substring_index(utf8_binary, utf8_lcase, 2) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION", + "sqlState" : "42P22", + "messageParameters" : { + "expr" : "\"substring_index(utf8_binary, utf8_lcase, 2)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "substring_index(utf8_binary, utf8_lcase, 2)" + } ] +} + + +-- !query +select substring_index(s, utf8_binary,1) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + + + + +bbAbaAbA +İo + + +-- !query +select substring_index(utf8_binary collate utf8_binary, s collate utf8_lcase, 3) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\"" + } +} + + +-- !query +select substring_index(utf8_binary, utf8_lcase collate utf8_binary, 2) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. +Something else. Nothing here. +Spark +aaAaAAaA +aaAaAAaA +aaAaAAaA +abc +abcdcba +bbAbAAbA +efd2 +kitten +İo +İo +İo +İo + + +-- !query +select substring_index(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. +Something else. Nothing here. +Spark +aaAaAAaA +aaAaAAaA +aaAaAAaA +abc +abcdcba +bbAb +efd2 +kitten +İo +İo +İo +İo + + +-- !query +select substring_index(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 2) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"collate(utf8_binary, unicode_ai)\"", + "inputType" : "\"STRING COLLATE UNICODE_AI\"", + "paramIndex" : "first", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"substring_index(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai), 2)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 88, + "fragment" : "substring_index(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 2)" + } ] +} + + +-- !query +select substring_index(utf8_binary, 'a', 2), substring_index(utf8_lcase, 'a', 2) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +a a +a a +a a +abc abc +abcdcb aBcDCb +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select substring_index(utf8_binary, 'AaAA' collate utf8_lcase, 2), substring_index(utf8_lcase, 'AAa' collate utf8_binary, 2) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +a aaAaAAaA +a aaAaaAaA +a aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select substring_index(utf8_binary, 'AaAA ' collate utf8_lcase_rtrim, 2), substring_index(utf8_lcase, 'AAa' collate utf8_binary, 2) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +a aaAaAAaA +a aaAaaAaA +a aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select instr(utf8_binary, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION", + "sqlState" : "42P22", + "messageParameters" : { + "expr" : "\"instr(utf8_binary, utf8_lcase)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "instr(utf8_binary, utf8_lcase)" + } ] +} + + +-- !query +select instr(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + + +-- !query +select instr(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\"" + } +} + + +-- !query +select instr(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +0 +1 +1 +1 +1 +1 +1 +1 + + +-- !query +select instr(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +3 + + +-- !query +select instr(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"collate(utf8_binary, unicode_ai)\"", + "inputType" : "\"STRING COLLATE UNICODE_AI\"", + "paramIndex" : "first", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"instr(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "instr(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)" + } ] +} + + +-- !query +select instr(utf8_binary, 'a'), instr(utf8_lcase, 'a') from t5 +-- !query schema +struct +-- !query output +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 1 +1 1 +1 1 +1 1 +1 1 +1 1 +21 21 +3 0 + + +-- !query +select instr(utf8_binary, 'AaAA' collate utf8_lcase), instr(utf8_lcase, 'AAa' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +1 0 +1 0 +1 5 + + +-- !query +select find_in_set(utf8_binary, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION", + "sqlState" : "42P22", + "messageParameters" : { + "expr" : "\"find_in_set(utf8_binary, utf8_lcase)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "find_in_set(utf8_binary, utf8_lcase)" + } ] +} + + +-- !query +select find_in_set(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + + +-- !query +select find_in_set(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\"" + } +} + + +-- !query +select find_in_set(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +1 +1 +1 +1 + + +-- !query +select find_in_set(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +1 +1 +1 +1 +1 +1 +1 +1 + + +-- !query +select find_in_set(utf8_binary, 'aaAaaAaA,i̇o'), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o') from t5 +-- !query schema +struct +-- !query output +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 1 +0 1 +0 2 +0 2 +0 2 + + +-- !query +select find_in_set(utf8_binary, 'aaAaaAaA,i̇o' collate utf8_lcase), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +1 0 +1 0 +1 1 +2 0 +2 0 +2 2 + + +-- !query +select find_in_set(utf8_binary, 'aaAaaAaA,i̇o ' collate utf8_lcase_rtrim), find_in_set(utf8_lcase, 'aaAaaAaA,i̇o' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +1 0 +1 0 +1 1 +2 0 +2 0 +2 0 +2 2 + + +-- !query +select startswith(utf8_binary, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION", + "sqlState" : "42P22", + "messageParameters" : { + "expr" : "\"startswith(utf8_binary, utf8_lcase)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "startswith(utf8_binary, utf8_lcase)" + } ] +} + + +-- !query +select startswith(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output +false +false +true +true +true +true +true +true +true +true +true +true +true +true +true + + +-- !query +select startswith(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\"" + } +} + + +-- !query +select startswith(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +false +false +false +false +false +false +false +false +true +true +true +true +true +true +true + + +-- !query +select startswith(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +false +false +false +false +false +true +true +true +true +true +true +true +true +true +true + + +-- !query +select startswith(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"collate(utf8_binary, unicode_ai)\"", + "inputType" : "\"STRING COLLATE UNICODE_AI\"", + "paramIndex" : "first", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"startswith(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "startswith(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)" + } ] +} + + +-- !query +select startswith(utf8_binary, 'aaAaaAaA'), startswith(utf8_lcase, 'aaAaaAaA') from t5 +-- !query schema +struct +-- !query output +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false true +false true +false true + + +-- !query +select startswith(utf8_binary, 'aaAaaAaA' collate utf8_lcase), startswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +true false +true true +true true + + +-- !query +select startswith(utf8_binary, 'aaAaaAaA ' collate utf8_lcase_rtrim), startswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +true false +true true +true true + + +-- !query +select translate(utf8_lcase, utf8_lcase, '12345') from t5 +-- !query schema +struct +-- !query output +1 +11111111 +11111111 +111111111111111111111111 +12 +12 +123 +123 +123 +123 +12332 +12335532 +1234 +1234321 +123454142544 + + +-- !query +select translate(utf8_binary, utf8_lcase, '12345') from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION", + "sqlState" : "42P22", + "messageParameters" : { + "expr" : "\"translate(utf8_binary, utf8_lcase, 12345)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "translate(utf8_binary, utf8_lcase, '12345')" + } ] +} + + +-- !query +select translate(utf8_binary, 'aBc' collate utf8_lcase, '12345' collate utf8_binary) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING COLLATE UTF8_LCASE\", \"STRING\"" + } +} + + +-- !query +select translate(utf8_binary, 'SQL' collate utf8_lcase, '12345' collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +1omething e31e. Nothing here. +1park +He33o, wor3d! Nice day. +aaAaAAaA +aaAaAAaA +aaAaAAaA +abc +abcdcba +bbAbAAbA +efd2 +kitten +İo +İo +İo +İo + + +-- !query +select translate(utf8_binary, 'SQL' collate unicode_ai, '12345' collate unicode_ai) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"utf8_binary\"", + "inputType" : "\"STRING COLLATE UNICODE_AI\"", + "paramIndex" : "first", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"translate(utf8_binary, collate(SQL, unicode_ai), collate(12345, unicode_ai))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 83, + "fragment" : "translate(utf8_binary, 'SQL' collate unicode_ai, '12345' collate unicode_ai)" + } ] +} + + +-- !query +select translate(utf8_lcase, 'aaAaaAaA', '12345'), translate(utf8_binary, 'aaAaaAaA', '12345') from t5 +-- !query schema +struct +-- !query output +1 bb3b33b3 +11111111 11313313 +11111111 11313313 +111111111111111111111111 11313313 +1BcDCb1 1bcdcb1 +1bc 1bc +Hello, world! Nice d1y. Hello, world! Nice d1y. +SQL Sp1rk +Something else. Nothing here. Something else. Nothing here. +efd2 efd2 +i̇o İo +sitTing kitten +İo İo +İo İo +İo İo + + +-- !query +select translate(utf8_lcase, 'aBc' collate utf8_binary, '12345'), translate(utf8_binary, 'aBc' collate utf8_lcase, '12345') from t5 +-- !query schema +struct +-- !query output +1 22121121 +11A11A1A 11111111 +11A11A1A11A11A1A11A11A1A 11111111 +11A1AA1A 11111111 +123DCbA 123d321 +1b3 123 +Hello, world! Ni3e d1y. Hello, world! Ni3e d1y. +SQL Sp1rk +Something else. Nothing here. Something else. Nothing here. +efd2 efd2 +i̇o İo +sitTing kitten +İo İo +İo İo +İo İo + + +-- !query +select translate(utf8_lcase, 'aBc ' collate utf8_binary_rtrim, '12345'), translate(utf8_binary, 'aBc' collate utf8_lcase, '12345') from t5 +-- !query schema +struct +-- !query output +1 22121121 +11A11A1A 11111111 +11A11A1A11A11A1A11A11A1A 11111111 +11A1AA1A 11111111 +123DCbA 123d321 +1b3 123 +Hello,4world!4Ni3e4d1y. Hello, world! Ni3e d1y. +SQL Sp1rk +Something4else.4Nothing4here. Something else. Nothing here. +efd2 efd2 +i̇o İo +sitTing kitten +İo İo +İo İo +İo4 İo + + +-- !query +select replace(utf8_binary, utf8_lcase, 'abc') from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION", + "sqlState" : "42P22", + "messageParameters" : { + "expr" : "\"replace(utf8_binary, utf8_lcase, abc)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "replace(utf8_binary, utf8_lcase, 'abc')" + } ] +} + + +-- !query +select replace(s, utf8_binary, 'abc') from t5 +-- !query schema +struct +-- !query output +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +bbAbaAbA +İo + + +-- !query +select replace(utf8_binary collate utf8_binary, s collate utf8_lcase, 'abc') from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\"" + } +} + + +-- !query +select replace(utf8_binary, utf8_lcase collate utf8_binary, 'abc') from t5 +-- !query schema +struct +-- !query output +Spark +aaAaAAaA +aaAaAAaA +abc +abc +abc +abc +abc +abc +abc +abcdcba +bbAbAAbA +kitten +İo +İo + + +-- !query +select replace(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 'abc') from t5 +-- !query schema +struct +-- !query output +Spark +aaAaAAaA +abc +abc +abc +abc +abc +abc +abc +abc +abc +abc +bbabcbabcabcbabc +kitten +İo + + +-- !query +select replace(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 'abc') from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"collate(utf8_binary, unicode_ai)\"", + "inputType" : "\"STRING COLLATE UNICODE_AI\"", + "paramIndex" : "first", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"replace(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai), 'abc' collate UNICODE_AI)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 84, + "fragment" : "replace(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 'abc')" + } ] +} + + +-- !query +select replace(utf8_binary, 'aaAaaAaA', 'abc'), replace(utf8_lcase, 'aaAaaAaA', 'abc') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA abc +aaAaAAaA abc +aaAaAAaA abcabcabc +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select replace(utf8_binary, 'aaAaaAaA' collate utf8_lcase, 'abc'), replace(utf8_lcase, 'aaAaaAaA' collate utf8_binary, 'abc') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +abc aaAaAAaA +abc abc +abc abc +abc abcabcabc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select replace(utf8_binary, 'aaAaaAaA ' collate utf8_lcase_rtrim, 'abc'), replace(utf8_lcase, 'aaAaaAaA' collate utf8_binary, 'abc') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA abc +aaAaAAaA abcabcabc +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select endswith(utf8_binary, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION", + "sqlState" : "42P22", + "messageParameters" : { + "expr" : "\"endswith(utf8_binary, utf8_lcase)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "endswith(utf8_binary, utf8_lcase)" + } ] +} + + +-- !query +select endswith(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output +false +false +true +true +true +true +true +true +true +true +true +true +true +true +true + + +-- !query +select endswith(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\"" + } +} + + +-- !query +select endswith(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +false +false +false +false +false +false +false +false +false +true +true +true +true +true +true + + +-- !query +select endswith(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +false +false +false +false +false +true +true +true +true +true +true +true +true +true +true + + +-- !query +select endswith(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"collate(utf8_binary, unicode_ai)\"", + "inputType" : "\"STRING COLLATE UNICODE_AI\"", + "paramIndex" : "first", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"endswith(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "endswith(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)" + } ] +} + + +-- !query +select endswith(utf8_binary, 'aaAaaAaA'), endswith(utf8_lcase, 'aaAaaAaA') from t5 +-- !query schema +struct +-- !query output +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false true +false true +false true + + +-- !query +select endswith(utf8_binary, 'aaAaaAaA' collate utf8_lcase), endswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +true false +true true +true true + + +-- !query +select endswith(utf8_binary, 'aaAaaAaA ' collate utf8_lcase_rtrim), endswith(utf8_lcase, 'aaAaaAaA' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +false false +true false +true true +true true + + +-- !query +select repeat(utf8_binary, 3), repeat(utf8_lcase, 2) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day.Hello, world! Nice day.Hello, world! Nice day. Hello, world! Nice day.Hello, world! Nice day. +Something else. Nothing here.Something else. Nothing here.Something else. Nothing here. Something else. Nothing here.Something else. Nothing here. +SparkSparkSpark SQLSQL +aaAaAAaAaaAaAAaAaaAaAAaA aaAaAAaAaaAaAAaA +aaAaAAaAaaAaAAaAaaAaAAaA aaAaaAaAaaAaaAaA +aaAaAAaAaaAaAAaAaaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaAaaAaaAaAaaAaaAaAaaAaaAaA +abcabcabc abcabc +abcdcbaabcdcbaabcdcba aBcDCbAaBcDCbA +bbAbAAbAbbAbAAbAbbAbAAbA aa +efd2efd2efd2 efd2efd2 +kittenkittenkitten sitTingsitTing +İo İo İo İoİo +İoİoİo i̇oi̇o +İoİoİo İo İo +İoİoİo İoİo + + +-- !query +select repeat(utf8_binary collate utf8_lcase, 3), repeat(utf8_lcase collate utf8_binary, 2) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day.Hello, world! Nice day.Hello, world! Nice day. Hello, world! Nice day.Hello, world! Nice day. +Something else. Nothing here.Something else. Nothing here.Something else. Nothing here. Something else. Nothing here.Something else. Nothing here. +SparkSparkSpark SQLSQL +aaAaAAaAaaAaAAaAaaAaAAaA aaAaAAaAaaAaAAaA +aaAaAAaAaaAaAAaAaaAaAAaA aaAaaAaAaaAaaAaA +aaAaAAaAaaAaAAaAaaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaAaaAaaAaAaaAaaAaAaaAaaAaA +abcabcabc abcabc +abcdcbaabcdcbaabcdcba aBcDCbAaBcDCbA +bbAbAAbAbbAbAAbAbbAbAAbA aa +efd2efd2efd2 efd2efd2 +kittenkittenkitten sitTingsitTing +İo İo İo İoİo +İoİoİo i̇oi̇o +İoİoİo İo İo +İoİoİo İoİo + + +-- !query +select ascii(utf8_binary), ascii(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +101 101 +107 115 +304 105 +304 304 +304 304 +304 304 +72 72 +83 83 +83 83 +97 97 +97 97 +97 97 +97 97 +97 97 +98 97 + + +-- !query +select ascii(utf8_binary collate utf8_lcase), ascii(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +101 101 +107 115 +304 105 +304 304 +304 304 +304 304 +72 72 +83 83 +83 83 +97 97 +97 97 +97 97 +97 97 +97 97 +98 97 + + +-- !query +select unbase64(utf8_binary), unbase64(utf8_lcase) from t10 +-- !query schema +struct +-- !query output +i�� i�h� +y�v y�v + + +-- !query +select unbase64(utf8_binary collate utf8_lcase), unbase64(utf8_lcase collate utf8_binary) from t10 +-- !query schema +struct +-- !query output +i�� i�h� +y�v y�v + + +-- !query +select chr(ascii) from t6 +-- !query schema +struct +-- !query output +B +a + + +-- !query +select base64(utf8_binary), base64(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +SGVsbG8sIHdvcmxkISBOaWNlIGRheS4= SGVsbG8sIHdvcmxkISBOaWNlIGRheS4= +U29tZXRoaW5nIGVsc2UuIE5vdGhpbmcgaGVyZS4= U29tZXRoaW5nIGVsc2UuIE5vdGhpbmcgaGVyZS4= +U3Bhcms= U1FM +YWFBYUFBYUE= YWFBYUFBYUE= +YWFBYUFBYUE= YWFBYWFBYUE= +YWFBYUFBYUE= YWFBYWFBYUFhYUFhYUFhQWFhQWFhQWFB +YWJj YWJj +YWJjZGNiYQ== YUJjRENiQQ== +YmJBYkFBYkE= YQ== +ZWZkMg== ZWZkMg== +a2l0dGVu c2l0VGluZw== +xLBv acyHbw== +xLBv xLBv +xLBv xLBvIA== +xLBvIA== xLBv + + +-- !query +select base64(utf8_binary collate utf8_lcase), base64(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +SGVsbG8sIHdvcmxkISBOaWNlIGRheS4= SGVsbG8sIHdvcmxkISBOaWNlIGRheS4= +U29tZXRoaW5nIGVsc2UuIE5vdGhpbmcgaGVyZS4= U29tZXRoaW5nIGVsc2UuIE5vdGhpbmcgaGVyZS4= +U3Bhcms= U1FM +YWFBYUFBYUE= YWFBYUFBYUE= +YWFBYUFBYUE= YWFBYWFBYUE= +YWFBYUFBYUE= YWFBYWFBYUFhYUFhYUFhQWFhQWFhQWFB +YWJj YWJj +YWJjZGNiYQ== YUJjRENiQQ== +YmJBYkFBYkE= YQ== +ZWZkMg== ZWZkMg== +a2l0dGVu c2l0VGluZw== +xLBv acyHbw== +xLBv xLBv +xLBv xLBvIA== +xLBvIA== xLBv + + +-- !query +select decode(encode(utf8_binary, 'utf-8'), 'utf-8'), decode(encode(utf8_lcase, 'utf-8'), 'utf-8') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select decode(encode(utf8_binary collate utf8_lcase, 'utf-8'), 'utf-8'), decode(encode(utf8_lcase collate utf8_binary, 'utf-8'), 'utf-8') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select format_number(ascii, '###.###') from t7 +-- !query schema +struct +-- !query output +66.421 +97.521 + + +-- !query +select format_number(ascii, '###.###' collate utf8_lcase) from t7 +-- !query schema +struct +-- !query output +66.421 +97.521 + + +-- !query +select encode(utf8_binary, 'utf-8'), encode(utf8_lcase, 'utf-8') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select encode(utf8_binary collate utf8_lcase, 'utf-8'), encode(utf8_lcase collate utf8_binary, 'utf-8') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select to_binary(utf8_binary, 'utf-8'), to_binary(utf8_lcase, 'utf-8') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select to_binary(utf8_binary collate utf8_lcase, 'utf-8'), to_binary(utf8_lcase collate utf8_binary, 'utf-8') from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select sentences(utf8_binary), sentences(utf8_lcase) from t5 +-- !query schema +struct>,sentences(utf8_lcase, , ):array>> +-- !query output +[["Hello","world"],["Nice","day"]] [["Hello","world"],["Nice","day"]] +[["Something","else"],["Nothing","here"]] [["Something","else"],["Nothing","here"]] +[["Spark"]] [["SQL"]] +[["aaAaAAaA"]] [["aaAaAAaA"]] +[["aaAaAAaA"]] [["aaAaaAaA"]] +[["aaAaAAaA"]] [["aaAaaAaAaaAaaAaAaaAaaAaA"]] +[["abc"]] [["abc"]] +[["abcdcba"]] [["aBcDCbA"]] +[["bbAbAAbA"]] [["a"]] +[["efd2"]] [["efd2"]] +[["kitten"]] [["sitTing"]] +[["İo"]] [["i̇o"]] +[["İo"]] [["İo"]] +[["İo"]] [["İo"]] +[["İo"]] [["İo"]] + + +-- !query +select sentences(utf8_binary collate utf8_lcase), sentences(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct>,sentences(collate(utf8_lcase, utf8_binary), , ):array>> +-- !query output +[["Hello","world"],["Nice","day"]] [["Hello","world"],["Nice","day"]] +[["Something","else"],["Nothing","here"]] [["Something","else"],["Nothing","here"]] +[["Spark"]] [["SQL"]] +[["aaAaAAaA"]] [["aaAaAAaA"]] +[["aaAaAAaA"]] [["aaAaaAaA"]] +[["aaAaAAaA"]] [["aaAaaAaAaaAaaAaAaaAaaAaA"]] +[["abc"]] [["abc"]] +[["abcdcba"]] [["aBcDCbA"]] +[["bbAbAAbA"]] [["a"]] +[["efd2"]] [["efd2"]] +[["kitten"]] [["sitTing"]] +[["İo"]] [["i̇o"]] +[["İo"]] [["İo"]] +[["İo"]] [["İo"]] +[["İo"]] [["İo"]] + + +-- !query +select upper(utf8_binary), upper(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +AAAAAAAA AAAAAAAA +AAAAAAAA AAAAAAAA +AAAAAAAA AAAAAAAAAAAAAAAAAAAAAAAA +ABC ABC +ABCDCBA ABCDCBA +BBABAABA A +EFD2 EFD2 +HELLO, WORLD! NICE DAY. HELLO, WORLD! NICE DAY. +KITTEN SITTING +SOMETHING ELSE. NOTHING HERE. SOMETHING ELSE. NOTHING HERE. +SPARK SQL +İO İO +İO İO +İO İO +İO İO + + +-- !query +select upper(utf8_binary collate utf8_lcase), upper(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +AAAAAAAA AAAAAAAA +AAAAAAAA AAAAAAAA +AAAAAAAA AAAAAAAAAAAAAAAAAAAAAAAA +ABC ABC +ABCDCBA ABCDCBA +BBABAABA A +EFD2 EFD2 +HELLO, WORLD! NICE DAY. HELLO, WORLD! NICE DAY. +KITTEN SITTING +SOMETHING ELSE. NOTHING HERE. SOMETHING ELSE. NOTHING HERE. +SPARK SQL +İO İO +İO İO +İO İO +İO İO + + +-- !query +select lower(utf8_binary), lower(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +aaaaaaaa aaaaaaaa +aaaaaaaa aaaaaaaa +aaaaaaaa aaaaaaaaaaaaaaaaaaaaaaaa +abc abc +abcdcba abcdcba +bbabaaba a +efd2 efd2 +hello, world! nice day. hello, world! nice day. +i̇o i̇o +i̇o i̇o +i̇o i̇o +i̇o i̇o +kitten sitting +something else. nothing here. something else. nothing here. +spark sql + + +-- !query +select lower(utf8_binary collate utf8_lcase), lower(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +aaaaaaaa aaaaaaaa +aaaaaaaa aaaaaaaa +aaaaaaaa aaaaaaaaaaaaaaaaaaaaaaaa +abc abc +abcdcba abcdcba +bbabaaba a +efd2 efd2 +hello, world! nice day. hello, world! nice day. +i̇o i̇o +i̇o i̇o +i̇o i̇o +i̇o i̇o +kitten sitting +something else. nothing here. something else. nothing here. +spark sql + + +-- !query +select initcap(utf8_binary), initcap(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Aaaaaaaa Aaaaaaaa +Aaaaaaaa Aaaaaaaa +Aaaaaaaa Aaaaaaaaaaaaaaaaaaaaaaaa +Abc Abc +Abcdcba Abcdcba +Bbabaaba A +Efd2 Efd2 +Hello, World! Nice Day. Hello, World! Nice Day. +Kitten Sitting +Something Else. Nothing Here. Something Else. Nothing Here. +Spark Sql +İo İo +İo İo +İo İo +İo İo + + +-- !query +select initcap(utf8_binary collate utf8_lcase), initcap(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +Aaaaaaaa Aaaaaaaa +Aaaaaaaa Aaaaaaaa +Aaaaaaaa Aaaaaaaaaaaaaaaaaaaaaaaa +Abc Abc +Abcdcba Abcdcba +Bbabaaba A +Efd2 Efd2 +Hello, World! Nice Day. Hello, World! Nice Day. +Kitten Sitting +Something Else. Nothing Here. Something Else. Nothing Here. +Spark Sql +İo İo +İo İo +İo İo +İo İo + + +-- !query +select overlay(utf8_binary, utf8_lcase, 2) from t5 +-- !query schema +struct +-- !query output +HHello, world! Nice day. +SSQLk +SSomething else. Nothing here. +aaBcDCbA +aaaAaAAaA +aaaAaaAaA +aaaAaaAaAaaAaaAaAaaAaaAaA +aabc +baAbAAbA +eefd2 +ksitTing +İi̇o +İİo +İİo +İİo + + +-- !query +select overlay(s, utf8_binary,1) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. +Something else. Nothing here. +Spark +aaAaAAaA +aaAaAAaA +aaAaAAaA +abc +abcdcba +bbAbAAbA +efd2 +kitten +İo +İo +İo +İo + + +-- !query +select overlay(utf8_binary collate utf8_binary, s collate utf8_lcase, 3) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\"" + } +} + + +-- !query +select overlay(utf8_binary, utf8_lcase collate utf8_binary, 2) from t5 +-- !query schema +struct +-- !query output +HHello, world! Nice day. +SSQLk +SSomething else. Nothing here. +aaBcDCbA +aaaAaAAaA +aaaAaaAaA +aaaAaaAaAaaAaaAaAaaAaaAaA +aabc +baAbAAbA +eefd2 +ksitTing +İi̇o +İİo +İİo +İİo + + +-- !query +select overlay(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 2) from t5 +-- !query schema +struct +-- !query output +HHello, world! Nice day. +SSQLk +SSomething else. Nothing here. +aaBcDCbA +aaaAaAAaA +aaaAaaAaA +aaaAaaAaAaaAaaAaAaaAaaAaA +aabc +baAbAAbA +eefd2 +ksitTing +İi̇o +İİo +İİo +İİo + + +-- !query +select overlay(utf8_binary, 'a', 2), overlay(utf8_lcase, 'a', 2) from t5 +-- !query schema +struct +-- !query output +Hallo, world! Nice day. Hallo, world! Nice day. +Saark SaL +Samething else. Nothing here. Samething else. Nothing here. +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +aac aac +aacdcba aacDCbA +baAbAAbA aa +ead2 ead2 +katten satTing +İa iao +İa İa +İa İa +İa İa + + +-- !query +select overlay(utf8_binary, 'AaAA' collate utf8_lcase, 2), overlay(utf8_lcase, 'AAa' collate utf8_binary, 2) from t5 +-- !query schema +struct +-- !query output +HAaAA, world! Nice day. HAAao, world! Nice day. +SAaAA SAAa +SAaAAhing else. Nothing here. SAAathing else. Nothing here. +aAaAA aAAa +aAaAAAaA aAAaAAaA +aAaAAAaA aAAaaAaA +aAaAAAaA aAAaaAaAaaAaaAaAaaAaaAaA +aAaAAba aAAaCbA +bAaAAAbA aAAa +eAaAA eAAa +kAaAAn sAAaing +İAaAA iAAa +İAaAA İAAa +İAaAA İAAa +İAaAA İAAa + + +-- !query +select format_string(format, utf8_binary, utf8_lcase) from t8 +-- !query schema +struct +-- !query output +abCdEabCdE + + +-- !query +select format_string(format collate utf8_lcase, utf8_lcase, utf8_binary collate utf8_lcase, 3), format_string(format, utf8_lcase collate utf8_binary, utf8_binary) from t8 +-- !query schema +struct +-- !query output +abCdEabCdE abCdEabCdE + + +-- !query +select format_string(format, utf8_binary, utf8_lcase) from t8 +-- !query schema +struct +-- !query output +abCdEabCdE + + +-- !query +select soundex(utf8_binary), soundex(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +A000 A000 +A000 A000 +A000 A000 +A120 A120 +A123 A123 +B110 A000 +E130 E130 +H464 H464 +K350 S352 +S162 S400 +S535 S535 +İo I000 +İo İo +İo İo +İo İo + + +-- !query +select soundex(utf8_binary collate utf8_lcase), soundex(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +A000 A000 +A000 A000 +A000 A000 +A120 A120 +A123 A123 +B110 A000 +E130 E130 +H464 H464 +K350 S352 +S162 S400 +S535 S535 +İo I000 +İo İo +İo İo +İo İo + + +-- !query +select length(utf8_binary), length(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +2 2 +2 3 +2 3 +23 23 +29 29 +3 2 +3 3 +4 4 +5 3 +6 7 +7 7 +8 1 +8 24 +8 8 +8 8 + + +-- !query +select length(utf8_binary collate utf8_lcase), length(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +2 2 +2 3 +2 3 +23 23 +29 29 +3 2 +3 3 +4 4 +5 3 +6 7 +7 7 +8 1 +8 24 +8 8 +8 8 + + +-- !query +select bit_length(utf8_binary), bit_length(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +184 184 +232 232 +24 24 +24 24 +24 32 +24 32 +32 24 +32 32 +40 24 +48 56 +56 56 +64 192 +64 64 +64 64 +64 8 + + +-- !query +select bit_length(utf8_binary collate utf8_lcase), bit_length(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +184 184 +232 232 +24 24 +24 24 +24 32 +24 32 +32 24 +32 32 +40 24 +48 56 +56 56 +64 192 +64 64 +64 64 +64 8 + + +-- !query +select octet_length(utf8_binary), octet_length(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +23 23 +29 29 +3 3 +3 3 +3 4 +3 4 +4 3 +4 4 +5 3 +6 7 +7 7 +8 1 +8 24 +8 8 +8 8 + + +-- !query +select octet_length(utf8_binary collate utf8_lcase), octet_length(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +23 23 +29 29 +3 3 +3 3 +3 4 +3 4 +4 3 +4 4 +5 3 +6 7 +7 7 +8 1 +8 24 +8 8 +8 8 + + +-- !query +select octet_length(utf8_binary collate utf8_lcase_rtrim), octet_length(utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query schema +struct +-- !query output +23 23 +29 29 +3 3 +3 3 +3 4 +3 4 +4 3 +4 4 +5 3 +6 7 +7 7 +8 1 +8 24 +8 8 +8 8 + + +-- !query +select luhn_check(num) from t9 +-- !query schema +struct +-- !query output +false +false + + +-- !query +select levenshtein(utf8_binary, utf8_lcase) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +1 +1 +1 +16 +2 +4 +4 +4 +8 + + +-- !query +select levenshtein(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +1 +1 + + +-- !query +select levenshtein(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\"" + } +} + + +-- !query +select levenshtein(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +1 +1 +1 +16 +2 +4 +4 +4 +8 + + +-- !query +select levenshtein(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +1 +1 +1 +16 +2 +4 +4 +4 +8 + + +-- !query +select levenshtein(utf8_binary, 'a'), levenshtein(utf8_lcase, 'a') from t5 +-- !query schema +struct +-- !query output +2 2 +2 2 +2 3 +2 3 +22 22 +29 29 +3 2 +4 3 +4 4 +6 6 +6 7 +7 23 +7 7 +7 7 +8 0 + + +-- !query +select levenshtein(utf8_binary, 'AaAA' collate utf8_lcase, 3), levenshtein(utf8_lcase, 'AAa' collate utf8_binary, 4) from t5 +-- !query schema +struct +-- !query output +-1 -1 +-1 -1 +-1 -1 +-1 -1 +-1 -1 +-1 -1 +-1 -1 +-1 2 +-1 3 +-1 3 +-1 3 +-1 3 +-1 3 +-1 4 +3 3 + + +-- !query +select is_valid_utf8(utf8_binary), is_valid_utf8(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true + + +-- !query +select is_valid_utf8(utf8_binary collate utf8_lcase), is_valid_utf8(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true + + +-- !query +select is_valid_utf8(utf8_binary collate utf8_lcase_rtrim), is_valid_utf8(utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query schema +struct +-- !query output +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true +true true + + +-- !query +select make_valid_utf8(utf8_binary), make_valid_utf8(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select make_valid_utf8(utf8_binary collate utf8_lcase), make_valid_utf8(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select make_valid_utf8(utf8_binary collate utf8_lcase_rtrim), make_valid_utf8(utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select validate_utf8(utf8_binary), validate_utf8(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select validate_utf8(utf8_binary collate utf8_lcase), validate_utf8(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select validate_utf8(utf8_binary collate utf8_lcase_rtrim), validate_utf8(utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select try_validate_utf8(utf8_binary), try_validate_utf8(utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select try_validate_utf8(utf8_binary collate utf8_lcase), try_validate_utf8(utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select try_validate_utf8(utf8_binary collate utf8_lcase_rtrim), try_validate_utf8(utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +abcdcba aBcDCbA +bbAbAAbA a +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select substr(utf8_binary, 2, 2), substr(utf8_lcase, 2, 2) from t5 +-- !query schema +struct +-- !query output +aA aA +aA aA +aA aA +bA +bc Bc +bc bc +el el +fd fd +it it +o o +o o +o ̇o +o o +om om +pa QL + + +-- !query +select substr(utf8_binary collate utf8_lcase, 2, 2), substr(utf8_lcase collate utf8_binary, 2, 2) from t5 +-- !query schema +struct +-- !query output +aA aA +aA aA +aA aA +bA +bc Bc +bc bc +el el +fd fd +it it +o o +o o +o ̇o +o o +om om +pa QL + + +-- !query +select right(utf8_binary, 2), right(utf8_lcase, 2) from t5 +-- !query schema +struct +-- !query output +aA aA +aA aA +aA aA +bA a +ba bA +bc bc +d2 d2 +e. e. +en ng +o İo +rk QL +y. y. +İo o +İo İo +İo ̇o + + +-- !query +select right(utf8_binary collate utf8_lcase, 2), right(utf8_lcase collate utf8_binary, 2) from t5 +-- !query schema +struct +-- !query output +aA aA +aA aA +aA aA +bA a +ba bA +bc bc +d2 d2 +e. e. +en ng +o İo +rk QL +y. y. +İo o +İo İo +İo ̇o + + +-- !query +select left(utf8_binary, '2' collate utf8_lcase), left(utf8_lcase, 2) from t5 +-- !query schema +struct +-- !query output +He He +So So +Sp SQ +aa aa +aa aa +aa aa +ab aB +ab ab +bb a +ef ef +ki si +İo i̇ +İo İo +İo İo +İo İo + + +-- !query +select left(utf8_binary collate utf8_lcase, 2), left(utf8_lcase collate utf8_binary, 2) from t5 +-- !query schema +struct +-- !query output +He He +So So +Sp SQ +aa aa +aa aa +aa aa +ab aB +ab ab +bb a +ef ef +ki si +İo i̇ +İo İo +İo İo +İo İo + + +-- !query +select rpad(utf8_binary, 8, utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, w +Somethin +SparkSQL +aaAaAAaA +aaAaAAaA +aaAaAAaA +abcabcab +abcdcbaa +bbAbAAbA +efd2efd2 +kittensi +İo İoİoİ +İoi̇oi̇o +İoİo İo +İoİoİoİo + + +-- !query +select rpad(s, 8, utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, w +Somethin +SparkSpa +aaAaAAaA +aaAaAAaA +aaAaAAaA +abcabcab +abcdcbaa +bbAbaAbA +efd2efd2 +kittenki +İoİo İo +İoİoİoİo +İoİoİoİo +İoİoİoİo + + +-- !query +select rpad(utf8_binary collate utf8_binary, 8, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\"" + } +} + + +-- !query +select rpad(utf8_binary, 8, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, w +Somethin +SparkSQL +aaAaAAaA +aaAaAAaA +aaAaAAaA +abcabcab +abcdcbaa +bbAbAAbA +efd2efd2 +kittensi +İo İoİoİ +İoi̇oi̇o +İoİo İo +İoİoİoİo + + +-- !query +select rpad(utf8_binary collate utf8_lcase, 8, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, w +Somethin +SparkSQL +aaAaAAaA +aaAaAAaA +aaAaAAaA +abcabcab +abcdcbaa +bbAbAAbA +efd2efd2 +kittensi +İo İoİoİ +İoi̇oi̇o +İoİo İo +İoİoİoİo + + +-- !query +select lpad(utf8_binary collate utf8_binary_rtrim, 8, utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query schema +struct +-- !query output +Hello, w +SQLSpark +Somethin +aaAaAAaA +aaAaAAaA +aaAaAAaA +aabcdcba +abcababc +bbAbAAbA +efd2efd2 +i̇oi̇oİo +sikitten +İo İo İo +İoİoİoİo +İoİoİİo + + +-- !query +select rpad(utf8_binary, 8, 'a'), rpad(utf8_lcase, 8, 'a') from t5 +-- !query schema +struct +-- !query output +Hello, w Hello, w +Somethin Somethin +Sparkaaa SQLaaaaa +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaA +abcaaaaa abcaaaaa +abcdcbaa aBcDCbAa +bbAbAAbA aaaaaaaa +efd2aaaa efd2aaaa +kittenaa sitTinga +İo aaaaa İoaaaaaa +İoaaaaaa i̇oaaaaa +İoaaaaaa İo aaaaa +İoaaaaaa İoaaaaaa + + +-- !query +select rpad(utf8_binary, 8, 'AaAA' collate utf8_lcase), rpad(utf8_lcase, 8, 'AAa' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, w Hello, w +Somethin Somethin +SparkAaA SQLAAaAA +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaA +abcAaAAA abcAAaAA +abcdcbaA aBcDCbAA +bbAbAAbA aAAaAAaA +efd2AaAA efd2AAaA +kittenAa sitTingA +İo AaAAA İoAAaAAa +İoAaAAAa i̇oAAaAA +İoAaAAAa İo AAaAA +İoAaAAAa İoAAaAAa + + +-- !query +select lpad(utf8_binary, 8, utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, w +SQLSpark +Somethin +aaAaAAaA +aaAaAAaA +aaAaAAaA +aabcdcba +abcababc +bbAbAAbA +efd2efd2 +i̇oi̇oİo +sikitten +İo İo İo +İoİoİoİo +İoİoİİo + + +-- !query +select lpad(s, 8, utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, w +Somethin +SpaSpark +aaAaAAaA +aaAaAAaA +aaAaAAaA +aabcdcba +abcababc +bbAbaAbA +efd2efd2 +kikitten +İo İo İo +İoİoİoİo +İoİoİoİo +İoİoİoİo + + +-- !query +select lpad(utf8_binary collate utf8_binary, 8, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\"" + } +} + + +-- !query +select lpad(utf8_binary, 8, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +Hello, w +SQLSpark +Somethin +aaAaAAaA +aaAaAAaA +aaAaAAaA +aabcdcba +abcababc +bbAbAAbA +efd2efd2 +i̇oi̇oİo +sikitten +İo İo İo +İoİoİoİo +İoİoİİo + + +-- !query +select lpad(utf8_binary collate utf8_lcase, 8, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, w +SQLSpark +Somethin +aaAaAAaA +aaAaAAaA +aaAaAAaA +aabcdcba +abcababc +bbAbAAbA +efd2efd2 +i̇oi̇oİo +sikitten +İo İo İo +İoİoİoİo +İoİoİİo + + +-- !query +select lpad(utf8_binary collate utf8_binary_rtrim, 8, utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query schema +struct +-- !query output +Hello, w +SQLSpark +Somethin +aaAaAAaA +aaAaAAaA +aaAaAAaA +aabcdcba +abcababc +bbAbAAbA +efd2efd2 +i̇oi̇oİo +sikitten +İo İo İo +İoİoİoİo +İoİoİİo + + +-- !query +select lpad(utf8_binary, 8, 'a'), lpad(utf8_lcase, 8, 'a') from t5 +-- !query schema +struct +-- !query output +Hello, w Hello, w +Somethin Somethin +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaA +aaaSpark aaaaaSQL +aaaaaabc aaaaaabc +aaaaaaİo aaaaaaİo +aaaaaaİo aaaaai̇o +aaaaaaİo aaaaaİo +aaaaaİo aaaaaaİo +aaaaefd2 aaaaefd2 +aabcdcba aaBcDCbA +aakitten asitTing +bbAbAAbA aaaaaaaa + + +-- !query +select lpad(utf8_binary, 8, 'AaAA' collate utf8_lcase), lpad(utf8_lcase, 8, 'AAa' collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +AaAAAabc AAaAAabc +AaAAAaİo AAaAAaİo +AaAAAaİo AAaAAi̇o +AaAAAaİo AAaAAİo +AaAAAİo AAaAAaİo +AaAAefd2 AAaAefd2 +AaASpark AAaAASQL +Aabcdcba AaBcDCbA +Aakitten AsitTing +Hello, w Hello, w +Somethin Somethin +aaAaAAaA aaAaAAaA +aaAaAAaA aaAaaAaA +aaAaAAaA aaAaaAaA +bbAbAAbA AAaAAaAa + + +-- !query +select locate(utf8_binary, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION", + "sqlState" : "42P22", + "messageParameters" : { + "expr" : "\"locate(utf8_binary, utf8_lcase, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "locate(utf8_binary, utf8_lcase)" + } ] +} + + +-- !query +select locate(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + + +-- !query +select locate(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\"" + } +} + + +-- !query +select locate(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +0 +1 +1 +1 +1 +1 +1 +1 + + +-- !query +select locate(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase, 3) from t5 +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +3 + + +-- !query +select locate(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 3) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"collate(utf8_binary, unicode_ai)\"", + "inputType" : "\"STRING COLLATE UNICODE_AI\"", + "paramIndex" : "first", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"locate(collate(utf8_binary, unicode_ai), collate(utf8_lcase, unicode_ai), 3)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "locate(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai, 3)" + } ] +} + + +-- !query +select locate(utf8_binary, 'a'), locate(utf8_lcase, 'a') from t5 +-- !query schema +struct +-- !query output +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 1 + + +-- !query +select locate(utf8_binary, 'AaAA' collate utf8_lcase, 4), locate(utf8_lcase, 'AAa' collate utf8_binary, 4) from t5 +-- !query schema +struct +-- !query output +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 + + +-- !query +select locate(utf8_binary, 'AaAA ' collate utf8_binary_rtrim, 4), locate(utf8_lcase, 'AAa ' collate utf8_binary, 4) from t5 +-- !query schema +struct +-- !query output +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 +0 0 + + +-- !query +select TRIM(utf8_binary, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION", + "sqlState" : "42P22", + "messageParameters" : { + "expr" : "\"TRIM(BOTH utf8_binary FROM utf8_lcase)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "TRIM(utf8_binary, utf8_lcase)" + } ] +} + + +-- !query +select TRIM(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output + + + +-- !query +select TRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING COLLATE UTF8_LCASE\", \"STRING\"" + } +} + + +-- !query +select TRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + +BcDCbA +QL +a +i̇ +sitTing + + +-- !query +select TRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + + + + +QL +sitTing + + +-- !query +select TRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"collate(utf8_lcase, unicode_ai)\"", + "inputType" : "\"STRING COLLATE UNICODE_AI\"", + "paramIndex" : "first", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"TRIM(BOTH collate(utf8_binary, unicode_ai) FROM collate(utf8_lcase, unicode_ai))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "TRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)" + } ] +} + + +-- !query +select TRIM(utf8_binary collate utf8_binary_rtrim, utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + +BcDCbA +QL +a +i̇ +sitTing + + +-- !query +select TRIM('ABc', utf8_binary), TRIM('ABc', utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAa +aaAaAAa +aaAaAAa +ab +abcdcba D +bbAbAAb +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select TRIM('ABc' collate utf8_lcase, utf8_binary), TRIM('AAa' collate utf8_binary, utf8_lcase) from t5 +-- !query schema +struct +-- !query output + + + + + bc +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +d BcDCb +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select BTRIM(utf8_binary, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION", + "sqlState" : "42P22", + "messageParameters" : { + "expr" : "\"TRIM(BOTH utf8_lcase FROM utf8_binary)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "BTRIM(utf8_binary, utf8_lcase)" + } ] +} + + +-- !query +select BTRIM(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + + + + + +a + + +-- !query +select BTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING\", \"STRING COLLATE UTF8_LCASE\"" + } +} + + +-- !query +select BTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + +bbAbAAbA +d +kitte +park +İ + + +-- !query +select BTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + + +bbAbAAb +kitte +park +İ + + +-- !query +select BTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"collate(utf8_binary, unicode_ai)\"", + "inputType" : "\"STRING COLLATE UNICODE_AI\"", + "paramIndex" : "first", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"TRIM(BOTH collate(utf8_lcase, unicode_ai) FROM collate(utf8_binary, unicode_ai))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "BTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)" + } ] +} + + +-- !query +select BTRIM(utf8_binary collate utf8_binary_rtrim, utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + +bbAbAAbA +d +kitte +park +İ + + +-- !query +select BTRIM('ABc', utf8_binary), BTRIM('ABc', utf8_lcase) from t5 +-- !query schema +struct +-- !query output +AB +AB +AB B +ABc ABc +ABc ABc +ABc ABc +ABc ABc +ABc ABc +ABc ABc +ABc ABc +ABc ABc +Bc Bc +Bc Bc +Bc Bc +Bc Bc + + +-- !query +select BTRIM('ABc' collate utf8_lcase, utf8_binary), BTRIM('AAa' collate utf8_binary, utf8_lcase) from t5 +-- !query schema +struct +-- !query output + + AA +ABc AAa +ABc AAa +ABc AAa +ABc AAa +ABc AAa +ABc AAa +ABc AAa +B AA +Bc +Bc +Bc +Bc AAa +c AA + + +-- !query +select LTRIM(utf8_binary, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION", + "sqlState" : "42P22", + "messageParameters" : { + "expr" : "\"TRIM(LEADING utf8_binary FROM utf8_lcase)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "LTRIM(utf8_binary, utf8_lcase)" + } ] +} + + +-- !query +select LTRIM(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output + + + +-- !query +select LTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING COLLATE UTF8_LCASE\", \"STRING\"" + } +} + + +-- !query +select LTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + +BcDCbA +QL +a +i̇o +sitTing + + +-- !query +select LTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + + + + +QL +sitTing + + +-- !query +select LTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"collate(utf8_lcase, unicode_ai)\"", + "inputType" : "\"STRING COLLATE UNICODE_AI\"", + "paramIndex" : "first", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"TRIM(LEADING collate(utf8_binary, unicode_ai) FROM collate(utf8_lcase, unicode_ai))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "LTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)" + } ] +} + + +-- !query +select LTRIM(utf8_binary collate utf8_binary_rtrim, utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + +BcDCbA +QL +a +i̇o +sitTing + + +-- !query +select LTRIM('ABc', utf8_binary), LTRIM('ABc', utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAaA +aaAaAAaA +aaAaAAaA +abc +abcdcba DCbA +bbAbAAbA +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select LTRIM('ABc' collate utf8_lcase, utf8_binary), LTRIM('AAa' collate utf8_binary, utf8_lcase) from t5 +-- !query schema +struct +-- !query output + + + + + bc +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +dcba BcDCbA +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select RTRIM(utf8_binary, utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INDETERMINATE_COLLATION_IN_EXPRESSION", + "sqlState" : "42P22", + "messageParameters" : { + "expr" : "\"TRIM(TRAILING utf8_binary FROM utf8_lcase)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "RTRIM(utf8_binary, utf8_lcase)" + } ] +} + + +-- !query +select RTRIM(s, utf8_binary) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + + + + + +İo + + +-- !query +select RTRIM(utf8_binary collate utf8_binary, s collate utf8_lcase) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "COLLATION_MISMATCH.EXPLICIT", + "sqlState" : "42P21", + "messageParameters" : { + "explicitTypes" : "\"STRING COLLATE UTF8_LCASE\", \"STRING\"" + } +} + + +-- !query +select RTRIM(utf8_binary, utf8_lcase collate utf8_binary) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + +SQL +a +aBcDCbA +i̇ +sitTing +İo + + +-- !query +select RTRIM(utf8_binary collate utf8_lcase, utf8_lcase collate utf8_lcase) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + + + +SQL +sitTing +İo + + +-- !query +select RTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai) from t5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"collate(utf8_lcase, unicode_ai)\"", + "inputType" : "\"STRING COLLATE UNICODE_AI\"", + "paramIndex" : "first", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"TRIM(TRAILING collate(utf8_binary, unicode_ai) FROM collate(utf8_lcase, unicode_ai))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "RTRIM(utf8_binary collate unicode_ai, utf8_lcase collate unicode_ai)" + } ] +} + + +-- !query +select RTRIM(utf8_binary collate utf8_binary_rtrim, utf8_lcase collate utf8_binary_rtrim) from t5 +-- !query schema +struct +-- !query output + + + + + + + + + + +SQL +a +aBcDCbA +i̇ +sitTing + + +-- !query +select RTRIM('ABc', utf8_binary), RTRIM('ABc', utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +aaAaAAa +aaAaAAa +aaAaAAa +ab +abcdcba aBcD +bbAbAAb +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select RTRIM('ABc' collate utf8_lcase, utf8_binary), RTRIM('AAa' collate utf8_binary, utf8_lcase) from t5 +-- !query schema +struct +-- !query output + + + + + abc +Hello, world! Nice day. Hello, world! Nice day. +Something else. Nothing here. Something else. Nothing here. +Spark SQL +abcd aBcDCb +efd2 efd2 +kitten sitTing +İo i̇o +İo İo +İo İo +İo İo + + +-- !query +select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +SQL SQL +Something else. Nothing here. Something else. Nothing here. +a a +aBcDCbA aBcDCbA +aaAaAAaA aaAaAAaA +aaAaaAaA aaAaaAaA +aaAaaAaAaaAaaAaAaaAaaAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +efd2 efd2 +i̇o i̇o +sitTing sitTing +İo İo +İo İo +İo İo + + +-- !query +select `concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)` from ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 +) +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +SQL SQL +Something else. Nothing here. Something else. Nothing here. +a a +aBcDCbA aBcDCbA +aaAaAAaA aaAaAAaA +aaAaaAaA aaAaaAaA +aaAaaAaAaaAaaAaAaaAaaAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +efd2 efd2 +i̇o i̇o +sitTing sitTing +İo İo +İo İo +İo İo + + +-- !query +select * from ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 +) +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +SQL SQL +Something else. Nothing here. Something else. Nothing here. +a a +aBcDCbA aBcDCbA +aaAaAAaA aaAaAAaA +aaAaaAaA aaAaaAaA +aaAaaAaAaaAaaAaAaaAaaAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +efd2 efd2 +i̇o i̇o +sitTing sitTing +İo İo +İo İo +İo İo + + +-- !query +select subq1.* from ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 +) AS subq1 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +SQL SQL +Something else. Nothing here. Something else. Nothing here. +a a +aBcDCbA aBcDCbA +aaAaAAaA aaAaAAaA +aaAaaAaA aaAaaAaA +aaAaaAaAaaAaaAaAaaAaaAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +efd2 efd2 +i̇o i̇o +sitTing sitTing +İo İo +İo İo +İo İo + + +-- !query +with cte as ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 +) +select * from cte +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +SQL SQL +Something else. Nothing here. Something else. Nothing here. +a a +aBcDCbA aBcDCbA +aaAaAAaA aaAaAAaA +aaAaaAaA aaAaaAaA +aaAaaAaAaaAaaAaAaaAaaAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +efd2 efd2 +i̇o i̇o +sitTing sitTing +İo İo +İo İo +İo İo + + +-- !query +select * from values (1) where exists ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 +) +-- !query schema +struct +-- !query output +1 + + +-- !query +select ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 limit 1 +) +-- !query schema +struct +-- !query output +Something else. Nothing here. Something else. Nothing here. + + +-- !query +select ( + with cte as ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 + ) + select * from cte limit 1 +) +-- !query schema +struct +-- !query output +Something else. Nothing here. Something else. Nothing here. + + +-- !query +select * from ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 limit 1 +) +where ( + `concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)` == 'aaa' +) +-- !query schema +struct +-- !query output + + + +-- !query +select lower(`concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)`) from ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 + group by 1 + order by 1 +) +-- !query schema +struct +-- !query output +a a +aaaaaaaa aaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaa aaaaaaaaaaaaaaaaaaaaaaaa +abc abc +abcdcba abcdcba +efd2 efd2 +hello, world! nice day. hello, world! nice day. +i̇o i̇o +i̇o i̇o +sitting sitting +something else. nothing here. something else. nothing here. +sql sql + + +-- !query +select lower(`concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)`) from ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 + group by 1 + order by max(concat_ws(' ', utf8_lcase, utf8_lcase)) +) +-- !query schema +struct +-- !query output +a a +aaaaaaaa aaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaa aaaaaaaaaaaaaaaaaaaaaaaa +abc abc +abcdcba abcdcba +efd2 efd2 +hello, world! nice day. hello, world! nice day. +i̇o i̇o +i̇o i̇o +sitting sitting +something else. nothing here. something else. nothing here. +sql sql + + +-- !query +create temporary view v1 as ( + select concat_ws(' ', utf8_lcase, utf8_lcase) from t5 +) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from v1 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +SQL SQL +Something else. Nothing here. Something else. Nothing here. +a a +aBcDCbA aBcDCbA +aaAaAAaA aaAaAAaA +aaAaaAaA aaAaaAaA +aaAaaAaAaaAaaAaAaaAaaAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +efd2 efd2 +i̇o i̇o +sitTing sitTing +İo İo +İo İo +İo İo + + +-- !query +select `concat_ws(' ' collate UTF8_LCASE, utf8_lcase, utf8_lcase)` from v1 +-- !query schema +struct +-- !query output +Hello, world! Nice day. Hello, world! Nice day. +SQL SQL +Something else. Nothing here. Something else. Nothing here. +a a +aBcDCbA aBcDCbA +aaAaAAaA aaAaAAaA +aaAaaAaA aaAaaAaA +aaAaaAaAaaAaaAaAaaAaaAaA aaAaaAaAaaAaaAaAaaAaaAaA +abc abc +efd2 efd2 +i̇o i̇o +sitTing sitTing +İo İo +İo İo +İo İo + + +-- !query +drop view v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t5 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t6 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t7 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t8 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t9 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t10 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/column-resolution-aggregate.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/column-resolution-aggregate.sql.out new file mode 100644 index 000000000000..bc4942be7860 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/column-resolution-aggregate.sql.out @@ -0,0 +1,129 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW v1 AS VALUES (1, 1, 1), (2, 2, 1) AS t(a, b, k) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v2 AS VALUES (1, 1, 1), (2, 2, 1) AS t(x, y, all) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT max(a) AS b, b FROM v1 GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"b\"", + "expressionAnyValue" : "\"any_value(b)\"" + } +} + + +-- !query +SELECT a FROM v1 WHERE (12, 13) IN (SELECT max(x + 10) AS a, a + 1 FROM v2) +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT a AS k FROM v1 GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT x FROM v2 GROUP BY all +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"x\"", + "expressionAnyValue" : "\"any_value(x)\"" + } +} + + +-- !query +SELECT a AS all, b FROM v1 GROUP BY all +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"b\"", + "expressionAnyValue" : "\"any_value(b)\"" + } +} + + +-- !query +SELECT k AS lca, lca + 1 AS col FROM v1 GROUP BY k, col +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_GROUP_BY", + "sqlState" : "0A000" +} + + +-- !query +SELECT k AS lca, lca + 1 AS col FROM v1 GROUP BY all +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_GROUP_BY", + "sqlState" : "0A000" +} + + +-- !query +SELECT k AS lca, lca + 1 AS col FROM v1 GROUP BY lca +-- !query schema +struct +-- !query output +1 2 + + +-- !query +SELECT * FROM v2 WHERE EXISTS (SELECT a, b FROM v1 GROUP BY all) +-- !query schema +struct +-- !query output +1 1 1 +2 2 1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/column-resolution-sort.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/column-resolution-sort.sql.out new file mode 100644 index 000000000000..67323d734c90 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/column-resolution-sort.sql.out @@ -0,0 +1,42 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW v1 AS VALUES (1, 2, 2), (2, 1, 1) AS t(a, b, k) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v2 AS VALUES (1, 2, 2), (2, 1, 1) AS t(a, b, all) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT max(a) AS b FROM v1 GROUP BY k ORDER BY b +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT max(a) FROM v2 GROUP BY all ORDER BY all +-- !query schema +struct +-- !query output +2 +1 + + +-- !query +SELECT (SELECT b FROM v1 ORDER BY all LIMIT 1) FROM v2 +-- !query schema +struct +-- !query output +1 +1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/columnresolution-negative.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/columnresolution-negative.sql.out new file mode 100644 index 000000000000..385ad35002be --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/columnresolution-negative.sql.out @@ -0,0 +1,535 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE DATABASE mydb1 +-- !query schema +struct<> +-- !query output + + + +-- !query +USE mydb1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t1 USING parquet AS SELECT 1 AS i1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE DATABASE mydb2 +-- !query schema +struct<> +-- !query output + + + +-- !query +USE mydb2 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t1 USING parquet AS SELECT 20 AS i1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET spark.sql.crossJoin.enabled = true +-- !query schema +struct +-- !query output +spark.sql.crossJoin.enabled true + + +-- !query +USE mydb1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT i1 FROM t1, mydb1.t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`i1`", + "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb1`.`t1`.`i1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 9, + "fragment" : "i1" + } ] +} + + +-- !query +SELECT t1.i1 FROM t1, mydb1.t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`t1`.`i1`", + "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb1`.`t1`.`i1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "t1.i1" + } ] +} + + +-- !query +SELECT mydb1.t1.i1 FROM t1, mydb1.t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`mydb1`.`t1`.`i1`", + "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb1`.`t1`.`i1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "mydb1.t1.i1" + } ] +} + + +-- !query +SELECT i1 FROM t1, mydb2.t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`i1`", + "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 9, + "fragment" : "i1" + } ] +} + + +-- !query +SELECT t1.i1 FROM t1, mydb2.t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`t1`.`i1`", + "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "t1.i1" + } ] +} + + +-- !query +USE mydb2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT i1 FROM t1, mydb1.t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`i1`", + "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 9, + "fragment" : "i1" + } ] +} + + +-- !query +SELECT t1.i1 FROM t1, mydb1.t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`t1`.`i1`", + "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "t1.i1" + } ] +} + + +-- !query +SELECT i1 FROM t1, mydb2.t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`i1`", + "referenceNames" : "[`spark_catalog`.`mydb2`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 9, + "fragment" : "i1" + } ] +} + + +-- !query +SELECT t1.i1 FROM t1, mydb2.t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`t1`.`i1`", + "referenceNames" : "[`spark_catalog`.`mydb2`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "t1.i1" + } ] +} + + +-- !query +SELECT db1.t1.i1 FROM t1, mydb2.t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`db1`.`t1`.`i1`", + "proposal" : "`spark_catalog`.`mydb2`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "db1.t1.i1" + } ] +} + + +-- !query +SET spark.sql.crossJoin.enabled = false +-- !query schema +struct +-- !query output +spark.sql.crossJoin.enabled false + + +-- !query +USE mydb1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT mydb1.t1 FROM t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`mydb1`.`t1`", + "proposal" : "`t1`.`i1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "mydb1.t1" + } ] +} + + +-- !query +SELECT t1.x.y.* FROM t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CANNOT_RESOLVE_STAR_EXPAND", + "sqlState" : "42704", + "messageParameters" : { + "columns" : "`i1`", + "targetString" : "`t1`.`x`.`y`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "t1.x.y.*" + } ] +} + + +-- !query +SELECT t1 FROM mydb1.t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t1`", + "proposal" : "`i1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 9, + "fragment" : "t1" + } ] +} + + +-- !query +USE mydb2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT mydb1.t1.i1 FROM t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`mydb1`.`t1`.`i1`", + "proposal" : "`spark_catalog`.`mydb2`.`t1`.`i1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "mydb1.t1.i1" + } ] +} + + +-- !query +USE mydb1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v1 AS SELECT * FROM t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t1 USING parquet AS SELECT 1 AS i2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "i1", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE VIEW spark_catalog.mydb1.v1 AS SELECT * FROM t1", + "viewName" : "`spark_catalog`.`mydb1`.`v1`" + } +} + + +-- !query +USE mydb2 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMP VIEW v2 AS SELECT * FROM t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t1 USING parquet AS SELECT 1 AS i2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "i1", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE TEMPORARY VIEW", + "viewName" : "`v2`" + } +} + + +-- !query +DROP DATABASE mydb1 CASCADE +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP DATABASE mydb2 CASCADE +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/columnresolution-views.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/columnresolution-views.sql.out new file mode 100644 index 000000000000..91e1eb8ee5d2 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/columnresolution-views.sql.out @@ -0,0 +1,135 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW view1 AS SELECT 2 AS i1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT view1.* FROM view1 +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT * FROM view1 +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT view1.i1 FROM view1 +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT i1 FROM view1 +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT a.i1 FROM view1 AS a +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT i1 FROM view1 AS a +-- !query schema +struct +-- !query output +2 + + +-- !query +DROP VIEW view1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE GLOBAL TEMPORARY VIEW view1 as SELECT 1 as i1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM global_temp.view1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT global_temp.view1.* FROM global_temp.view1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT i1 FROM global_temp.view1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT global_temp.view1.i1 FROM global_temp.view1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT view1.i1 FROM global_temp.view1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT a.i1 FROM global_temp.view1 AS a +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT i1 FROM global_temp.view1 AS a +-- !query schema +struct +-- !query output +1 + + +-- !query +DROP VIEW global_temp.view1 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/columnresolution.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/columnresolution.sql.out new file mode 100644 index 000000000000..80ddd32db21d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/columnresolution.sql.out @@ -0,0 +1,440 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE DATABASE mydb1 +-- !query schema +struct<> +-- !query output + + + +-- !query +USE mydb1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t1 USING parquet AS SELECT 1 AS i1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE DATABASE mydb2 +-- !query schema +struct<> +-- !query output + + + +-- !query +USE mydb2 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t1 USING parquet AS SELECT 20 AS i1 +-- !query schema +struct<> +-- !query output + + + +-- !query +USE mydb1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT i1 FROM t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT i1 FROM mydb1.t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT t1.i1 FROM t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT t1.i1 FROM mydb1.t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT mydb1.t1.i1 FROM t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT mydb1.t1.i1 FROM mydb1.t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +USE mydb2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT i1 FROM t1 +-- !query schema +struct +-- !query output +20 + + +-- !query +SELECT i1 FROM mydb1.t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT t1.i1 FROM t1 +-- !query schema +struct +-- !query output +20 + + +-- !query +SELECT t1.i1 FROM mydb1.t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT mydb1.t1.i1 FROM mydb1.t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +USE mydb1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.* FROM t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT mydb1.t1.* FROM mydb1.t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT t1.* FROM mydb1.t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +USE mydb2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.* FROM t1 +-- !query schema +struct +-- !query output +20 + + +-- !query +SELECT mydb1.t1.* FROM mydb1.t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT t1.* FROM mydb1.t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT a.* FROM mydb1.t1 AS a +-- !query schema +struct +-- !query output +1 + + +-- !query +USE mydb1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t3 USING parquet AS SELECT * FROM VALUES (4,1), (3,1) AS t3(c1, c2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t4 USING parquet AS SELECT * FROM VALUES (4,1), (2,1) AS t4(c2, c3) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM t3 WHERE c1 IN (SELECT c2 FROM t4 WHERE t4.c3 = t3.c2) +-- !query schema +struct +-- !query output +4 1 + + +-- !query +SELECT * FROM mydb1.t3 WHERE c1 IN + (SELECT mydb1.t4.c2 FROM mydb1.t4 WHERE mydb1.t4.c3 = mydb1.t3.c2) +-- !query schema +struct +-- !query output +4 1 + + +-- !query +SET spark.sql.crossJoin.enabled = true +-- !query schema +struct +-- !query output +spark.sql.crossJoin.enabled true + + +-- !query +SELECT mydb1.t1.i1 FROM t1, mydb2.t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT mydb1.t1.i1 FROM mydb1.t1, mydb2.t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +USE mydb2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT mydb1.t1.i1 FROM t1, mydb1.t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SET spark.sql.crossJoin.enabled = false +-- !query schema +struct +-- !query output +spark.sql.crossJoin.enabled false + + +-- !query +USE mydb1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t5(i1 INT, t5 STRUCT) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t5 VALUES(1, (2, 3)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t5.i1 FROM t5 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT t5.t5.i1 FROM t5 +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT t5.t5.i1 FROM mydb1.t5 +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT t5.i1 FROM mydb1.t5 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT t5.* FROM mydb1.t5 +-- !query schema +struct> +-- !query output +1 {"i1":2,"i2":3} + + +-- !query +SELECT t5.t5.* FROM mydb1.t5 +-- !query schema +struct +-- !query output +2 3 + + +-- !query +SELECT mydb1.t5.t5.i1 FROM mydb1.t5 +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT mydb1.t5.t5.i2 FROM mydb1.t5 +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT mydb1.t5.* FROM mydb1.t5 +-- !query schema +struct> +-- !query output +1 {"i1":2,"i2":3} + + +-- !query +SELECT mydb1.t5.* FROM t5 +-- !query schema +struct> +-- !query output +1 {"i1":2,"i2":3} + + +-- !query +USE default +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP DATABASE mydb1 CASCADE +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP DATABASE mydb2 CASCADE +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/comments.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/comments.sql.out new file mode 100644 index 000000000000..1e2fc99a798c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/comments.sql.out @@ -0,0 +1,159 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +/* This is the first example of bracketed comment. +SELECT 'ommented out content' AS first; +*/ +SELECT 'selected content' AS first +-- !query schema +struct +-- !query output +selected content + + +-- !query +/* This is the second example of bracketed comment. +SELECT '/', 'ommented out content' AS second; +*/ +SELECT '/', 'selected content' AS second +-- !query schema +struct +-- !query output +/ selected content + + +-- !query +/* This is the third example of bracketed comment. + *SELECT '*', 'ommented out content' AS third; + */ +SELECT '*', 'selected content' AS third +-- !query schema +struct<*:string,third:string> +-- !query output +* selected content + + +-- !query +/**/ +SELECT 'selected content' AS fourth +-- !query schema +struct +-- !query output +selected content + + +-- !query +/* This is the first example of nested bracketed comment. +/* I am a nested bracketed comment.*/ +*/ +SELECT 'selected content' AS fifth +-- !query schema +struct +-- !query output +selected content + + +-- !query +/* This is the second example of nested bracketed comment. +/* I am a nested bracketed comment. + */ + */ +SELECT 'selected content' AS sixth +-- !query schema +struct +-- !query output +selected content + + +-- !query +/* + * This is the third example of nested bracketed comment. + /* + * I am a nested bracketed comment. + */ + */ +SELECT 'selected content' AS seventh +-- !query schema +struct +-- !query output +selected content + + +-- !query +/* + * This is the fourth example of nested bracketed comment. +SELECT /* I am a nested bracketed comment.*/ * FROM testData; + */ +SELECT 'selected content' AS eighth +-- !query schema +struct +-- !query output +selected content + + +-- !query +SELECT /* + * This is the fifth example of nested bracketed comment. +/* I am a second level of nested bracketed comment. +/* I am a third level of nested bracketed comment. +Other information of third level. +SELECT 'ommented out content' AS ninth; +*/ +Other information of second level. +*/ +Other information of first level. +*/ +'selected content' AS ninth +-- !query schema +struct +-- !query output +selected content + + +-- !query +/*/**/*/ +SELECT 'selected content' AS tenth +-- !query schema +struct +-- !query output +selected content + + +-- !query +/*abc*/ +select 1 as a +/* + +2 as b +/*abc*/ +, 3 as c + +/**/ +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNCLOSED_BRACKETED_COMMENT", + "sqlState" : "42601" +} + + +-- !query +/*abc*/ +select 1 as a +/* + +2 as b +/*abc*/ +, 3 as c + +/**/ +select 4 as d +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNCLOSED_BRACKETED_COMMENT", + "sqlState" : "42601" +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/comparator.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/comparator.sql.out new file mode 100644 index 000000000000..f979a0ef609b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/comparator.sql.out @@ -0,0 +1,79 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select x'00' < x'0f' +-- !query schema +struct<(X'00' < X'0F'):boolean> +-- !query output +true + + +-- !query +select x'00' < x'ff' +-- !query schema +struct<(X'00' < X'FF'):boolean> +-- !query output +true + + +-- !query +select '1 ' = 1Y +-- !query schema +struct<(1 = 1):boolean> +-- !query output +true + + +-- !query +select '\t1 ' = 1Y +-- !query schema +struct<( 1 = 1):boolean> +-- !query output +true + + +-- !query +select '1 ' = 1S +-- !query schema +struct<(1 = 1):boolean> +-- !query output +true + + +-- !query +select '1 ' = 1 +-- !query schema +struct<(1 = 1):boolean> +-- !query output +true + + +-- !query +select ' 1' = 1L +-- !query schema +struct<( 1 = 1):boolean> +-- !query output +true + + +-- !query +select ' 1' = cast(1.0 as float) +-- !query schema +struct<( 1 = CAST(1.0 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +select ' 1.0 ' = 1.0D +-- !query schema +struct<( 1.0 = 1.0):boolean> +-- !query output +true + + +-- !query +select ' 1.0 ' = 1.0BD +-- !query schema +struct<( 1.0 = 1.0):boolean> +-- !query output +true diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/conditional-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/conditional-functions.sql.out new file mode 100644 index 000000000000..26293cad10ce --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/conditional-functions.sql.out @@ -0,0 +1,204 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE conditional_t USING PARQUET AS SELECT c1, c2 FROM VALUES(1d, 0),(2d, 1),(null, 1),(CAST('NaN' AS DOUBLE), 0) AS t(c1, c2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT nanvl(c2, c1/c2 + c1/c2) FROM conditional_t +-- !query schema +struct +-- !query output +0.0 +0.0 +1.0 +1.0 + + +-- !query +SELECT nanvl(c2, 1/0) FROM conditional_t +-- !query schema +struct +-- !query output +0.0 +0.0 +1.0 +1.0 + + +-- !query +SELECT nanvl(1-0, 1/0) FROM conditional_t +-- !query schema +struct +-- !query output +1.0 +1.0 +1.0 +1.0 + + +-- !query +SELECT if(c2 >= 0, 1-0, 1/0) from conditional_t +-- !query schema +struct<(IF((c2 >= 0), (1 - 0), (1 / 0))):double> +-- !query output +1.0 +1.0 +1.0 +1.0 + + +-- !query +SELECT if(1 == 1, 1, 1/0) +-- !query schema +struct<(IF((1 = 1), 1, (1 / 0))):double> +-- !query output +1.0 + + +-- !query +SELECT if(1 != 1, 1/0, 1) +-- !query schema +struct<(IF((NOT (1 = 1)), (1 / 0), 1)):double> +-- !query output +1.0 + + +-- !query +SELECT coalesce(c2, 1/0) from conditional_t +-- !query schema +struct +-- !query output +0.0 +0.0 +1.0 +1.0 + + +-- !query +SELECT coalesce(1, 1/0) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT coalesce(null, 1, 1/0) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT case when c2 >= 0 then 1 else 1/0 end from conditional_t +-- !query schema +struct= 0) THEN 1 ELSE (1 / 0) END:double> +-- !query output +1.0 +1.0 +1.0 +1.0 + + +-- !query +SELECT case when 1 < 2 then 1 else 1/0 end +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT case when 1 > 2 then 1/0 else 1 end +-- !query schema +struct 2) THEN (1 / 0) ELSE 1 END:double> +-- !query output +1.0 + + +-- !query +SELECT nullifzero(0), + nullifzero(cast(0 as tinyint)), + nullifzero(cast(0 as bigint)), + nullifzero('0'), + nullifzero(0.0), + nullifzero(1), + nullifzero(null) +-- !query schema +struct +-- !query output +NULL NULL NULL NULL NULL 1 NULL + + +-- !query +SELECT nullifzero('abc') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'abc'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "nullifzero('abc')" + } ] +} + + +-- !query +SELECT zeroifnull(null), + zeroifnull(1), + zeroifnull(cast(1 as tinyint)), + zeroifnull(cast(1 as bigint)) +-- !query schema +struct +-- !query output +0 1 1 1 + + +-- !query +SELECT zeroifnull('abc') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'abc'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "zeroifnull('abc')" + } ] +} + + +-- !query +DROP TABLE conditional_t +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/count.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/count.sql.out new file mode 100644 index 000000000000..714c95cdd3c7 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/count.sql.out @@ -0,0 +1,214 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + count(*), count(1), count(null), count(a), count(b), count(a + b), count((a, b)) +FROM testData +-- !query schema +struct +-- !query output +7 7 0 5 5 4 7 + + +-- !query +SELECT + count(DISTINCT 1), + count(DISTINCT null), + count(DISTINCT a), + count(DISTINCT b), + count(DISTINCT (a + b)), + count(DISTINCT (a, b)) +FROM testData +-- !query schema +struct +-- !query output +1 0 2 2 2 6 + + +-- !query +SELECT count(a, b), count(b, a), count(testData.*, testData.*) FROM testData +-- !query schema +struct +-- !query output +4 4 4 + + +-- !query +SELECT + count(DISTINCT a, b), count(DISTINCT b, a), count(DISTINCT *), count(DISTINCT testData.*, testData.*) +FROM testData +-- !query schema +struct +-- !query output +3 3 3 3 + + +-- !query +SELECT count(DISTINCT 3,2) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT count(DISTINCT 2), count(DISTINCT 2,3) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT count(DISTINCT 2), count(DISTINCT 3,2) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT count(DISTINCT a), count(DISTINCT 2,3) FROM testData +-- !query schema +struct +-- !query output +2 1 + + +-- !query +SELECT count(DISTINCT a), count(DISTINCT 3,2) FROM testData +-- !query schema +struct +-- !query output +2 1 + + +-- !query +SELECT count(DISTINCT a), count(DISTINCT 2), count(DISTINCT 2,3) FROM testData +-- !query schema +struct +-- !query output +2 1 1 + + +-- !query +SELECT count(DISTINCT a), count(DISTINCT 2), count(DISTINCT 3,2) FROM testData +-- !query schema +struct +-- !query output +2 1 1 + + +-- !query +SELECT count(distinct 0.8), percentile_approx(distinct a, 0.8) FROM testData +-- !query schema +struct +-- !query output +1 2 + + +-- !query +set spark.sql.legacy.allowParameterlessCount=true +-- !query schema +struct +-- !query output +spark.sql.legacy.allowParameterlessCount true + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding + + +-- !query +SELECT count() FROM testData +-- !query schema +struct +-- !query output +0 + + +-- !query +set spark.sql.legacy.allowParameterlessCount=false +-- !query schema +struct +-- !query output +spark.sql.legacy.allowParameterlessCount false + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +SELECT count() FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITH_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "expectedNum" : " >= 1", + "functionName" : "`count`", + "legacyConfKey" : "\"spark.sql.legacy.allowParameterlessCount\"", + "legacyConfValue" : "\"true\"", + "legacyNum" : "0" + } +} + + +-- !query +set spark.sql.legacy.allowStarWithSingleTableIdentifierInCount=true +-- !query schema +struct +-- !query output +spark.sql.legacy.allowStarWithSingleTableIdentifierInCount true + + +-- !query +SELECT count(testData.*) FROM testData +-- !query schema +struct +-- !query output +4 + + +-- !query +set spark.sql.legacy.allowStarWithSingleTableIdentifierInCount=false +-- !query schema +struct +-- !query output +spark.sql.legacy.allowStarWithSingleTableIdentifierInCount false + + +-- !query +SELECT count(testData.*) FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1021", + "messageParameters" : { + "targetString" : "testData" + } +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cross-join.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cross-join.sql.out new file mode 100644 index 000000000000..5be5883df012 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cross-join.sql.out @@ -0,0 +1,137 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM nt1 cross join nt2 +-- !query schema +struct +-- !query output +one 1 one 1 +one 1 one 5 +one 1 two 22 +three 3 one 1 +three 3 one 5 +three 3 two 22 +two 2 one 1 +two 2 one 5 +two 2 two 22 + + +-- !query +SELECT * FROM nt1 cross join nt2 where nt1.k = nt2.k +-- !query schema +struct +-- !query output +one 1 one 1 +one 1 one 5 +two 2 two 22 + + +-- !query +SELECT * FROM nt1 cross join nt2 on (nt1.k = nt2.k) +-- !query schema +struct +-- !query output +one 1 one 1 +one 1 one 5 +two 2 two 22 + + +-- !query +SELECT * FROM nt1 cross join nt2 where nt1.v1 = 1 and nt2.v2 = 22 +-- !query schema +struct +-- !query output +one 1 two 22 + + +-- !query +SELECT a.key, b.key FROM +(SELECT k key FROM nt1 WHERE v1 < 2) a +CROSS JOIN +(SELECT k key FROM nt2 WHERE v2 = 22) b +-- !query schema +struct +-- !query output +one two + + +-- !query +create temporary view A(a, va) as select * from nt1 +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view B(b, vb) as select * from nt1 +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view C(c, vc) as select * from nt1 +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view D(d, vd) as select * from nt1 +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from ((A join B on (a = b)) cross join C) join D on (a = d) +-- !query schema +struct +-- !query output +one 1 one 1 one 1 one 1 +one 1 one 1 three 3 one 1 +one 1 one 1 two 2 one 1 +three 3 three 3 one 1 three 3 +three 3 three 3 three 3 three 3 +three 3 three 3 two 2 three 3 +two 2 two 2 one 1 two 2 +two 2 two 2 three 3 two 2 +two 2 two 2 two 2 two 2 + + +-- !query +SELECT * FROM nt1 CROSS JOIN nt2 ON (nt1.k > nt2.k) +-- !query schema +struct +-- !query output +three 3 one 1 +three 3 one 5 +two 2 one 1 +two 2 one 5 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/csv-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/csv-functions.sql.out new file mode 100644 index 000000000000..19b24cc12017 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/csv-functions.sql.out @@ -0,0 +1,286 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select from_csv('1, 3.14', 'a INT, f FLOAT') +-- !query schema +struct> +-- !query output +{"a":1,"f":3.14} + + +-- !query +select from_csv('26/08/2015', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')) +-- !query schema +struct> +-- !query output +{"time":2015-08-26 00:00:00} + + +-- !query +select from_csv('1', 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_SCHEMA.NON_STRING_LITERAL", + "sqlState" : "42K07", + "messageParameters" : { + "inputSchema" : "\"1\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "from_csv('1', 1)" + } ] +} + + +-- !query +select from_csv('1', 'a InvalidType') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'InvalidType'", + "hint" : ": extra input 'InvalidType'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "from_csv('1', 'a InvalidType')" + } ] +} + + +-- !query +select from_csv('1', 'Array') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_SCHEMA.NON_STRUCT_TYPE", + "sqlState" : "42K07", + "messageParameters" : { + "dataType" : "\"ARRAY\"", + "inputSchema" : "\"Array\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "from_csv('1', 'Array')" + } ] +} + + +-- !query +select from_csv('1', 'a INT', named_struct('mode', 'PERMISSIVE')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_MAP_FUNCTION", + "sqlState" : "42K06", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 65, + "fragment" : "from_csv('1', 'a INT', named_struct('mode', 'PERMISSIVE'))" + } ] +} + + +-- !query +select from_csv('1', 'a INT', map('mode', 1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_STRING_TYPE", + "sqlState" : "42K06", + "messageParameters" : { + "mapType" : "\"MAP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "from_csv('1', 'a INT', map('mode', 1))" + } ] +} + + +-- !query +select from_csv() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3]", + "functionName" : "`from_csv`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "from_csv()" + } ] +} + + +-- !query +select from_csv('1,abc', schema_of_csv('1,abc')) +-- !query schema +struct> +-- !query output +{"_c0":1,"_c1":"abc"} + + +-- !query +select schema_of_csv('1|abc', map('delimiter', '|')) +-- !query schema +struct +-- !query output +STRUCT<_c0: INT, _c1: STRING> + + +-- !query +select schema_of_csv(null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_NULL", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "csv", + "sqlExpr" : "\"schema_of_csv(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "schema_of_csv(null)" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW csvTable(csvField, a) AS SELECT * FROM VALUES ('1,abc', 'a') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT schema_of_csv(csvField) FROM csvTable +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"csvField\"", + "inputName" : "`csv`", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"schema_of_csv(csvField)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "schema_of_csv(csvField)" + } ] +} + + +-- !query +DROP VIEW IF EXISTS csvTable +-- !query schema +struct<> +-- !query output + + + +-- !query +select to_csv(named_struct('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +1,2 + + +-- !query +select to_csv(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy')) +-- !query schema +struct +-- !query output +26/08/2015 + + +-- !query +select to_csv(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_MAP_FUNCTION", + "sqlState" : "42K06", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "to_csv(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE'))" + } ] +} + + +-- !query +select to_csv(named_struct('a', 1, 'b', 2), map('mode', 1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_STRING_TYPE", + "sqlState" : "42K06", + "messageParameters" : { + "mapType" : "\"MAP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "to_csv(named_struct('a', 1, 'b', 2), map('mode', 1))" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cte-command.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cte-command.sql.out new file mode 100644 index 000000000000..67ac321a1954 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cte-command.sql.out @@ -0,0 +1,121 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE cte_tbl USING csv AS WITH s AS (SELECT 42 AS col) SELECT * FROM s +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM cte_tbl +-- !query schema +struct +-- !query output +42 + + +-- !query +CREATE TEMPORARY VIEW cte_view AS WITH s AS (SELECT 42 AS col) SELECT * FROM s +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM cte_view +-- !query schema +struct +-- !query output +42 + + +-- !query +WITH s AS (SELECT 43 AS col) +INSERT INTO cte_tbl SELECT * FROM S +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM cte_tbl +-- !query schema +struct +-- !query output +42 +43 + + +-- !query +INSERT INTO cte_tbl WITH s AS (SELECT 44 AS col) SELECT * FROM s +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM cte_tbl +-- !query schema +struct +-- !query output +42 +43 +44 + + +-- !query +CREATE TABLE cte_tbl2 (col INT) USING csv +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH s AS (SELECT 45 AS col) +FROM s +INSERT INTO cte_tbl SELECT col +INSERT INTO cte_tbl2 SELECT col +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM cte_tbl +-- !query schema +struct +-- !query output +42 +43 +44 +45 + + +-- !query +SELECT * FROM cte_tbl2 +-- !query schema +struct +-- !query output +45 + + +-- !query +DROP TABLE cte_tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE cte_tbl2 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cte-legacy.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cte-legacy.sql.out new file mode 100644 index 000000000000..1255e8b51f30 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cte-legacy.sql.out @@ -0,0 +1,417 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +WITH t as ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 +) +SELECT * FROM t +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 1) + SELECT * FROM t +) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT ( + WITH t AS (SELECT 1) + SELECT * FROM t +) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT ( + WITH unreferenced AS (SELECT id) + SELECT 1 +) FROM range(1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT ( + WITH unreferenced AS (SELECT 1) + SELECT id +) FROM range(1) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT * FROM + ( + WITH cte AS (SELECT * FROM range(10)) + SELECT * FROM cte WHERE id = 8 + ) a +UNION +SELECT * FROM cte +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 120, + "stopIndex" : 122, + "fragment" : "cte" + } ] +} + + +-- !query +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +SELECT * FROM t2 +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +WITH + t(c) AS (SELECT 1), + t2 AS ( + SELECT ( + SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) + ) + ) +SELECT * FROM t2 +-- !query schema +struct +-- !query output +1 + + +-- !query +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2), + t2 AS ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) + SELECT * FROM t2 + ) +SELECT * FROM t2 +-- !query schema +struct<2:int> +-- !query output +2 + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +) +-- !query schema +struct +-- !query output +2 + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query schema +struct +-- !query output +2 + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query schema +struct +-- !query output +3 + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t +) +-- !query schema +struct +-- !query output +1 + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query schema +struct +-- !query output +1 + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query schema +struct +-- !query output +1 + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT * FROM t +WHERE c IN ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +) +-- !query schema +struct +-- !query output +1 + + +-- !query +WITH + t AS ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 + ), + t2 AS (SELECT 2) +SELECT * FROM t +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +WITH + abc AS (SELECT 1), + t AS ( + WITH aBc AS (SELECT 2) + SELECT * FROM aBC + ) +SELECT * FROM t +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +WITH abc AS (SELECT 1) +SELECT ( + WITH aBc AS (SELECT 2) + SELECT * FROM aBC +) +-- !query schema +struct +-- !query output +1 + + +-- !query +WITH + t1 AS (SELECT 1), + t2 AS ( + WITH t3 AS ( + SELECT * FROM t1 + ) + SELECT * FROM t3 + ) +SELECT * FROM t2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 73, + "stopIndex" : 74, + "fragment" : "t1" + } ] +} + + +-- !query +WITH cte_outer AS ( + SELECT 1 +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_outer + ) + SELECT * FROM cte_inner +) +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +WITH cte_outer AS ( + SELECT 1 +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM ( + WITH cte_inner_inner AS ( + SELECT * FROM cte_outer + ) + SELECT * FROM cte_inner_inner + ) + ) + SELECT * FROM cte_inner +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte_outer`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 146, + "stopIndex" : 154, + "fragment" : "cte_outer" + } ] +} + + +-- !query +WITH cte_outer AS ( + WITH cte_invisible_inner AS ( + SELECT 1 + ) + SELECT * FROM cte_invisible_inner +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_invisible_inner + ) + SELECT * FROM cte_inner +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte_invisible_inner`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 164, + "stopIndex" : 182, + "fragment" : "cte_invisible_inner" + } ] +} + + +-- !query +WITH cte_outer AS ( + SELECT * FROM ( + WITH cte_invisible_inner AS ( + SELECT 1 + ) + SELECT * FROM cte_invisible_inner + ) +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_invisible_inner + ) + SELECT * FROM cte_inner +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte_invisible_inner`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 194, + "stopIndex" : 212, + "fragment" : "cte_invisible_inner" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cte-nested.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cte-nested.sql.out new file mode 100644 index 000000000000..7cf488ce8cad --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cte-nested.sql.out @@ -0,0 +1,389 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +WITH t as ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 +) +SELECT * FROM t +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 1) + SELECT * FROM t +) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT ( + WITH t AS (SELECT 1) + SELECT * FROM t +) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT ( + WITH unreferenced AS (SELECT id) + SELECT 1 +) FROM range(1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT ( + WITH unreferenced AS (SELECT 1) + SELECT id +) FROM range(1) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT * FROM + ( + WITH cte AS (SELECT * FROM range(10)) + SELECT * FROM cte WHERE id = 8 + ) a +UNION +SELECT * FROM cte +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 120, + "stopIndex" : 122, + "fragment" : "cte" + } ] +} + + +-- !query +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +SELECT * FROM t2 +-- !query schema +struct<2:int> +-- !query output +2 + + +-- !query +WITH + t(c) AS (SELECT 1), + t2 AS ( + SELECT ( + SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) + ) + ) +SELECT * FROM t2 +-- !query schema +struct +-- !query output +2 + + +-- !query +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2), + t2 AS ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) + SELECT * FROM t2 + ) +SELECT * FROM t2 +-- !query schema +struct<3:int> +-- !query output +3 + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +) +-- !query schema +struct +-- !query output +2 + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query schema +struct +-- !query output +2 + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query schema +struct +-- !query output +3 + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t +) +-- !query schema +struct +-- !query output +2 + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query schema +struct +-- !query output +2 + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query schema +struct +-- !query output +3 + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT * FROM t +WHERE c IN ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +) +-- !query schema +struct +-- !query output + + + +-- !query +WITH + t AS ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 + ), + t2 AS (SELECT 2) +SELECT * FROM t +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +WITH + abc AS (SELECT 1), + t AS ( + WITH aBc AS (SELECT 2) + SELECT * FROM aBC + ) +SELECT * FROM t +-- !query schema +struct<2:int> +-- !query output +2 + + +-- !query +WITH abc AS (SELECT 1) +SELECT ( + WITH aBc AS (SELECT 2) + SELECT * FROM aBC +) +-- !query schema +struct +-- !query output +2 + + +-- !query +WITH + t1 AS (SELECT 1), + t2 AS ( + WITH t3 AS ( + SELECT * FROM t1 + ) + SELECT * FROM t3 + ) +SELECT * FROM t2 +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +WITH cte_outer AS ( + SELECT 1 +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_outer + ) + SELECT * FROM cte_inner +) +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +WITH cte_outer AS ( + SELECT 1 +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM ( + WITH cte_inner_inner AS ( + SELECT * FROM cte_outer + ) + SELECT * FROM cte_inner_inner + ) + ) + SELECT * FROM cte_inner +) +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +WITH cte_outer AS ( + WITH cte_invisible_inner AS ( + SELECT 1 + ) + SELECT * FROM cte_invisible_inner +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_invisible_inner + ) + SELECT * FROM cte_inner +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte_invisible_inner`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 164, + "stopIndex" : 182, + "fragment" : "cte_invisible_inner" + } ] +} + + +-- !query +WITH cte_outer AS ( + SELECT * FROM ( + WITH cte_invisible_inner AS ( + SELECT 1 + ) + SELECT * FROM cte_invisible_inner + ) +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_invisible_inner + ) + SELECT * FROM cte_inner +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte_invisible_inner`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 194, + "stopIndex" : 212, + "fragment" : "cte_invisible_inner" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cte-nonlegacy.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cte-nonlegacy.sql.out new file mode 100644 index 000000000000..94ef47397eff --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cte-nonlegacy.sql.out @@ -0,0 +1,461 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +WITH t as ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 +) +SELECT * FROM t +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 1) + SELECT * FROM t +) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT ( + WITH t AS (SELECT 1) + SELECT * FROM t +) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT ( + WITH unreferenced AS (SELECT id) + SELECT 1 +) FROM range(1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT ( + WITH unreferenced AS (SELECT 1) + SELECT id +) FROM range(1) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT * FROM + ( + WITH cte AS (SELECT * FROM range(10)) + SELECT * FROM cte WHERE id = 8 + ) a +UNION +SELECT * FROM cte +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 120, + "stopIndex" : 122, + "fragment" : "cte" + } ] +} + + +-- !query +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +SELECT * FROM t2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_ALIAS_IN_NESTED_CTE", + "sqlState" : "42KD0", + "messageParameters" : { + "config" : "\"spark.sql.legacy.ctePrecedencePolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "name" : "`t`" + } +} + + +-- !query +WITH + t(c) AS (SELECT 1), + t2 AS ( + SELECT ( + SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) + ) + ) +SELECT * FROM t2 +-- !query schema +struct +-- !query output +2 + + +-- !query +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2), + t2 AS ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) + SELECT * FROM t2 + ) +SELECT * FROM t2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_ALIAS_IN_NESTED_CTE", + "sqlState" : "42KD0", + "messageParameters" : { + "config" : "\"spark.sql.legacy.ctePrecedencePolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "name" : "`t`" + } +} + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +) +-- !query schema +struct +-- !query output +2 + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query schema +struct +-- !query output +2 + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query schema +struct +-- !query output +3 + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_ALIAS_IN_NESTED_CTE", + "sqlState" : "42KD0", + "messageParameters" : { + "config" : "\"spark.sql.legacy.ctePrecedencePolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "name" : "`t`" + } +} + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_ALIAS_IN_NESTED_CTE", + "sqlState" : "42KD0", + "messageParameters" : { + "config" : "\"spark.sql.legacy.ctePrecedencePolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "name" : "`t`" + } +} + + +-- !query +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_ALIAS_IN_NESTED_CTE", + "sqlState" : "42KD0", + "messageParameters" : { + "config" : "\"spark.sql.legacy.ctePrecedencePolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "name" : "`t`" + } +} + + +-- !query +WITH t(c) AS (SELECT 1) +SELECT * FROM t +WHERE c IN ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_ALIAS_IN_NESTED_CTE", + "sqlState" : "42KD0", + "messageParameters" : { + "config" : "\"spark.sql.legacy.ctePrecedencePolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "name" : "`t`" + } +} + + +-- !query +WITH + t AS ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 + ), + t2 AS (SELECT 2) +SELECT * FROM t +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +WITH + abc AS (SELECT 1), + t AS ( + WITH aBc AS (SELECT 2) + SELECT * FROM aBC + ) +SELECT * FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_ALIAS_IN_NESTED_CTE", + "sqlState" : "42KD0", + "messageParameters" : { + "config" : "\"spark.sql.legacy.ctePrecedencePolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "name" : "`aBc`" + } +} + + +-- !query +WITH abc AS (SELECT 1) +SELECT ( + WITH aBc AS (SELECT 2) + SELECT * FROM aBC +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_ALIAS_IN_NESTED_CTE", + "sqlState" : "42KD0", + "messageParameters" : { + "config" : "\"spark.sql.legacy.ctePrecedencePolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "name" : "`aBc`" + } +} + + +-- !query +WITH + t1 AS (SELECT 1), + t2 AS ( + WITH t3 AS ( + SELECT * FROM t1 + ) + SELECT * FROM t3 + ) +SELECT * FROM t2 +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +WITH cte_outer AS ( + SELECT 1 +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_outer + ) + SELECT * FROM cte_inner +) +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +WITH cte_outer AS ( + SELECT 1 +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM ( + WITH cte_inner_inner AS ( + SELECT * FROM cte_outer + ) + SELECT * FROM cte_inner_inner + ) + ) + SELECT * FROM cte_inner +) +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +WITH cte_outer AS ( + WITH cte_invisible_inner AS ( + SELECT 1 + ) + SELECT * FROM cte_invisible_inner +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_invisible_inner + ) + SELECT * FROM cte_inner +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte_invisible_inner`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 164, + "stopIndex" : 182, + "fragment" : "cte_invisible_inner" + } ] +} + + +-- !query +WITH cte_outer AS ( + SELECT * FROM ( + WITH cte_invisible_inner AS ( + SELECT 1 + ) + SELECT * FROM cte_invisible_inner + ) +) +SELECT * FROM ( + WITH cte_inner AS ( + SELECT * FROM cte_invisible_inner + ) + SELECT * FROM cte_inner +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte_invisible_inner`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 194, + "stopIndex" : 212, + "fragment" : "cte_invisible_inner" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cte.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cte.sql.out new file mode 100644 index 000000000000..5dd2fdeca5ec --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/cte.sql.out @@ -0,0 +1,626 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t as select * from values 0, 1, 2 as t(id) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2 as select * from values 0, 1 as t(id) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t3 as select * from t +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH s AS (SELECT 1 FROM s) SELECT * FROM s +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`s`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 26, + "fragment" : "s" + } ] +} + + +-- !query +WITH r AS (SELECT (SELECT * FROM r)) +SELECT * FROM r +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`r`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 34, + "fragment" : "r" + } ] +} + + +-- !query +WITH t AS (SELECT 1 FROM t) SELECT * FROM t +-- !query schema +struct<1:int> +-- !query output +1 +1 +1 + + +-- !query +WITH t AS (SELECT 1) SELECT * FROM t3 +-- !query schema +struct +-- !query output +0 +1 +2 + + +-- !query +WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`s2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 28, + "fragment" : "s2" + } ] +} + + +-- !query +WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1 cross join t2 +-- !query schema +struct +-- !query output +0 2 +0 2 +1 2 +1 2 + + +-- !query +WITH CTE1 AS ( + SELECT b.id AS id + FROM T2 a + CROSS JOIN (SELECT id AS id FROM T2) b +) +SELECT t1.id AS c1, + t2.id AS c2 +FROM CTE1 t1 + CROSS JOIN CTE1 t2 +-- !query schema +struct +-- !query output +0 0 +0 0 +0 0 +0 0 +0 1 +0 1 +0 1 +0 1 +1 0 +1 0 +1 0 +1 0 +1 1 +1 1 +1 1 +1 1 + + +-- !query +WITH t(x) AS (SELECT 1) +SELECT * FROM t WHERE x = 1 +-- !query schema +struct +-- !query output +1 + + +-- !query +WITH t(x, y) AS (SELECT 1, 2) +SELECT * FROM t WHERE x = 1 AND y = 2 +-- !query schema +struct +-- !query output +1 2 + + +-- !query +WITH t(x, x) AS (SELECT 1, 2) +SELECT * FROM t +-- !query schema +struct +-- !query output +1 2 + + +-- !query +WITH t() AS (SELECT 1) +SELECT * FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "')'", + "hint" : "" + } +} + + +-- !query +WITH + t(x) AS (SELECT 1), + t(x) AS (SELECT 2) +SELECT * FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "DUPLICATED_CTE_NAMES", + "sqlState" : "42602", + "messageParameters" : { + "duplicateNames" : "`t`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "WITH\n t(x) AS (SELECT 1),\n t(x) AS (SELECT 2)\nSELECT * FROM t" + } ] +} + + +-- !query +WITH t AS (SELECT 1 FROM non_existing_table) +SELECT 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`non_existing_table`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 43, + "fragment" : "non_existing_table" + } ] +} + + +-- !query +SELECT count(*) FROM ( + WITH q1(x) AS (SELECT random() FROM range(1, 5)) + SELECT * FROM q1 + UNION + SELECT * FROM q1 +) ss +-- !query schema +struct +-- !query output +4 + + +-- !query +WITH w1(c1) AS + (WITH w2(c2) AS + (WITH w3(c3) AS + (WITH w4(c4) AS + (WITH w5(c5) AS + (WITH w6(c6) AS + (WITH w7(c7) AS + (WITH w8(c8) AS + (SELECT 1) + SELECT * FROM w8) + SELECT * FROM w7) + SELECT * FROM w6) + SELECT * FROM w5) + SELECT * FROM w4) + SELECT * FROM w3) + SELECT * FROM w2) +SELECT * FROM w1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT ( WITH cte(foo) AS ( VALUES(id) ) + SELECT (SELECT foo FROM cte) ) +FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_INLINE_TABLE.CANNOT_EVALUATE_EXPRESSION_IN_INLINE_TABLE", + "sqlState" : "42000", + "messageParameters" : { + "expr" : "\"outer(t.id)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 37, + "fragment" : "id" + } ] +} + + +-- !query +WITH same_name AS (SELECT 42) +SELECT * FROM same_name, (SELECT 10) AS same_name +-- !query schema +struct<42:int,10:int> +-- !query output +42 10 + + +-- !query +WITH same_name(x) AS (SELECT 42) +SELECT same_name.x FROM (SELECT 10) AS same_name(x), same_name +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`same_name`.`x`", + "referenceNames" : "[`same_name`.`x`, `same_name`.`x`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 41, + "stopIndex" : 51, + "fragment" : "same_name.x" + } ] +} + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query + +WITH q AS (SELECT 'foo' AS x) +SELECT x, typeof(x) FROM q +-- !query schema +struct +-- !query output +foo string + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query + +with cte as (select id as id_alias from t) +select id from cte +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`id`", + "proposal" : "`id_alias`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 52, + "fragment" : "id" + } ] +} + + +-- !query +with r1 as (select * from r2), + r2 as (select 1) +select 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`r2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 28, + "fragment" : "r2" + } ] +} + + +-- !query +SELECT * FROM + (WITH q AS (select 1 x) SELECT x+1 AS y FROM q) +-- !query schema +struct +-- !query output +2 + + +-- !query +select (with q as (select 1 x) select * from q) +-- !query schema +struct +-- !query output +1 + + +-- !query +select 1 in (with q as (select 1) select * from q) +-- !query schema +struct<(1 IN (listquery())):boolean> +-- !query output +true + + +-- !query +SELECT * FROM + (WITH q AS (select 1 x) SELECT x+1 AS y FROM q), + q +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`q`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 68, + "stopIndex" : 68, + "fragment" : "q" + } ] +} + + +-- !query +WITH T1 as (select 1 a) +select * +from + T1 x, + (WITH T1 as (select 2 b) select * from T1) y, + T1 z +-- !query schema +struct +-- !query output +1 2 1 + + +-- !query +WITH TTtt as (select 1 a), + `tTTt_2` as (select 2 a) +select * +from + (WITH TtTt as (select 3 c) select * from ttTT, `tttT_2`) +-- !query schema +struct +-- !query output +3 2 + + +-- !query +select + (WITH q AS (select T.x) select * from q) +from (select 1 x, 2 y) T +-- !query schema +struct +-- !query output +1 + + +-- !query +select + (WITH q AS (select 3 z) select x + t.y + z from q) +from (select 1 x, 2 y) T +-- !query schema +struct +-- !query output +6 + + +-- !query +WITH q1 as (select 1 x) +select * from + (with q2 as (select * from q1) select * from q2) +-- !query schema +struct +-- !query output +1 + + +-- !query +WITH q1 as (select 1 x) +select * from + (with q1 as (select x+1 from q1) select * from q1) +-- !query schema +struct<(x + 1):int> +-- !query output +2 + + +-- !query +with cte1 as (select 42), cte1 as (select 42) select * FROM cte1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "DUPLICATED_CTE_NAMES", + "sqlState" : "42602", + "messageParameters" : { + "duplicateNames" : "`cte1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 64, + "fragment" : "with cte1 as (select 42), cte1 as (select 42) select * FROM cte1" + } ] +} + + +-- !query +with cte1 as (Select id as j from t) +select * from cte1 where j = (select max(j) from cte1 as cte2) +-- !query schema +struct +-- !query output +2 + + +-- !query +with cte AS (SELECT * FROM va) SELECT * FROM cte +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`va`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 29, + "fragment" : "va" + } ] +} + + +-- !query +with cte as (select * from cte) select * from cte +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`cte`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 30, + "fragment" : "cte" + } ] +} + + +-- !query +DROP VIEW IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS t3 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/current_database_catalog.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/current_database_catalog.sql.out new file mode 100644 index 000000000000..67db0adee7f0 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/current_database_catalog.sql.out @@ -0,0 +1,7 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select current_database(), current_schema(), current_catalog() +-- !query schema +struct +-- !query output +default default spark_catalog diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/date.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/date.sql.out new file mode 100644 index 000000000000..aec320fe6b1a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/date.sql.out @@ -0,0 +1,1219 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view date_view as select '2011-11-11' date_str, '1' int_str +-- !query schema +struct<> +-- !query output + + + +-- !query +select date '2019-01-01\t' +-- !query schema +struct +-- !query output +2019-01-01 + + +-- !query +select date '2020-01-01中文' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2020-01-01中文'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "date '2020-01-01中文'" + } ] +} + + +-- !query +select make_date(2019, 1, 1), make_date(12, 12, 12) +-- !query schema +struct +-- !query output +2019-01-01 0012-12-12 + + +-- !query +select make_date(2000, 13, 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for MonthOfYear (valid values 1 - 12): 13" + } +} + + +-- !query +select make_date(2000, 1, 33) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for DayOfMonth (valid values 1 - 28/31): 33" + } +} + + +-- !query +select date'015' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'015'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "date'015'" + } ] +} + + +-- !query +select date'2021-4294967297-11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2021-4294967297-11'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "date'2021-4294967297-11'" + } ] +} + + +-- !query +select current_date = current_date +-- !query schema +struct<(current_date() = current_date()):boolean> +-- !query output +true + + +-- !query +select current_date() = current_date() +-- !query schema +struct<(current_date() = current_date()):boolean> +-- !query output +true + + +-- !query +select curdate(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "0", + "functionName" : "`curdate`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "curdate(1)" + } ] +} + + +-- !query +select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null) +-- !query schema +struct +-- !query output +1970-01-01 1972-09-27 NULL + + +-- !query +select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(null) +-- !query schema +struct +-- !query output +0 18600 NULL + + +-- !query +select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL 2016-12-31 2016-12-31 + + +-- !query +select to_date("16", "dd") +-- !query schema +struct +-- !query output +1970-01-16 + + +-- !query +select to_date("02-29", "MM-dd") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Invalid date 'February 29' as '1970' is not a leap year" + } +} + + +-- !query +select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), + dayofweek('1582-10-15 13:10:15'), dayofweek(timestamp_ltz'1582-10-15 13:10:15'), dayofweek(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +7 5 7 NULL 6 6 6 + + +-- !query +select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null), + weekday('1582-10-15 13:10:15'), weekday(timestamp_ltz'1582-10-15 13:10:15'), weekday(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +5 3 5 NULL 4 4 4 + + +-- !query +select year('1500-01-01'), year('1582-10-15 13:10:15'), year(timestamp_ltz'1582-10-15 13:10:15'), year(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +1500 1582 1582 1582 + + +-- !query +select month('1500-01-01'), month('1582-10-15 13:10:15'), month(timestamp_ltz'1582-10-15 13:10:15'), month(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +1 10 10 10 + + +-- !query +select dayOfYear('1500-01-01'), dayOfYear('1582-10-15 13:10:15'), dayOfYear(timestamp_ltz'1582-10-15 13:10:15'), dayOfYear(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +1 288 288 288 + + +-- !query +select next_day("2015-07-23", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day("2015-07-23", "xx") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ILLEGAL_DAY_OF_WEEK", + "sqlState" : "22009", + "messageParameters" : { + "string" : "xx" + } +} + + +-- !query +select next_day("2015-07-23 12:12:12", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day(timestamp_ltz"2015-07-23 12:12:12", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day(timestamp_ntz"2015-07-23 12:12:12", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day("xx", "Mon") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'xx'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "next_day(\"xx\", \"Mon\")" + } ] +} + + +-- !query +select next_day(null, "Mon") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select next_day(null, "xx") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_add(date'2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1Y) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1S) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1L) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "date_add('2011-11-11', 1L)" + } ] +} + + +-- !query +select date_add('2011-11-11', 1.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_add('2011-11-11', 1.0)" + } ] +} + + +-- !query +select date_add('2011-11-11', 1E1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_add('2011-11-11', 1E1)" + } ] +} + + +-- !query +select date_add('2011-11-11', '1') +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', '1.2') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1.2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "date_add('2011-11-11', '1.2')" + } ] +} + + +-- !query +select date_add(null, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_add(date'2011-11-11', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_add(timestamp_ltz'2011-11-11 12:12:12', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add(timestamp_ntz'2011-11-11 12:12:12', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_sub(date'2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub('2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub('2011-11-11', 1Y) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub('2011-11-11', 1S) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub('2011-11-11', 1L) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "date_sub('2011-11-11', 1L)" + } ] +} + + +-- !query +select date_sub('2011-11-11', 1.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_sub('2011-11-11', 1.0)" + } ] +} + + +-- !query +select date_sub('2011-11-11', 1E1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_sub('2011-11-11', 1E1)" + } ] +} + + +-- !query +select date_sub(date'2011-11-11', '1') +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub(date'2011-11-11', '1.2') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1.2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "date_sub(date'2011-11-11', '1.2')" + } ] +} + + +-- !query +select date_sub(null, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_sub(date'2011-11-11', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_sub(timestamp_ltz'2011-11-11 12:12:12', 1) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub(timestamp_ntz'2011-11-11 12:12:12', 1) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_add('2011-11-11', int_str) from date_view +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_sub('2011-11-11', int_str) from date_view +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_add(date_str, 1) from date_view +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_sub(date_str, 1) from date_view +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date '2011-11-11' + 1E1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(DATE '2011-11-11', 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "date '2011-11-11' + 1E1" + } ] +} + + +-- !query +select date '2001-09-28' + 7Y +-- !query schema +struct +-- !query output +2001-10-05 + + +-- !query +select 7S + date '2001-09-28' +-- !query schema +struct +-- !query output +2001-10-05 + + +-- !query +select date '2001-10-01' - 7 +-- !query schema +struct +-- !query output +2001-09-24 + + +-- !query +select date '2001-10-01' - date '2001-09-28' +-- !query schema +struct<(DATE '2001-10-01' - DATE '2001-09-28'):interval day> +-- !query output +3 00:00:00.000000000 + + +-- !query +select date '2001-10-01' - '2001-09-28' +-- !query schema +struct<(DATE '2001-10-01' - 2001-09-28):interval day> +-- !query output +3 00:00:00.000000000 + + +-- !query +select '2001-10-01' - date '2001-09-28' +-- !query schema +struct<(2001-10-01 - DATE '2001-09-28'):interval day> +-- !query output +3 00:00:00.000000000 + + +-- !query +select date '2001-09-28' - null +-- !query schema +struct +-- !query output +NULL + + +-- !query +select null - date '2019-10-06' +-- !query schema +struct<(NULL - DATE '2019-10-06'):interval day> +-- !query output +NULL + + +-- !query +select date_str - date '2001-09-28' from date_view +-- !query schema +struct<(date_str - DATE '2001-09-28'):interval day> +-- !query output +3696 00:00:00.000000000 + + +-- !query +select date '2001-09-28' - date_str from date_view +-- !query schema +struct<(DATE '2001-09-28' - date_str):interval day> +-- !query output +-3696 00:00:00.000000000 + + +-- !query +select date'2011-11-11' + '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"DATE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "date'2011-11-11' + '1'" + } ] +} + + +-- !query +select '1' + date'2011-11-11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"DATE '2011-11-11'\"", + "inputType" : "\"DATE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(1, DATE '2011-11-11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' + date'2011-11-11'" + } ] +} + + +-- !query +select date'2011-11-11' + null +-- !query schema +struct +-- !query output +NULL + + +-- !query +select null + date'2011-11-11' +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date '2012-01-01' - interval '2-2' year to month, + date '2011-11-11' - interval '2' day, + date '2012-01-01' + interval '-2-2' year to month, + date '2011-11-11' + interval '-2' month, + - interval '2-2' year to month + date '2012-01-01', + interval '-2' day + date '2011-11-11' +-- !query schema +struct +-- !query output +2009-11-01 2011-11-09 2009-11-01 2011-09-11 2009-11-01 2011-11-09 + + +-- !query +select to_date('26/October/2015', 'dd/MMMMM/yyyy') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd/MMMMM/yyyy'" + } +} + + +-- !query +select from_json('{"d":"26/October/2015"}', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd/MMMMM/yyyy'" + } +} + + +-- !query +select from_csv('26/October/2015', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd/MMMMM/yyyy'" + } +} + + +-- !query +select dateadd(MICROSECOND, 1001, timestamp'2022-02-25 01:02:03.123') +-- !query schema +struct +-- !query output +2022-02-25 01:02:03.124001 + + +-- !query +select date_add(MILLISECOND, -1, timestamp'2022-02-25 01:02:03.456') +-- !query schema +struct +-- !query output +2022-02-25 01:02:03.455 + + +-- !query +select dateadd(SECOND, 58, timestamp'2022-02-25 01:02:03') +-- !query schema +struct +-- !query output +2022-02-25 01:03:01 + + +-- !query +select date_add(MINUTE, -100, date'2022-02-25') +-- !query schema +struct +-- !query output +2022-02-24 22:20:00 + + +-- !query +select dateadd(HOUR, -1, timestamp'2022-02-25 01:02:03') +-- !query schema +struct +-- !query output +2022-02-25 00:02:03 + + +-- !query +select date_add(DAY, 367, date'2022-02-25') +-- !query schema +struct +-- !query output +2023-02-27 00:00:00 + + +-- !query +select dateadd(WEEK, -4, timestamp'2022-02-25 01:02:03') +-- !query schema +struct +-- !query output +2022-01-28 01:02:03 + + +-- !query +select date_add(MONTH, -1, timestamp'2022-02-25 01:02:03') +-- !query schema +struct +-- !query output +2022-01-25 01:02:03 + + +-- !query +select dateadd(QUARTER, 5, date'2022-02-25') +-- !query schema +struct +-- !query output +2023-05-25 00:00:00 + + +-- !query +select date_add(YEAR, 1, date'2022-02-25') +-- !query schema +struct +-- !query output +2023-02-25 00:00:00 + + +-- !query +select dateadd('MICROSECOND', 1001, timestamp'2022-02-25 01:02:03.123') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`dateadd`", + "invalidValue" : "'MICROSECOND'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "dateadd('MICROSECOND', 1001, timestamp'2022-02-25 01:02:03.123')" + } ] +} + + +-- !query +select date_add('QUARTER', 5, date'2022-02-25') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`date_add`", + "invalidValue" : "'QUARTER'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "date_add('QUARTER', 5, date'2022-02-25')" + } ] +} + + +-- !query +select datediff(MICROSECOND, timestamp'2022-02-25 01:02:03.123', timestamp'2022-02-25 01:02:03.124001') +-- !query schema +struct +-- !query output +1001 + + +-- !query +select date_diff(MILLISECOND, timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select datediff(SECOND, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 01:03:01') +-- !query schema +struct +-- !query output +58 + + +-- !query +select date_diff(MINUTE, date'2022-02-25', timestamp'2022-02-24 22:20:00') +-- !query schema +struct +-- !query output +-100 + + +-- !query +select datediff(HOUR, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 00:02:03') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select date_diff(DAY, date'2022-02-25', timestamp'2023-02-27 00:00:00') +-- !query schema +struct +-- !query output +367 + + +-- !query +select datediff(WEEK, timestamp'2022-02-25 01:02:03', timestamp'2022-01-28 01:02:03') +-- !query schema +struct +-- !query output +-4 + + +-- !query +select date_diff(MONTH, timestamp'2022-02-25 01:02:03', timestamp'2022-01-25 01:02:03') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select datediff(QUARTER, date'2022-02-25', date'2023-05-25') +-- !query schema +struct +-- !query output +5 + + +-- !query +select date_diff(YEAR, date'2022-02-25', date'2023-02-25') +-- !query schema +struct +-- !query output +1 + + +-- !query +select date_diff('MILLISECOND', timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`date_diff`", + "invalidValue" : "'MILLISECOND'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 103, + "fragment" : "date_diff('MILLISECOND', timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455')" + } ] +} + + +-- !query +select datediff('YEAR', date'2022-02-25', date'2023-02-25') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`datediff`", + "invalidValue" : "'YEAR'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "datediff('YEAR', date'2022-02-25', date'2023-02-25')" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-formatting-invalid.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-formatting-invalid.sql.out new file mode 100644 index 000000000000..66b36befaa33 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-formatting-invalid.sql.out @@ -0,0 +1,567 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select date_format('2018-11-17 13:33:33.333', 'GGGGG') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'GGGGG'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'yyyyyyy') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'yyyyyyy'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'qqqqq') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_DATETIME_PATTERN.LENGTH", + "sqlState" : "22007", + "messageParameters" : { + "pattern" : "qqqqq" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'QQQQQ') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_DATETIME_PATTERN.LENGTH", + "sqlState" : "22007", + "messageParameters" : { + "pattern" : "QQQQQ" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'MMMMM') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'MMMMM'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'LLLLL') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'LLLLL'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'EEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'EEEEE'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'FF') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'FF'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'ddd') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'ddd'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'DDDD') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'DDDD'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'HHH') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'HHH'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'hhh') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'hhh'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'kkk') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'kkk'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'KKK') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'KKK'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'mmm') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'mmm'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'sss') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'sss'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'SSSSSSSSSS') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'SSSSSSSSSS'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'aa') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'aa'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'V') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Pattern letter count must be 2: V + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'zzzzz') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'zzzzz'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'XXXXXX') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Too many pattern letters: X + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'ZZZZZZ') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'ZZZZZZ'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'OO') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Pattern letter count must be 1 or 4: O + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'xxxxxx') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Too many pattern letters: x + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'A') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_DATETIME_PATTERN.ILLEGAL_CHARACTER", + "sqlState" : "22007", + "messageParameters" : { + "c" : "A", + "pattern" : "A" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'n') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_DATETIME_PATTERN.ILLEGAL_CHARACTER", + "sqlState" : "22007", + "messageParameters" : { + "c" : "n", + "pattern" : "n" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'N') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_DATETIME_PATTERN.ILLEGAL_CHARACTER", + "sqlState" : "22007", + "messageParameters" : { + "c" : "N", + "pattern" : "N" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'p') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_DATETIME_PATTERN.ILLEGAL_CHARACTER", + "sqlState" : "22007", + "messageParameters" : { + "c" : "p", + "pattern" : "p" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'Y') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'Y'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'w') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'w'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'W') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'W'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'u') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'u'" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'e') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_WEEK_BASED_PATTERN", + "sqlState" : "42K0B", + "messageParameters" : { + "c" : "e" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'c') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_WEEK_BASED_PATTERN", + "sqlState" : "42K0B", + "messageParameters" : { + "c" : "c" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'B') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_DATETIME_PATTERN.ILLEGAL_CHARACTER", + "sqlState" : "22007", + "messageParameters" : { + "c" : "B", + "pattern" : "B" + } +} + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'C') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Unknown pattern letter: C + + +-- !query +select date_format('2018-11-17 13:33:33.333', 'I') +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Unknown pattern letter: I diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-formatting-legacy.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-formatting-legacy.sql.out new file mode 100644 index 000000000000..b920f8f29658 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-formatting-legacy.sql.out @@ -0,0 +1,422 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view v as select col from values + (timestamp '1582-06-01 11:33:33.123UTC+080000'), + (timestamp '1970-01-01 00:00:00.000Europe/Paris'), + (timestamp '1970-12-31 23:59:59.999Asia/Srednekolymsk'), + (timestamp '1996-04-01 00:33:33.123Australia/Darwin'), + (timestamp '2018-11-17 13:33:33.123Z'), + (timestamp '2020-01-01 01:33:33.123Asia/Shanghai'), + (timestamp '2100-01-01 01:33:33.123America/Los_Angeles') t(col) +-- !query schema +struct<> +-- !query output + + + +-- !query +select col, date_format(col, 'G GG GGG GGGG'), to_char(col, 'G GG GGG GGGG'), to_varchar(col, 'G GG GGG GGGG') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 AD AD AD AD AD AD AD AD AD AD AD AD +1969-12-31 15:00:00 AD AD AD AD AD AD AD AD AD AD AD AD +1970-12-31 04:59:59.999 AD AD AD AD AD AD AD AD AD AD AD AD +1996-03-31 07:03:33.123 AD AD AD AD AD AD AD AD AD AD AD AD +2018-11-17 05:33:33.123 AD AD AD AD AD AD AD AD AD AD AD AD +2019-12-31 09:33:33.123 AD AD AD AD AD AD AD AD AD AD AD AD +2100-01-01 01:33:33.123 AD AD AD AD AD AD AD AD AD AD AD AD + + +-- !query +select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy'), to_char(col, 'y yy yyy yyyy yyyyy yyyyyy'), to_varchar(col, 'y yy yyy yyyy yyyyy yyyyyy') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 1582 82 1582 1582 01582 001582 1582 82 1582 1582 01582 001582 1582 82 1582 1582 01582 001582 +1969-12-31 15:00:00 1969 69 1969 1969 01969 001969 1969 69 1969 1969 01969 001969 1969 69 1969 1969 01969 001969 +1970-12-31 04:59:59.999 1970 70 1970 1970 01970 001970 1970 70 1970 1970 01970 001970 1970 70 1970 1970 01970 001970 +1996-03-31 07:03:33.123 1996 96 1996 1996 01996 001996 1996 96 1996 1996 01996 001996 1996 96 1996 1996 01996 001996 +2018-11-17 05:33:33.123 2018 18 2018 2018 02018 002018 2018 18 2018 2018 02018 002018 2018 18 2018 2018 02018 002018 +2019-12-31 09:33:33.123 2019 19 2019 2019 02019 002019 2019 19 2019 2019 02019 002019 2019 19 2019 2019 02019 002019 +2100-01-01 01:33:33.123 2100 00 2100 2100 02100 002100 2100 00 2100 2100 02100 002100 2100 00 2100 2100 02100 002100 + + +-- !query +select col, date_format(col, 'q qq'), to_char(col, 'q qq'), to_varchar(col, 'q qq') from v +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'q' + + +-- !query +select col, date_format(col, 'Q QQ QQQ QQQQ'), to_char(col, 'Q QQ QQQ QQQQ'), to_varchar(col, 'Q QQ QQQ QQQQ') from v +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'Q' + + +-- !query +select col, date_format(col, 'M MM MMM MMMM'), to_char(col, 'M MM MMM MMMM'), to_varchar(col, 'M MM MMM MMMM') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 5 05 May May 5 05 May May 5 05 May May +1969-12-31 15:00:00 12 12 Dec December 12 12 Dec December 12 12 Dec December +1970-12-31 04:59:59.999 12 12 Dec December 12 12 Dec December 12 12 Dec December +1996-03-31 07:03:33.123 3 03 Mar March 3 03 Mar March 3 03 Mar March +2018-11-17 05:33:33.123 11 11 Nov November 11 11 Nov November 11 11 Nov November +2019-12-31 09:33:33.123 12 12 Dec December 12 12 Dec December 12 12 Dec December +2100-01-01 01:33:33.123 1 01 Jan January 1 01 Jan January 1 01 Jan January + + +-- !query +select col, date_format(col, 'L LL'), to_char(col, 'L LL'), to_varchar(col, 'L LL') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 5 05 5 05 5 05 +1969-12-31 15:00:00 12 12 12 12 12 12 +1970-12-31 04:59:59.999 12 12 12 12 12 12 +1996-03-31 07:03:33.123 3 03 3 03 3 03 +2018-11-17 05:33:33.123 11 11 11 11 11 11 +2019-12-31 09:33:33.123 12 12 12 12 12 12 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'E EE EEE EEEE'), to_char(col, 'E EE EEE EEEE'), to_varchar(col, 'E EE EEE EEEE') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 Thu Thu Thu Thursday Thu Thu Thu Thursday Thu Thu Thu Thursday +1969-12-31 15:00:00 Wed Wed Wed Wednesday Wed Wed Wed Wednesday Wed Wed Wed Wednesday +1970-12-31 04:59:59.999 Thu Thu Thu Thursday Thu Thu Thu Thursday Thu Thu Thu Thursday +1996-03-31 07:03:33.123 Sun Sun Sun Sunday Sun Sun Sun Sunday Sun Sun Sun Sunday +2018-11-17 05:33:33.123 Sat Sat Sat Saturday Sat Sat Sat Saturday Sat Sat Sat Saturday +2019-12-31 09:33:33.123 Tue Tue Tue Tuesday Tue Tue Tue Tuesday Tue Tue Tue Tuesday +2100-01-01 01:33:33.123 Fri Fri Fri Friday Fri Fri Fri Friday Fri Fri Fri Friday + + +-- !query +select col, date_format(col, 'F'), to_char(col, 'F'), to_varchar(col, 'F') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 5 5 5 +1969-12-31 15:00:00 5 5 5 +1970-12-31 04:59:59.999 5 5 5 +1996-03-31 07:03:33.123 5 5 5 +2018-11-17 05:33:33.123 3 3 3 +2019-12-31 09:33:33.123 5 5 5 +2100-01-01 01:33:33.123 1 1 1 + + +-- !query +select col, date_format(col, 'd dd'), to_char(col, 'd dd'), to_varchar(col, 'd dd') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 31 31 31 31 31 31 +1969-12-31 15:00:00 31 31 31 31 31 31 +1970-12-31 04:59:59.999 31 31 31 31 31 31 +1996-03-31 07:03:33.123 31 31 31 31 31 31 +2018-11-17 05:33:33.123 17 17 17 17 17 17 +2019-12-31 09:33:33.123 31 31 31 31 31 31 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'DD'), to_char(col, 'DD'), to_varchar(col, 'DD') from v where col = timestamp '2100-01-01 01:33:33.123America/Los_Angeles' +-- !query schema +struct +-- !query output +2100-01-01 01:33:33.123 01 01 01 + + +-- !query +select col, date_format(col, 'D DDD'), to_char(col, 'D DDD'), to_varchar(col, 'D DDD') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 151 151 151 151 151 151 +1969-12-31 15:00:00 365 365 365 365 365 365 +1970-12-31 04:59:59.999 365 365 365 365 365 365 +1996-03-31 07:03:33.123 91 091 91 091 91 091 +2018-11-17 05:33:33.123 321 321 321 321 321 321 +2019-12-31 09:33:33.123 365 365 365 365 365 365 +2100-01-01 01:33:33.123 1 001 1 001 1 001 + + +-- !query +select col, date_format(col, 'H HH'), to_char(col, 'H HH'), to_varchar(col, 'H HH') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 19 19 19 19 19 19 +1969-12-31 15:00:00 15 15 15 15 15 15 +1970-12-31 04:59:59.999 4 04 4 04 4 04 +1996-03-31 07:03:33.123 7 07 7 07 7 07 +2018-11-17 05:33:33.123 5 05 5 05 5 05 +2019-12-31 09:33:33.123 9 09 9 09 9 09 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'h hh'), to_char(col, 'h hh'), to_varchar(col, 'h hh') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 7 07 7 07 7 07 +1969-12-31 15:00:00 3 03 3 03 3 03 +1970-12-31 04:59:59.999 4 04 4 04 4 04 +1996-03-31 07:03:33.123 7 07 7 07 7 07 +2018-11-17 05:33:33.123 5 05 5 05 5 05 +2019-12-31 09:33:33.123 9 09 9 09 9 09 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'k kk'), to_char(col, 'k kk'), to_varchar(col, 'k kk') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 19 19 19 19 19 19 +1969-12-31 15:00:00 15 15 15 15 15 15 +1970-12-31 04:59:59.999 4 04 4 04 4 04 +1996-03-31 07:03:33.123 7 07 7 07 7 07 +2018-11-17 05:33:33.123 5 05 5 05 5 05 +2019-12-31 09:33:33.123 9 09 9 09 9 09 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'K KK'), to_char(col, 'K KK'), to_varchar(col, 'K KK') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 7 07 7 07 7 07 +1969-12-31 15:00:00 3 03 3 03 3 03 +1970-12-31 04:59:59.999 4 04 4 04 4 04 +1996-03-31 07:03:33.123 7 07 7 07 7 07 +2018-11-17 05:33:33.123 5 05 5 05 5 05 +2019-12-31 09:33:33.123 9 09 9 09 9 09 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'm mm'), to_char(col, 'm mm'), to_varchar(col, 'm mm') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 40 40 40 40 40 40 +1969-12-31 15:00:00 0 00 0 00 0 00 +1970-12-31 04:59:59.999 59 59 59 59 59 59 +1996-03-31 07:03:33.123 3 03 3 03 3 03 +2018-11-17 05:33:33.123 33 33 33 33 33 33 +2019-12-31 09:33:33.123 33 33 33 33 33 33 +2100-01-01 01:33:33.123 33 33 33 33 33 33 + + +-- !query +select col, date_format(col, 's ss'), to_char(col, 's ss'), to_varchar(col, 's ss') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 35 35 35 35 35 35 +1969-12-31 15:00:00 0 00 0 00 0 00 +1970-12-31 04:59:59.999 59 59 59 59 59 59 +1996-03-31 07:03:33.123 33 33 33 33 33 33 +2018-11-17 05:33:33.123 33 33 33 33 33 33 +2019-12-31 09:33:33.123 33 33 33 33 33 33 +2100-01-01 01:33:33.123 33 33 33 33 33 33 + + +-- !query +select col, date_format(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS'), to_char(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS'), to_varchar(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 123 123 123 0123 00123 000123 0000123 00000123 000000123 123 123 123 0123 00123 000123 0000123 00000123 000000123 123 123 123 0123 00123 000123 0000123 00000123 000000123 +1969-12-31 15:00:00 0 00 000 0000 00000 000000 0000000 00000000 000000000 0 00 000 0000 00000 000000 0000000 00000000 000000000 0 00 000 0000 00000 000000 0000000 00000000 000000000 +1970-12-31 04:59:59.999 999 999 999 0999 00999 000999 0000999 00000999 000000999 999 999 999 0999 00999 000999 0000999 00000999 000000999 999 999 999 0999 00999 000999 0000999 00000999 000000999 +1996-03-31 07:03:33.123 123 123 123 0123 00123 000123 0000123 00000123 000000123 123 123 123 0123 00123 000123 0000123 00000123 000000123 123 123 123 0123 00123 000123 0000123 00000123 000000123 +2018-11-17 05:33:33.123 123 123 123 0123 00123 000123 0000123 00000123 000000123 123 123 123 0123 00123 000123 0000123 00000123 000000123 123 123 123 0123 00123 000123 0000123 00000123 000000123 +2019-12-31 09:33:33.123 123 123 123 0123 00123 000123 0000123 00000123 000000123 123 123 123 0123 00123 000123 0000123 00000123 000000123 123 123 123 0123 00123 000123 0000123 00000123 000000123 +2100-01-01 01:33:33.123 123 123 123 0123 00123 000123 0000123 00000123 000000123 123 123 123 0123 00123 000123 0000123 00000123 000000123 123 123 123 0123 00123 000123 0000123 00000123 000000123 + + +-- !query +select col, date_format(col, 'a'), to_char(col, 'a'), to_varchar(col, 'a') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 PM PM PM +1969-12-31 15:00:00 PM PM PM +1970-12-31 04:59:59.999 AM AM AM +1996-03-31 07:03:33.123 AM AM AM +2018-11-17 05:33:33.123 AM AM AM +2019-12-31 09:33:33.123 AM AM AM +2100-01-01 01:33:33.123 AM AM AM + + +-- !query +select col, date_format(col, 'VV'), to_char(col, 'VV'), to_varchar(col, 'VV') from v +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'V' + + +-- !query +select col, date_format(col, 'z zz zzz zzzz'), to_char(col, 'z zz zzz zzzz'), to_varchar(col, 'z zz zzz zzzz') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time +1969-12-31 15:00:00 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time +1970-12-31 04:59:59.999 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time +1996-03-31 07:03:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time +2018-11-17 05:33:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time +2019-12-31 09:33:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time +2100-01-01 01:33:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time + + +-- !query +select col, date_format(col, 'X XX XXX'), to_char(col, 'X XX XXX'), to_varchar(col, 'X XX XXX') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 +1969-12-31 15:00:00 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 +1970-12-31 04:59:59.999 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 +1996-03-31 07:03:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 +2018-11-17 05:33:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 +2019-12-31 09:33:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 +2100-01-01 01:33:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 + + +-- !query +select col, date_format(col, 'XXXX XXXXX'), to_char(col, 'XXXX XXXXX'), to_varchar(col, 'XXXX XXXXX') from v +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +invalid ISO 8601 format: length=4 + + +-- !query +select col, date_format(col, 'Z ZZ ZZZ ZZZZ ZZZZZ'), to_char(col, 'Z ZZ ZZZ ZZZZ ZZZZZ'), to_varchar(col, 'Z ZZ ZZZ ZZZZ ZZZZZ') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 +1969-12-31 15:00:00 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 +1970-12-31 04:59:59.999 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 +1996-03-31 07:03:33.123 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 +2018-11-17 05:33:33.123 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 +2019-12-31 09:33:33.123 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 +2100-01-01 01:33:33.123 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 -0800 + + +-- !query +select col, date_format(col, 'O OOOO'), to_char(col, 'O OOOO'), to_varchar(col, 'O OOOO') from v +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'O' + + +-- !query +select col, date_format(col, 'x xx xxx xxxx xxxx xxxxx'), to_char(col, 'x xx xxx xxxx xxxx xxxxx'), to_varchar(col, 'x xx xxx xxxx xxxx xxxxx') from v +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal pattern character 'x' + + +-- !query +select col, date_format(col, '[yyyy-MM-dd HH:mm:ss]'), to_char(col, '[yyyy-MM-dd HH:mm:ss]'), to_varchar(col, '[yyyy-MM-dd HH:mm:ss]') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 [1582-05-31 19:40:35] [1582-05-31 19:40:35] [1582-05-31 19:40:35] +1969-12-31 15:00:00 [1969-12-31 15:00:00] [1969-12-31 15:00:00] [1969-12-31 15:00:00] +1970-12-31 04:59:59.999 [1970-12-31 04:59:59] [1970-12-31 04:59:59] [1970-12-31 04:59:59] +1996-03-31 07:03:33.123 [1996-03-31 07:03:33] [1996-03-31 07:03:33] [1996-03-31 07:03:33] +2018-11-17 05:33:33.123 [2018-11-17 05:33:33] [2018-11-17 05:33:33] [2018-11-17 05:33:33] +2019-12-31 09:33:33.123 [2019-12-31 09:33:33] [2019-12-31 09:33:33] [2019-12-31 09:33:33] +2100-01-01 01:33:33.123 [2100-01-01 01:33:33] [2100-01-01 01:33:33] [2100-01-01 01:33:33] + + +-- !query +select col, date_format(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'"), to_char(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'"), to_varchar(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'") from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV +1969-12-31 15:00:00 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV +1970-12-31 04:59:59.999 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV +1996-03-31 07:03:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV +2018-11-17 05:33:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV +2019-12-31 09:33:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV +2100-01-01 01:33:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV + + +-- !query +select col, date_format(col, "''"), to_char(col, "''"), to_varchar(col, "''") from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 ' ' ' +1969-12-31 15:00:00 ' ' ' +1970-12-31 04:59:59.999 ' ' ' +1996-03-31 07:03:33.123 ' ' ' +2018-11-17 05:33:33.123 ' ' ' +2019-12-31 09:33:33.123 ' ' ' +2100-01-01 01:33:33.123 ' ' ' + + +-- !query +select col, date_format(col, ''), to_char(col, ''), to_varchar(col, '') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 +1969-12-31 15:00:00 +1970-12-31 04:59:59.999 +1996-03-31 07:03:33.123 +2018-11-17 05:33:33.123 +2019-12-31 09:33:33.123 +2100-01-01 01:33:33.123 + + +-- !query +select date_format(date'2023-08-18', 'yyyy-MM-dd'), to_char(date'2023-08-18', 'yyyy-MM-dd'), to_varchar(date'2023-08-18', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +2023-08-18 2023-08-18 2023-08-18 + + +-- !query +select date_format(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'), to_char(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'), to_varchar(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ') +-- !query schema +struct +-- !query output +2023-08-18 02:13:14.000123-0700 2023-08-18 02:13:14.000123-0700 2023-08-18 02:13:14.000123-0700 + + +-- !query +select date_format(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS'), to_char(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS'), to_varchar(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS') +-- !query schema +struct +-- !query output +2023-08-18 09:13:14.000123 2023-08-18 09:13:14.000123 2023-08-18 09:13:14.000123 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-formatting.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-formatting.sql.out new file mode 100644 index 000000000000..f659dbf0083d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-formatting.sql.out @@ -0,0 +1,452 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view v as select col from values + (timestamp '1582-06-01 11:33:33.123UTC+080000'), + (timestamp '1970-01-01 00:00:00.000Europe/Paris'), + (timestamp '1970-12-31 23:59:59.999Asia/Srednekolymsk'), + (timestamp '1996-04-01 00:33:33.123Australia/Darwin'), + (timestamp '2018-11-17 13:33:33.123Z'), + (timestamp '2020-01-01 01:33:33.123Asia/Shanghai'), + (timestamp '2100-01-01 01:33:33.123America/Los_Angeles') t(col) +-- !query schema +struct<> +-- !query output + + + +-- !query +select col, date_format(col, 'G GG GGG GGGG'), to_char(col, 'G GG GGG GGGG'), to_varchar(col, 'G GG GGG GGGG') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini +1969-12-31 15:00:00 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini +1970-12-31 04:59:59.999 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini +1996-03-31 07:03:33.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini +2018-11-17 05:33:33.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini +2019-12-31 09:33:33.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini +2100-01-01 01:33:33.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini + + +-- !query +select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy'), to_char(col, 'y yy yyy yyyy yyyyy yyyyyy'), to_varchar(col, 'y yy yyy yyyy yyyyy yyyyyy') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 1582 82 1582 1582 01582 001582 1582 82 1582 1582 01582 001582 1582 82 1582 1582 01582 001582 +1969-12-31 15:00:00 1969 69 1969 1969 01969 001969 1969 69 1969 1969 01969 001969 1969 69 1969 1969 01969 001969 +1970-12-31 04:59:59.999 1970 70 1970 1970 01970 001970 1970 70 1970 1970 01970 001970 1970 70 1970 1970 01970 001970 +1996-03-31 07:03:33.123 1996 96 1996 1996 01996 001996 1996 96 1996 1996 01996 001996 1996 96 1996 1996 01996 001996 +2018-11-17 05:33:33.123 2018 18 2018 2018 02018 002018 2018 18 2018 2018 02018 002018 2018 18 2018 2018 02018 002018 +2019-12-31 09:33:33.123 2019 19 2019 2019 02019 002019 2019 19 2019 2019 02019 002019 2019 19 2019 2019 02019 002019 +2100-01-01 01:33:33.123 2100 00 2100 2100 02100 002100 2100 00 2100 2100 02100 002100 2100 00 2100 2100 02100 002100 + + +-- !query +select col, date_format(col, 'q qq'), to_char(col, 'q qq'), to_varchar(col, 'q qq') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 2 02 2 02 2 02 +1969-12-31 15:00:00 4 04 4 04 4 04 +1970-12-31 04:59:59.999 4 04 4 04 4 04 +1996-03-31 07:03:33.123 1 01 1 01 1 01 +2018-11-17 05:33:33.123 4 04 4 04 4 04 +2019-12-31 09:33:33.123 4 04 4 04 4 04 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'Q QQ QQQ QQQQ'), to_char(col, 'Q QQ QQQ QQQQ'), to_varchar(col, 'Q QQ QQQ QQQQ') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 2 02 Q2 2nd quarter 2 02 Q2 2nd quarter 2 02 Q2 2nd quarter +1969-12-31 15:00:00 4 04 Q4 4th quarter 4 04 Q4 4th quarter 4 04 Q4 4th quarter +1970-12-31 04:59:59.999 4 04 Q4 4th quarter 4 04 Q4 4th quarter 4 04 Q4 4th quarter +1996-03-31 07:03:33.123 1 01 Q1 1st quarter 1 01 Q1 1st quarter 1 01 Q1 1st quarter +2018-11-17 05:33:33.123 4 04 Q4 4th quarter 4 04 Q4 4th quarter 4 04 Q4 4th quarter +2019-12-31 09:33:33.123 4 04 Q4 4th quarter 4 04 Q4 4th quarter 4 04 Q4 4th quarter +2100-01-01 01:33:33.123 1 01 Q1 1st quarter 1 01 Q1 1st quarter 1 01 Q1 1st quarter + + +-- !query +select col, date_format(col, 'M MM MMM MMMM'), to_char(col, 'M MM MMM MMMM'), to_varchar(col, 'M MM MMM MMMM') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 5 05 May May 5 05 May May 5 05 May May +1969-12-31 15:00:00 12 12 Dec December 12 12 Dec December 12 12 Dec December +1970-12-31 04:59:59.999 12 12 Dec December 12 12 Dec December 12 12 Dec December +1996-03-31 07:03:33.123 3 03 Mar March 3 03 Mar March 3 03 Mar March +2018-11-17 05:33:33.123 11 11 Nov November 11 11 Nov November 11 11 Nov November +2019-12-31 09:33:33.123 12 12 Dec December 12 12 Dec December 12 12 Dec December +2100-01-01 01:33:33.123 1 01 Jan January 1 01 Jan January 1 01 Jan January + + +-- !query +select col, date_format(col, 'L LL'), to_char(col, 'L LL'), to_varchar(col, 'L LL') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 5 05 5 05 5 05 +1969-12-31 15:00:00 12 12 12 12 12 12 +1970-12-31 04:59:59.999 12 12 12 12 12 12 +1996-03-31 07:03:33.123 3 03 3 03 3 03 +2018-11-17 05:33:33.123 11 11 11 11 11 11 +2019-12-31 09:33:33.123 12 12 12 12 12 12 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'E EE EEE EEEE'), to_char(col, 'E EE EEE EEEE'), to_varchar(col, 'E EE EEE EEEE') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 Mon Mon Mon Monday Mon Mon Mon Monday Mon Mon Mon Monday +1969-12-31 15:00:00 Wed Wed Wed Wednesday Wed Wed Wed Wednesday Wed Wed Wed Wednesday +1970-12-31 04:59:59.999 Thu Thu Thu Thursday Thu Thu Thu Thursday Thu Thu Thu Thursday +1996-03-31 07:03:33.123 Sun Sun Sun Sunday Sun Sun Sun Sunday Sun Sun Sun Sunday +2018-11-17 05:33:33.123 Sat Sat Sat Saturday Sat Sat Sat Saturday Sat Sat Sat Saturday +2019-12-31 09:33:33.123 Tue Tue Tue Tuesday Tue Tue Tue Tuesday Tue Tue Tue Tuesday +2100-01-01 01:33:33.123 Fri Fri Fri Friday Fri Fri Fri Friday Fri Fri Fri Friday + + +-- !query +select col, date_format(col, 'F'), to_char(col, 'F'), to_varchar(col, 'F') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 3 3 3 +1969-12-31 15:00:00 3 3 3 +1970-12-31 04:59:59.999 3 3 3 +1996-03-31 07:03:33.123 3 3 3 +2018-11-17 05:33:33.123 3 3 3 +2019-12-31 09:33:33.123 3 3 3 +2100-01-01 01:33:33.123 1 1 1 + + +-- !query +select col, date_format(col, 'd dd'), to_char(col, 'd dd'), to_varchar(col, 'd dd') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 31 31 31 31 31 31 +1969-12-31 15:00:00 31 31 31 31 31 31 +1970-12-31 04:59:59.999 31 31 31 31 31 31 +1996-03-31 07:03:33.123 31 31 31 31 31 31 +2018-11-17 05:33:33.123 17 17 17 17 17 17 +2019-12-31 09:33:33.123 31 31 31 31 31 31 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'DD'), to_char(col, 'DD'), to_varchar(col, 'DD') from v where col = timestamp '2100-01-01 01:33:33.123America/Los_Angeles' +-- !query schema +struct +-- !query output +2100-01-01 01:33:33.123 01 01 01 + + +-- !query +select col, date_format(col, 'D DDD'), to_char(col, 'D DDD'), to_varchar(col, 'D DDD') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 151 151 151 151 151 151 +1969-12-31 15:00:00 365 365 365 365 365 365 +1970-12-31 04:59:59.999 365 365 365 365 365 365 +1996-03-31 07:03:33.123 91 091 91 091 91 091 +2018-11-17 05:33:33.123 321 321 321 321 321 321 +2019-12-31 09:33:33.123 365 365 365 365 365 365 +2100-01-01 01:33:33.123 1 001 1 001 1 001 + + +-- !query +select col, date_format(col, 'H HH'), to_char(col, 'H HH'), to_varchar(col, 'H HH') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 19 19 19 19 19 19 +1969-12-31 15:00:00 15 15 15 15 15 15 +1970-12-31 04:59:59.999 4 04 4 04 4 04 +1996-03-31 07:03:33.123 7 07 7 07 7 07 +2018-11-17 05:33:33.123 5 05 5 05 5 05 +2019-12-31 09:33:33.123 9 09 9 09 9 09 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'h hh'), to_char(col, 'h hh'), to_varchar(col, 'h hh') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 7 07 7 07 7 07 +1969-12-31 15:00:00 3 03 3 03 3 03 +1970-12-31 04:59:59.999 4 04 4 04 4 04 +1996-03-31 07:03:33.123 7 07 7 07 7 07 +2018-11-17 05:33:33.123 5 05 5 05 5 05 +2019-12-31 09:33:33.123 9 09 9 09 9 09 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'k kk'), to_char(col, 'k kk'), to_varchar(col, 'k kk') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 19 19 19 19 19 19 +1969-12-31 15:00:00 15 15 15 15 15 15 +1970-12-31 04:59:59.999 4 04 4 04 4 04 +1996-03-31 07:03:33.123 7 07 7 07 7 07 +2018-11-17 05:33:33.123 5 05 5 05 5 05 +2019-12-31 09:33:33.123 9 09 9 09 9 09 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'K KK'), to_char(col, 'K KK'), to_varchar(col, 'K KK') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 7 07 7 07 7 07 +1969-12-31 15:00:00 3 03 3 03 3 03 +1970-12-31 04:59:59.999 4 04 4 04 4 04 +1996-03-31 07:03:33.123 7 07 7 07 7 07 +2018-11-17 05:33:33.123 5 05 5 05 5 05 +2019-12-31 09:33:33.123 9 09 9 09 9 09 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'm mm'), to_char(col, 'm mm'), to_varchar(col, 'm mm') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 40 40 40 40 40 40 +1969-12-31 15:00:00 0 00 0 00 0 00 +1970-12-31 04:59:59.999 59 59 59 59 59 59 +1996-03-31 07:03:33.123 3 03 3 03 3 03 +2018-11-17 05:33:33.123 33 33 33 33 33 33 +2019-12-31 09:33:33.123 33 33 33 33 33 33 +2100-01-01 01:33:33.123 33 33 33 33 33 33 + + +-- !query +select col, date_format(col, 's ss'), to_char(col, 's ss'), to_varchar(col, 's ss') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 35 35 35 35 35 35 +1969-12-31 15:00:00 0 00 0 00 0 00 +1970-12-31 04:59:59.999 59 59 59 59 59 59 +1996-03-31 07:03:33.123 33 33 33 33 33 33 +2018-11-17 05:33:33.123 33 33 33 33 33 33 +2019-12-31 09:33:33.123 33 33 33 33 33 33 +2100-01-01 01:33:33.123 33 33 33 33 33 33 + + +-- !query +select col, date_format(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS'), to_char(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS'), to_varchar(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 +1969-12-31 15:00:00 0 00 000 0000 00000 000000 0000000 00000000 000000000 0 00 000 0000 00000 000000 0000000 00000000 000000000 0 00 000 0000 00000 000000 0000000 00000000 000000000 +1970-12-31 04:59:59.999 9 99 999 9990 99900 999000 9990000 99900000 999000000 9 99 999 9990 99900 999000 9990000 99900000 999000000 9 99 999 9990 99900 999000 9990000 99900000 999000000 +1996-03-31 07:03:33.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 +2018-11-17 05:33:33.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 +2019-12-31 09:33:33.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 +2100-01-01 01:33:33.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 + + +-- !query +select col, date_format(col, 'a'), to_char(col, 'a'), to_varchar(col, 'a') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 PM PM PM +1969-12-31 15:00:00 PM PM PM +1970-12-31 04:59:59.999 AM AM AM +1996-03-31 07:03:33.123 AM AM AM +2018-11-17 05:33:33.123 AM AM AM +2019-12-31 09:33:33.123 AM AM AM +2100-01-01 01:33:33.123 AM AM AM + + +-- !query +select col, date_format(col, 'VV'), to_char(col, 'VV'), to_varchar(col, 'VV') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles +1969-12-31 15:00:00 America/Los_Angeles America/Los_Angeles America/Los_Angeles +1970-12-31 04:59:59.999 America/Los_Angeles America/Los_Angeles America/Los_Angeles +1996-03-31 07:03:33.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles +2018-11-17 05:33:33.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles +2019-12-31 09:33:33.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles +2100-01-01 01:33:33.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles + + +-- !query +select col, date_format(col, 'z zz zzz zzzz'), to_char(col, 'z zz zzz zzzz'), to_varchar(col, 'z zz zzz zzzz') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time +1969-12-31 15:00:00 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time +1970-12-31 04:59:59.999 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time +1996-03-31 07:03:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time +2018-11-17 05:33:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time +2019-12-31 09:33:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time +2100-01-01 01:33:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time + + +-- !query +select col, date_format(col, 'X XX XXX'), to_char(col, 'X XX XXX'), to_varchar(col, 'X XX XXX') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 -0752 -0752 -07:52 -0752 -0752 -07:52 -0752 -0752 -07:52 +1969-12-31 15:00:00 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 +1970-12-31 04:59:59.999 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 +1996-03-31 07:03:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 +2018-11-17 05:33:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 +2019-12-31 09:33:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 +2100-01-01 01:33:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 + + +-- !query +select col, date_format(col, 'XXXX XXXXX'), to_char(col, 'XXXX XXXXX'), to_varchar(col, 'XXXX XXXXX') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 -075258 -07:52:58 -075258 -07:52:58 -075258 -07:52:58 +1969-12-31 15:00:00 -0800 -08:00 -0800 -08:00 -0800 -08:00 +1970-12-31 04:59:59.999 -0800 -08:00 -0800 -08:00 -0800 -08:00 +1996-03-31 07:03:33.123 -0800 -08:00 -0800 -08:00 -0800 -08:00 +2018-11-17 05:33:33.123 -0800 -08:00 -0800 -08:00 -0800 -08:00 +2019-12-31 09:33:33.123 -0800 -08:00 -0800 -08:00 -0800 -08:00 +2100-01-01 01:33:33.123 -0800 -08:00 -0800 -08:00 -0800 -08:00 + + +-- !query +select col, date_format(col, 'Z ZZ ZZZ ZZZZ ZZZZZ'), to_char(col, 'Z ZZ ZZZ ZZZZ ZZZZZ'), to_varchar(col, 'Z ZZ ZZZ ZZZZ ZZZZZ') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 -0752 -0752 -0752 GMT-07:52:58 -07:52:58 -0752 -0752 -0752 GMT-07:52:58 -07:52:58 -0752 -0752 -0752 GMT-07:52:58 -07:52:58 +1969-12-31 15:00:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 +1970-12-31 04:59:59.999 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 +1996-03-31 07:03:33.123 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 +2018-11-17 05:33:33.123 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 +2019-12-31 09:33:33.123 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 +2100-01-01 01:33:33.123 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 + + +-- !query +select col, date_format(col, 'O OOOO'), to_char(col, 'O OOOO'), to_varchar(col, 'O OOOO') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 GMT-7:52:58 GMT-07:52:58 GMT-7:52:58 GMT-07:52:58 GMT-7:52:58 GMT-07:52:58 +1969-12-31 15:00:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00 +1970-12-31 04:59:59.999 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00 +1996-03-31 07:03:33.123 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00 +2018-11-17 05:33:33.123 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00 +2019-12-31 09:33:33.123 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00 +2100-01-01 01:33:33.123 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00 + + +-- !query +select col, date_format(col, 'x xx xxx xxxx xxxx xxxxx'), to_char(col, 'x xx xxx xxxx xxxx xxxxx'), to_varchar(col, 'x xx xxx xxxx xxxx xxxxx') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 -0752 -0752 -07:52 -075258 -075258 -07:52:58 -0752 -0752 -07:52 -075258 -075258 -07:52:58 -0752 -0752 -07:52 -075258 -075258 -07:52:58 +1969-12-31 15:00:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 +1970-12-31 04:59:59.999 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 +1996-03-31 07:03:33.123 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 +2018-11-17 05:33:33.123 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 +2019-12-31 09:33:33.123 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 +2100-01-01 01:33:33.123 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 + + +-- !query +select col, date_format(col, '[yyyy-MM-dd HH:mm:ss]'), to_char(col, '[yyyy-MM-dd HH:mm:ss]'), to_varchar(col, '[yyyy-MM-dd HH:mm:ss]') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 1582-05-31 19:40:35 1582-05-31 19:40:35 1582-05-31 19:40:35 +1969-12-31 15:00:00 1969-12-31 15:00:00 1969-12-31 15:00:00 1969-12-31 15:00:00 +1970-12-31 04:59:59.999 1970-12-31 04:59:59 1970-12-31 04:59:59 1970-12-31 04:59:59 +1996-03-31 07:03:33.123 1996-03-31 07:03:33 1996-03-31 07:03:33 1996-03-31 07:03:33 +2018-11-17 05:33:33.123 2018-11-17 05:33:33 2018-11-17 05:33:33 2018-11-17 05:33:33 +2019-12-31 09:33:33.123 2019-12-31 09:33:33 2019-12-31 09:33:33 2019-12-31 09:33:33 +2100-01-01 01:33:33.123 2100-01-01 01:33:33 2100-01-01 01:33:33 2100-01-01 01:33:33 + + +-- !query +select col, date_format(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'"), to_char(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'"), to_varchar(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'") from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV +1969-12-31 15:00:00 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV +1970-12-31 04:59:59.999 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV +1996-03-31 07:03:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV +2018-11-17 05:33:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV +2019-12-31 09:33:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV +2100-01-01 01:33:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV + + +-- !query +select col, date_format(col, "''"), to_char(col, "''"), to_varchar(col, "''") from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 ' ' ' +1969-12-31 15:00:00 ' ' ' +1970-12-31 04:59:59.999 ' ' ' +1996-03-31 07:03:33.123 ' ' ' +2018-11-17 05:33:33.123 ' ' ' +2019-12-31 09:33:33.123 ' ' ' +2100-01-01 01:33:33.123 ' ' ' + + +-- !query +select col, date_format(col, ''), to_char(col, ''), to_varchar(col, '') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 +1969-12-31 15:00:00 +1970-12-31 04:59:59.999 +1996-03-31 07:03:33.123 +2018-11-17 05:33:33.123 +2019-12-31 09:33:33.123 +2100-01-01 01:33:33.123 + + +-- !query +select date_format(date'2023-08-18', 'yyyy-MM-dd'), to_char(date'2023-08-18', 'yyyy-MM-dd'), to_varchar(date'2023-08-18', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +2023-08-18 2023-08-18 2023-08-18 + + +-- !query +select date_format(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'), to_char(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'), to_varchar(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ') +-- !query schema +struct +-- !query output +2023-08-18 02:13:14.123456-0700 2023-08-18 02:13:14.123456-0700 2023-08-18 02:13:14.123456-0700 + + +-- !query +select date_format(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS'), to_char(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS'), to_varchar(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS') +-- !query schema +struct +-- !query output +2023-08-18 09:13:14.123456 2023-08-18 09:13:14.123456 2023-08-18 09:13:14.123456 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-formatting.sql.out.java21 b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-formatting.sql.out.java21 new file mode 100644 index 000000000000..3ed7e08c88bd --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-formatting.sql.out.java21 @@ -0,0 +1,452 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view v as select col from values + (timestamp '1582-06-01 11:33:33.123UTC+080000'), + (timestamp '1970-01-01 00:00:00.000Europe/Paris'), + (timestamp '1970-12-31 23:59:59.999Asia/Srednekolymsk'), + (timestamp '1996-04-01 00:33:33.123Australia/Darwin'), + (timestamp '2018-11-17 13:33:33.123Z'), + (timestamp '2020-01-01 01:33:33.123Asia/Shanghai'), + (timestamp '2100-01-01 01:33:33.123America/Los_Angeles') t(col) +-- !query schema +struct<> +-- !query output + + + +-- !query +select col, date_format(col, 'G GG GGG GGGG'), to_char(col, 'G GG GGG GGGG'), to_varchar(col, 'G GG GGG GGGG') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini +1969-12-31 15:00:00 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini +1970-12-31 04:59:59.999 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini +1996-03-31 07:03:33.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini +2018-11-17 05:33:33.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini +2019-12-31 09:33:33.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini +2100-01-01 01:33:33.123 AD AD AD Anno Domini AD AD AD Anno Domini AD AD AD Anno Domini + + +-- !query +select col, date_format(col, 'y yy yyy yyyy yyyyy yyyyyy'), to_char(col, 'y yy yyy yyyy yyyyy yyyyyy'), to_varchar(col, 'y yy yyy yyyy yyyyy yyyyyy') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 1582 82 1582 1582 01582 001582 1582 82 1582 1582 01582 001582 1582 82 1582 1582 01582 001582 +1969-12-31 15:00:00 1969 69 1969 1969 01969 001969 1969 69 1969 1969 01969 001969 1969 69 1969 1969 01969 001969 +1970-12-31 04:59:59.999 1970 70 1970 1970 01970 001970 1970 70 1970 1970 01970 001970 1970 70 1970 1970 01970 001970 +1996-03-31 07:03:33.123 1996 96 1996 1996 01996 001996 1996 96 1996 1996 01996 001996 1996 96 1996 1996 01996 001996 +2018-11-17 05:33:33.123 2018 18 2018 2018 02018 002018 2018 18 2018 2018 02018 002018 2018 18 2018 2018 02018 002018 +2019-12-31 09:33:33.123 2019 19 2019 2019 02019 002019 2019 19 2019 2019 02019 002019 2019 19 2019 2019 02019 002019 +2100-01-01 01:33:33.123 2100 00 2100 2100 02100 002100 2100 00 2100 2100 02100 002100 2100 00 2100 2100 02100 002100 + + +-- !query +select col, date_format(col, 'q qq'), to_char(col, 'q qq'), to_varchar(col, 'q qq') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 2 02 2 02 2 02 +1969-12-31 15:00:00 4 04 4 04 4 04 +1970-12-31 04:59:59.999 4 04 4 04 4 04 +1996-03-31 07:03:33.123 1 01 1 01 1 01 +2018-11-17 05:33:33.123 4 04 4 04 4 04 +2019-12-31 09:33:33.123 4 04 4 04 4 04 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'Q QQ QQQ QQQQ'), to_char(col, 'Q QQ QQQ QQQQ'), to_varchar(col, 'Q QQ QQQ QQQQ') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 2 02 Q2 2nd quarter 2 02 Q2 2nd quarter 2 02 Q2 2nd quarter +1969-12-31 15:00:00 4 04 Q4 4th quarter 4 04 Q4 4th quarter 4 04 Q4 4th quarter +1970-12-31 04:59:59.999 4 04 Q4 4th quarter 4 04 Q4 4th quarter 4 04 Q4 4th quarter +1996-03-31 07:03:33.123 1 01 Q1 1st quarter 1 01 Q1 1st quarter 1 01 Q1 1st quarter +2018-11-17 05:33:33.123 4 04 Q4 4th quarter 4 04 Q4 4th quarter 4 04 Q4 4th quarter +2019-12-31 09:33:33.123 4 04 Q4 4th quarter 4 04 Q4 4th quarter 4 04 Q4 4th quarter +2100-01-01 01:33:33.123 1 01 Q1 1st quarter 1 01 Q1 1st quarter 1 01 Q1 1st quarter + + +-- !query +select col, date_format(col, 'M MM MMM MMMM'), to_char(col, 'M MM MMM MMMM'), to_varchar(col, 'M MM MMM MMMM') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 5 05 May May 5 05 May May 5 05 May May +1969-12-31 15:00:00 12 12 Dec December 12 12 Dec December 12 12 Dec December +1970-12-31 04:59:59.999 12 12 Dec December 12 12 Dec December 12 12 Dec December +1996-03-31 07:03:33.123 3 03 Mar March 3 03 Mar March 3 03 Mar March +2018-11-17 05:33:33.123 11 11 Nov November 11 11 Nov November 11 11 Nov November +2019-12-31 09:33:33.123 12 12 Dec December 12 12 Dec December 12 12 Dec December +2100-01-01 01:33:33.123 1 01 Jan January 1 01 Jan January 1 01 Jan January + + +-- !query +select col, date_format(col, 'L LL'), to_char(col, 'L LL'), to_varchar(col, 'L LL') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 5 05 5 05 5 05 +1969-12-31 15:00:00 12 12 12 12 12 12 +1970-12-31 04:59:59.999 12 12 12 12 12 12 +1996-03-31 07:03:33.123 3 03 3 03 3 03 +2018-11-17 05:33:33.123 11 11 11 11 11 11 +2019-12-31 09:33:33.123 12 12 12 12 12 12 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'E EE EEE EEEE'), to_char(col, 'E EE EEE EEEE'), to_varchar(col, 'E EE EEE EEEE') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 Mon Mon Mon Monday Mon Mon Mon Monday Mon Mon Mon Monday +1969-12-31 15:00:00 Wed Wed Wed Wednesday Wed Wed Wed Wednesday Wed Wed Wed Wednesday +1970-12-31 04:59:59.999 Thu Thu Thu Thursday Thu Thu Thu Thursday Thu Thu Thu Thursday +1996-03-31 07:03:33.123 Sun Sun Sun Sunday Sun Sun Sun Sunday Sun Sun Sun Sunday +2018-11-17 05:33:33.123 Sat Sat Sat Saturday Sat Sat Sat Saturday Sat Sat Sat Saturday +2019-12-31 09:33:33.123 Tue Tue Tue Tuesday Tue Tue Tue Tuesday Tue Tue Tue Tuesday +2100-01-01 01:33:33.123 Fri Fri Fri Friday Fri Fri Fri Friday Fri Fri Fri Friday + + +-- !query +select col, date_format(col, 'F'), to_char(col, 'F'), to_varchar(col, 'F') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 5 5 5 +1969-12-31 15:00:00 5 5 5 +1970-12-31 04:59:59.999 5 5 5 +1996-03-31 07:03:33.123 5 5 5 +2018-11-17 05:33:33.123 3 3 3 +2019-12-31 09:33:33.123 5 5 5 +2100-01-01 01:33:33.123 1 1 1 + + +-- !query +select col, date_format(col, 'd dd'), to_char(col, 'd dd'), to_varchar(col, 'd dd') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 31 31 31 31 31 31 +1969-12-31 15:00:00 31 31 31 31 31 31 +1970-12-31 04:59:59.999 31 31 31 31 31 31 +1996-03-31 07:03:33.123 31 31 31 31 31 31 +2018-11-17 05:33:33.123 17 17 17 17 17 17 +2019-12-31 09:33:33.123 31 31 31 31 31 31 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'DD'), to_char(col, 'DD'), to_varchar(col, 'DD') from v where col = timestamp '2100-01-01 01:33:33.123America/Los_Angeles' +-- !query schema +struct +-- !query output +2100-01-01 01:33:33.123 01 01 01 + + +-- !query +select col, date_format(col, 'D DDD'), to_char(col, 'D DDD'), to_varchar(col, 'D DDD') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 151 151 151 151 151 151 +1969-12-31 15:00:00 365 365 365 365 365 365 +1970-12-31 04:59:59.999 365 365 365 365 365 365 +1996-03-31 07:03:33.123 91 091 91 091 91 091 +2018-11-17 05:33:33.123 321 321 321 321 321 321 +2019-12-31 09:33:33.123 365 365 365 365 365 365 +2100-01-01 01:33:33.123 1 001 1 001 1 001 + + +-- !query +select col, date_format(col, 'H HH'), to_char(col, 'H HH'), to_varchar(col, 'H HH') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 19 19 19 19 19 19 +1969-12-31 15:00:00 15 15 15 15 15 15 +1970-12-31 04:59:59.999 4 04 4 04 4 04 +1996-03-31 07:03:33.123 7 07 7 07 7 07 +2018-11-17 05:33:33.123 5 05 5 05 5 05 +2019-12-31 09:33:33.123 9 09 9 09 9 09 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'h hh'), to_char(col, 'h hh'), to_varchar(col, 'h hh') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 7 07 7 07 7 07 +1969-12-31 15:00:00 3 03 3 03 3 03 +1970-12-31 04:59:59.999 4 04 4 04 4 04 +1996-03-31 07:03:33.123 7 07 7 07 7 07 +2018-11-17 05:33:33.123 5 05 5 05 5 05 +2019-12-31 09:33:33.123 9 09 9 09 9 09 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'k kk'), to_char(col, 'k kk'), to_varchar(col, 'k kk') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 19 19 19 19 19 19 +1969-12-31 15:00:00 15 15 15 15 15 15 +1970-12-31 04:59:59.999 4 04 4 04 4 04 +1996-03-31 07:03:33.123 7 07 7 07 7 07 +2018-11-17 05:33:33.123 5 05 5 05 5 05 +2019-12-31 09:33:33.123 9 09 9 09 9 09 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'K KK'), to_char(col, 'K KK'), to_varchar(col, 'K KK') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 7 07 7 07 7 07 +1969-12-31 15:00:00 3 03 3 03 3 03 +1970-12-31 04:59:59.999 4 04 4 04 4 04 +1996-03-31 07:03:33.123 7 07 7 07 7 07 +2018-11-17 05:33:33.123 5 05 5 05 5 05 +2019-12-31 09:33:33.123 9 09 9 09 9 09 +2100-01-01 01:33:33.123 1 01 1 01 1 01 + + +-- !query +select col, date_format(col, 'm mm'), to_char(col, 'm mm'), to_varchar(col, 'm mm') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 40 40 40 40 40 40 +1969-12-31 15:00:00 0 00 0 00 0 00 +1970-12-31 04:59:59.999 59 59 59 59 59 59 +1996-03-31 07:03:33.123 3 03 3 03 3 03 +2018-11-17 05:33:33.123 33 33 33 33 33 33 +2019-12-31 09:33:33.123 33 33 33 33 33 33 +2100-01-01 01:33:33.123 33 33 33 33 33 33 + + +-- !query +select col, date_format(col, 's ss'), to_char(col, 's ss'), to_varchar(col, 's ss') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 35 35 35 35 35 35 +1969-12-31 15:00:00 0 00 0 00 0 00 +1970-12-31 04:59:59.999 59 59 59 59 59 59 +1996-03-31 07:03:33.123 33 33 33 33 33 33 +2018-11-17 05:33:33.123 33 33 33 33 33 33 +2019-12-31 09:33:33.123 33 33 33 33 33 33 +2100-01-01 01:33:33.123 33 33 33 33 33 33 + + +-- !query +select col, date_format(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS'), to_char(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS'), to_varchar(col, 'S SS SSS SSSS SSSSS SSSSSS SSSSSSS SSSSSSSS SSSSSSSSS') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 +1969-12-31 15:00:00 0 00 000 0000 00000 000000 0000000 00000000 000000000 0 00 000 0000 00000 000000 0000000 00000000 000000000 0 00 000 0000 00000 000000 0000000 00000000 000000000 +1970-12-31 04:59:59.999 9 99 999 9990 99900 999000 9990000 99900000 999000000 9 99 999 9990 99900 999000 9990000 99900000 999000000 9 99 999 9990 99900 999000 9990000 99900000 999000000 +1996-03-31 07:03:33.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 +2018-11-17 05:33:33.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 +2019-12-31 09:33:33.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 +2100-01-01 01:33:33.123 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 1 12 123 1230 12300 123000 1230000 12300000 123000000 + + +-- !query +select col, date_format(col, 'a'), to_char(col, 'a'), to_varchar(col, 'a') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 PM PM PM +1969-12-31 15:00:00 PM PM PM +1970-12-31 04:59:59.999 AM AM AM +1996-03-31 07:03:33.123 AM AM AM +2018-11-17 05:33:33.123 AM AM AM +2019-12-31 09:33:33.123 AM AM AM +2100-01-01 01:33:33.123 AM AM AM + + +-- !query +select col, date_format(col, 'VV'), to_char(col, 'VV'), to_varchar(col, 'VV') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles +1969-12-31 15:00:00 America/Los_Angeles America/Los_Angeles America/Los_Angeles +1970-12-31 04:59:59.999 America/Los_Angeles America/Los_Angeles America/Los_Angeles +1996-03-31 07:03:33.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles +2018-11-17 05:33:33.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles +2019-12-31 09:33:33.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles +2100-01-01 01:33:33.123 America/Los_Angeles America/Los_Angeles America/Los_Angeles + + +-- !query +select col, date_format(col, 'z zz zzz zzzz'), to_char(col, 'z zz zzz zzzz'), to_varchar(col, 'z zz zzz zzzz') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time +1969-12-31 15:00:00 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time +1970-12-31 04:59:59.999 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time +1996-03-31 07:03:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time +2018-11-17 05:33:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time +2019-12-31 09:33:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time +2100-01-01 01:33:33.123 PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time PST PST PST Pacific Standard Time + + +-- !query +select col, date_format(col, 'X XX XXX'), to_char(col, 'X XX XXX'), to_varchar(col, 'X XX XXX') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 -0752 -0752 -07:52 -0752 -0752 -07:52 -0752 -0752 -07:52 +1969-12-31 15:00:00 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 +1970-12-31 04:59:59.999 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 +1996-03-31 07:03:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 +2018-11-17 05:33:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 +2019-12-31 09:33:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 +2100-01-01 01:33:33.123 -08 -0800 -08:00 -08 -0800 -08:00 -08 -0800 -08:00 + + +-- !query +select col, date_format(col, 'XXXX XXXXX'), to_char(col, 'XXXX XXXXX'), to_varchar(col, 'XXXX XXXXX') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 -075258 -07:52:58 -075258 -07:52:58 -075258 -07:52:58 +1969-12-31 15:00:00 -0800 -08:00 -0800 -08:00 -0800 -08:00 +1970-12-31 04:59:59.999 -0800 -08:00 -0800 -08:00 -0800 -08:00 +1996-03-31 07:03:33.123 -0800 -08:00 -0800 -08:00 -0800 -08:00 +2018-11-17 05:33:33.123 -0800 -08:00 -0800 -08:00 -0800 -08:00 +2019-12-31 09:33:33.123 -0800 -08:00 -0800 -08:00 -0800 -08:00 +2100-01-01 01:33:33.123 -0800 -08:00 -0800 -08:00 -0800 -08:00 + + +-- !query +select col, date_format(col, 'Z ZZ ZZZ ZZZZ ZZZZZ'), to_char(col, 'Z ZZ ZZZ ZZZZ ZZZZZ'), to_varchar(col, 'Z ZZ ZZZ ZZZZ ZZZZZ') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 -0752 -0752 -0752 GMT-07:52:58 -07:52:58 -0752 -0752 -0752 GMT-07:52:58 -07:52:58 -0752 -0752 -0752 GMT-07:52:58 -07:52:58 +1969-12-31 15:00:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 +1970-12-31 04:59:59.999 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 +1996-03-31 07:03:33.123 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 +2018-11-17 05:33:33.123 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 +2019-12-31 09:33:33.123 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 +2100-01-01 01:33:33.123 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 -0800 -0800 -0800 GMT-08:00 -08:00 + + +-- !query +select col, date_format(col, 'O OOOO'), to_char(col, 'O OOOO'), to_varchar(col, 'O OOOO') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 GMT-7:52:58 GMT-07:52:58 GMT-7:52:58 GMT-07:52:58 GMT-7:52:58 GMT-07:52:58 +1969-12-31 15:00:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00 +1970-12-31 04:59:59.999 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00 +1996-03-31 07:03:33.123 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00 +2018-11-17 05:33:33.123 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00 +2019-12-31 09:33:33.123 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00 +2100-01-01 01:33:33.123 GMT-8 GMT-08:00 GMT-8 GMT-08:00 GMT-8 GMT-08:00 + + +-- !query +select col, date_format(col, 'x xx xxx xxxx xxxx xxxxx'), to_char(col, 'x xx xxx xxxx xxxx xxxxx'), to_varchar(col, 'x xx xxx xxxx xxxx xxxxx') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 -0752 -0752 -07:52 -075258 -075258 -07:52:58 -0752 -0752 -07:52 -075258 -075258 -07:52:58 -0752 -0752 -07:52 -075258 -075258 -07:52:58 +1969-12-31 15:00:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 +1970-12-31 04:59:59.999 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 +1996-03-31 07:03:33.123 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 +2018-11-17 05:33:33.123 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 +2019-12-31 09:33:33.123 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 +2100-01-01 01:33:33.123 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 -08 -0800 -08:00 -0800 -0800 -08:00 + + +-- !query +select col, date_format(col, '[yyyy-MM-dd HH:mm:ss]'), to_char(col, '[yyyy-MM-dd HH:mm:ss]'), to_varchar(col, '[yyyy-MM-dd HH:mm:ss]') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 1582-05-31 19:40:35 1582-05-31 19:40:35 1582-05-31 19:40:35 +1969-12-31 15:00:00 1969-12-31 15:00:00 1969-12-31 15:00:00 1969-12-31 15:00:00 +1970-12-31 04:59:59.999 1970-12-31 04:59:59 1970-12-31 04:59:59 1970-12-31 04:59:59 +1996-03-31 07:03:33.123 1996-03-31 07:03:33 1996-03-31 07:03:33 1996-03-31 07:03:33 +2018-11-17 05:33:33.123 2018-11-17 05:33:33 2018-11-17 05:33:33 2018-11-17 05:33:33 +2019-12-31 09:33:33.123 2019-12-31 09:33:33 2019-12-31 09:33:33 2019-12-31 09:33:33 +2100-01-01 01:33:33.123 2100-01-01 01:33:33 2100-01-01 01:33:33 2100-01-01 01:33:33 + + +-- !query +select col, date_format(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'"), to_char(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'"), to_varchar(col, "姚123'GyYqQMLwWuEFDdhHmsSaVzZxXOV'") from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV +1969-12-31 15:00:00 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV +1970-12-31 04:59:59.999 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV +1996-03-31 07:03:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV +2018-11-17 05:33:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV +2019-12-31 09:33:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV +2100-01-01 01:33:33.123 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV 姚123GyYqQMLwWuEFDdhHmsSaVzZxXOV + + +-- !query +select col, date_format(col, "''"), to_char(col, "''"), to_varchar(col, "''") from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 ' ' ' +1969-12-31 15:00:00 ' ' ' +1970-12-31 04:59:59.999 ' ' ' +1996-03-31 07:03:33.123 ' ' ' +2018-11-17 05:33:33.123 ' ' ' +2019-12-31 09:33:33.123 ' ' ' +2100-01-01 01:33:33.123 ' ' ' + + +-- !query +select col, date_format(col, ''), to_char(col, ''), to_varchar(col, '') from v +-- !query schema +struct +-- !query output +1582-05-31 19:40:35.123 +1969-12-31 15:00:00 +1970-12-31 04:59:59.999 +1996-03-31 07:03:33.123 +2018-11-17 05:33:33.123 +2019-12-31 09:33:33.123 +2100-01-01 01:33:33.123 + + +-- !query +select date_format(date'2023-08-18', 'yyyy-MM-dd'), to_char(date'2023-08-18', 'yyyy-MM-dd'), to_varchar(date'2023-08-18', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +2023-08-18 2023-08-18 2023-08-18 + + +-- !query +select date_format(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'), to_char(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ'), to_varchar(timestamp_ltz'2023-08-18 09:13:14.123456Z', 'yyyy-MM-dd HH:mm:ss.SSSSSSZ') +-- !query schema +struct +-- !query output +2023-08-18 02:13:14.123456-0700 2023-08-18 02:13:14.123456-0700 2023-08-18 02:13:14.123456-0700 + + +-- !query +select date_format(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS'), to_char(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS'), to_varchar(timestamp_ntz'2023-08-18 09:13:14.123456', 'yyyy-MM-dd HH:mm:ss.SSSSSS') +-- !query schema +struct +-- !query output +2023-08-18 09:13:14.123456 2023-08-18 09:13:14.123456 2023-08-18 09:13:14.123456 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-legacy.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-legacy.sql.out new file mode 100644 index 000000000000..17ec4e584058 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-legacy.sql.out @@ -0,0 +1,2480 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view date_view as select '2011-11-11' date_str, '1' int_str +-- !query schema +struct<> +-- !query output + + + +-- !query +select date '2019-01-01\t' +-- !query schema +struct +-- !query output +2019-01-01 + + +-- !query +select date '2020-01-01中文' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2020-01-01中文'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "date '2020-01-01中文'" + } ] +} + + +-- !query +select make_date(2019, 1, 1), make_date(12, 12, 12) +-- !query schema +struct +-- !query output +2019-01-01 0012-12-12 + + +-- !query +select make_date(2000, 13, 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for MonthOfYear (valid values 1 - 12): 13" + } +} + + +-- !query +select make_date(2000, 1, 33) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for DayOfMonth (valid values 1 - 28/31): 33" + } +} + + +-- !query +select date'015' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'015'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "date'015'" + } ] +} + + +-- !query +select date'2021-4294967297-11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2021-4294967297-11'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "date'2021-4294967297-11'" + } ] +} + + +-- !query +select current_date = current_date +-- !query schema +struct<(current_date() = current_date()):boolean> +-- !query output +true + + +-- !query +select current_date() = current_date() +-- !query schema +struct<(current_date() = current_date()):boolean> +-- !query output +true + + +-- !query +select curdate(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "0", + "functionName" : "`curdate`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "curdate(1)" + } ] +} + + +-- !query +select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null) +-- !query schema +struct +-- !query output +1970-01-01 1972-09-27 NULL + + +-- !query +select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(null) +-- !query schema +struct +-- !query output +0 18600 NULL + + +-- !query +select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL 2016-12-31 2016-12-31 + + +-- !query +select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), + dayofweek('1582-10-15 13:10:15'), dayofweek(timestamp_ltz'1582-10-15 13:10:15'), dayofweek(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +7 5 7 NULL 6 6 6 + + +-- !query +select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null), + weekday('1582-10-15 13:10:15'), weekday(timestamp_ltz'1582-10-15 13:10:15'), weekday(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +5 3 5 NULL 4 4 4 + + +-- !query +select year('1500-01-01'), year('1582-10-15 13:10:15'), year(timestamp_ltz'1582-10-15 13:10:15'), year(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +1500 1582 1582 1582 + + +-- !query +select month('1500-01-01'), month('1582-10-15 13:10:15'), month(timestamp_ltz'1582-10-15 13:10:15'), month(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +1 10 10 10 + + +-- !query +select dayOfYear('1500-01-01'), dayOfYear('1582-10-15 13:10:15'), dayOfYear(timestamp_ltz'1582-10-15 13:10:15'), dayOfYear(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +1 288 288 288 + + +-- !query +select next_day("2015-07-23", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day("2015-07-23", "xx") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ILLEGAL_DAY_OF_WEEK", + "sqlState" : "22009", + "messageParameters" : { + "string" : "xx" + } +} + + +-- !query +select next_day("2015-07-23 12:12:12", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day(timestamp_ltz"2015-07-23 12:12:12", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day(timestamp_ntz"2015-07-23 12:12:12", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day("xx", "Mon") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'xx'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "next_day(\"xx\", \"Mon\")" + } ] +} + + +-- !query +select next_day(null, "Mon") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select next_day(null, "xx") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_add(date'2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1Y) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1S) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1L) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "date_add('2011-11-11', 1L)" + } ] +} + + +-- !query +select date_add('2011-11-11', 1.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_add('2011-11-11', 1.0)" + } ] +} + + +-- !query +select date_add('2011-11-11', 1E1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_add('2011-11-11', 1E1)" + } ] +} + + +-- !query +select date_add('2011-11-11', '1') +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', '1.2') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1.2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "date_add('2011-11-11', '1.2')" + } ] +} + + +-- !query +select date_add(null, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_add(date'2011-11-11', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_add(timestamp_ltz'2011-11-11 12:12:12', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add(timestamp_ntz'2011-11-11 12:12:12', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_sub(date'2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub('2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub('2011-11-11', 1Y) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub('2011-11-11', 1S) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub('2011-11-11', 1L) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "date_sub('2011-11-11', 1L)" + } ] +} + + +-- !query +select date_sub('2011-11-11', 1.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_sub('2011-11-11', 1.0)" + } ] +} + + +-- !query +select date_sub('2011-11-11', 1E1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_sub('2011-11-11', 1E1)" + } ] +} + + +-- !query +select date_sub(date'2011-11-11', '1') +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub(date'2011-11-11', '1.2') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1.2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "date_sub(date'2011-11-11', '1.2')" + } ] +} + + +-- !query +select date_sub(null, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_sub(date'2011-11-11', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_sub(timestamp_ltz'2011-11-11 12:12:12', 1) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub(timestamp_ntz'2011-11-11 12:12:12', 1) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_add('2011-11-11', int_str) from date_view +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_sub('2011-11-11', int_str) from date_view +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_add(date_str, 1) from date_view +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_sub(date_str, 1) from date_view +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date '2011-11-11' + 1E1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(DATE '2011-11-11', 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "date '2011-11-11' + 1E1" + } ] +} + + +-- !query +select date '2001-09-28' + 7Y +-- !query schema +struct +-- !query output +2001-10-05 + + +-- !query +select 7S + date '2001-09-28' +-- !query schema +struct +-- !query output +2001-10-05 + + +-- !query +select date '2001-10-01' - 7 +-- !query schema +struct +-- !query output +2001-09-24 + + +-- !query +select date '2001-10-01' - date '2001-09-28' +-- !query schema +struct<(DATE '2001-10-01' - DATE '2001-09-28'):interval day> +-- !query output +3 00:00:00.000000000 + + +-- !query +select date '2001-10-01' - '2001-09-28' +-- !query schema +struct<(DATE '2001-10-01' - 2001-09-28):interval day> +-- !query output +3 00:00:00.000000000 + + +-- !query +select '2001-10-01' - date '2001-09-28' +-- !query schema +struct<(2001-10-01 - DATE '2001-09-28'):interval day> +-- !query output +3 00:00:00.000000000 + + +-- !query +select date '2001-09-28' - null +-- !query schema +struct +-- !query output +NULL + + +-- !query +select null - date '2019-10-06' +-- !query schema +struct<(NULL - DATE '2019-10-06'):interval day> +-- !query output +NULL + + +-- !query +select date_str - date '2001-09-28' from date_view +-- !query schema +struct<(date_str - DATE '2001-09-28'):interval day> +-- !query output +3696 00:00:00.000000000 + + +-- !query +select date '2001-09-28' - date_str from date_view +-- !query schema +struct<(DATE '2001-09-28' - date_str):interval day> +-- !query output +-3696 00:00:00.000000000 + + +-- !query +select date'2011-11-11' + '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"DATE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "date'2011-11-11' + '1'" + } ] +} + + +-- !query +select '1' + date'2011-11-11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"DATE '2011-11-11'\"", + "inputType" : "\"DATE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(1, DATE '2011-11-11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' + date'2011-11-11'" + } ] +} + + +-- !query +select date'2011-11-11' + null +-- !query schema +struct +-- !query output +NULL + + +-- !query +select null + date'2011-11-11' +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date '2012-01-01' - interval '2-2' year to month, + date '2011-11-11' - interval '2' day, + date '2012-01-01' + interval '-2-2' year to month, + date '2011-11-11' + interval '-2' month, + - interval '2-2' year to month + date '2012-01-01', + interval '-2' day + date '2011-11-11' +-- !query schema +struct +-- !query output +2009-11-01 2011-11-09 2009-11-01 2011-09-11 2009-11-01 2011-11-09 + + +-- !query +select to_date('26/October/2015', 'dd/MMMMM/yyyy') +-- !query schema +struct +-- !query output +2015-10-26 + + +-- !query +select from_json('{"d":"26/October/2015"}', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct> +-- !query output +{"d":2015-10-26} + + +-- !query +select from_csv('26/October/2015', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct> +-- !query output +{"d":2015-10-26} + + +-- !query +select dateadd(MICROSECOND, 1001, timestamp'2022-02-25 01:02:03.123') +-- !query schema +struct +-- !query output +2022-02-25 01:02:03.124001 + + +-- !query +select date_add(MILLISECOND, -1, timestamp'2022-02-25 01:02:03.456') +-- !query schema +struct +-- !query output +2022-02-25 01:02:03.455 + + +-- !query +select dateadd(SECOND, 58, timestamp'2022-02-25 01:02:03') +-- !query schema +struct +-- !query output +2022-02-25 01:03:01 + + +-- !query +select date_add(MINUTE, -100, date'2022-02-25') +-- !query schema +struct +-- !query output +2022-02-24 22:20:00 + + +-- !query +select dateadd(HOUR, -1, timestamp'2022-02-25 01:02:03') +-- !query schema +struct +-- !query output +2022-02-25 00:02:03 + + +-- !query +select date_add(DAY, 367, date'2022-02-25') +-- !query schema +struct +-- !query output +2023-02-27 00:00:00 + + +-- !query +select dateadd(WEEK, -4, timestamp'2022-02-25 01:02:03') +-- !query schema +struct +-- !query output +2022-01-28 01:02:03 + + +-- !query +select date_add(MONTH, -1, timestamp'2022-02-25 01:02:03') +-- !query schema +struct +-- !query output +2022-01-25 01:02:03 + + +-- !query +select dateadd(QUARTER, 5, date'2022-02-25') +-- !query schema +struct +-- !query output +2023-05-25 00:00:00 + + +-- !query +select date_add(YEAR, 1, date'2022-02-25') +-- !query schema +struct +-- !query output +2023-02-25 00:00:00 + + +-- !query +select dateadd('MICROSECOND', 1001, timestamp'2022-02-25 01:02:03.123') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`dateadd`", + "invalidValue" : "'MICROSECOND'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "dateadd('MICROSECOND', 1001, timestamp'2022-02-25 01:02:03.123')" + } ] +} + + +-- !query +select date_add('QUARTER', 5, date'2022-02-25') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`date_add`", + "invalidValue" : "'QUARTER'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "date_add('QUARTER', 5, date'2022-02-25')" + } ] +} + + +-- !query +select datediff(MICROSECOND, timestamp'2022-02-25 01:02:03.123', timestamp'2022-02-25 01:02:03.124001') +-- !query schema +struct +-- !query output +1001 + + +-- !query +select date_diff(MILLISECOND, timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select datediff(SECOND, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 01:03:01') +-- !query schema +struct +-- !query output +58 + + +-- !query +select date_diff(MINUTE, date'2022-02-25', timestamp'2022-02-24 22:20:00') +-- !query schema +struct +-- !query output +-100 + + +-- !query +select datediff(HOUR, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 00:02:03') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select date_diff(DAY, date'2022-02-25', timestamp'2023-02-27 00:00:00') +-- !query schema +struct +-- !query output +367 + + +-- !query +select datediff(WEEK, timestamp'2022-02-25 01:02:03', timestamp'2022-01-28 01:02:03') +-- !query schema +struct +-- !query output +-4 + + +-- !query +select date_diff(MONTH, timestamp'2022-02-25 01:02:03', timestamp'2022-01-25 01:02:03') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select datediff(QUARTER, date'2022-02-25', date'2023-05-25') +-- !query schema +struct +-- !query output +5 + + +-- !query +select date_diff(YEAR, date'2022-02-25', date'2023-02-25') +-- !query schema +struct +-- !query output +1 + + +-- !query +select date_diff('MILLISECOND', timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`date_diff`", + "invalidValue" : "'MILLISECOND'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 103, + "fragment" : "date_diff('MILLISECOND', timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455')" + } ] +} + + +-- !query +select datediff('YEAR', date'2022-02-25', date'2023-02-25') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`datediff`", + "invalidValue" : "'YEAR'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "datediff('YEAR', date'2022-02-25', date'2023-02-25')" + } ] +} + + +-- !query +select timestamp '2019-01-01\t' +-- !query schema +struct +-- !query output +2019-01-01 00:00:00 + + +-- !query +select timestamp '2019-01-01中文' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2019-01-01中文'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "timestamp '2019-01-01中文'" + } ] +} + + +-- !query +select timestamp'4294967297' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'4294967297'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "timestamp'4294967297'" + } ] +} + + +-- !query +select timestamp'2021-01-01T12:30:4294967297.123456' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2021-01-01T12:30:4294967297.123456'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'" + } ] +} + + +-- !query +select current_timestamp = current_timestamp +-- !query schema +struct<(current_timestamp() = current_timestamp()):boolean> +-- !query output +true + + +-- !query +select current_timestamp() = current_timestamp() +-- !query schema +struct<(current_timestamp() = current_timestamp()):boolean> +-- !query output +true + + +-- !query +select localtimestamp() = localtimestamp() +-- !query schema +struct<(localtimestamp() = localtimestamp()):boolean> +-- !query output +true + + +-- !query +SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678) +-- !query schema +struct +-- !query output +2021-07-11 06:30:45.678 + + +-- !query +SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678, 'CET') +-- !query schema +struct +-- !query output +2021-07-10 21:30:45.678 + + +-- !query +SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "INVALID_FRACTION_OF_SECOND", + "sqlState" : "22023", + "messageParameters" : { + "secAndMicros" : "60.007" + } +} + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 1) +-- !query schema +struct +-- !query output +0001-01-01 01:01:01 + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 60) +-- !query schema +struct +-- !query output +0001-01-01 01:02:00 + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 61) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for SecondOfMinute (valid values 0 - 59): 61" + } +} + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 59.999999) +-- !query schema +struct +-- !query output +0001-01-01 01:01:59.999999 + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for SecondOfMinute (valid values 0 - 59): 99" + } +} + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for SecondOfMinute (valid values 0 - 59): 999" + } +} + + +-- !query +select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00 1931-01-07 00:30:00 NULL + + +-- !query +select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23)) +-- !query schema +struct +-- !query output +1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 + + +-- !query +select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00.123 1931-01-07 00:29:59.877 NULL + + +-- !query +select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00.123123 1931-01-07 00:29:59.876877 NULL + + +-- !query +select TIMESTAMP_SECONDS(1230219000123123) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_SECONDS(-1230219000123123) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_MILLIS(92233720368547758) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_MILLIS(-92233720368547758) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_SECONDS(0.1234567) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Rounding necessary + + +-- !query +select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567)) +-- !query schema +struct +-- !query output +1969-12-31 16:00:00.123456 1969-12-31 16:00:00.123456 + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + +-- !query +create temporary view ttf1 as select * from values + (1, 2), + (2, 3) + as ttf1(`current_date`, `current_timestamp`) +-- !query schema +struct<> +-- !query output + + + +-- !query +select typeof(current_date), typeof(current_timestamp) from ttf1 +-- !query schema +struct +-- !query output +int int +int int + + +-- !query +create temporary view ttf2 as select * from values + (1, 2), + (2, 3) + as ttf2(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2 +-- !query schema +struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean,a:int,b:int> +-- !query output +true true 1 2 +true true 2 3 + + +-- !query +select a, b from ttf2 order by a, current_date +-- !query schema +struct +-- !query output +1 2 +2 3 + + +-- !query +select UNIX_SECONDS(timestamp'2020-12-01 14:30:08Z'), UNIX_SECONDS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_SECONDS(null) +-- !query schema +struct +-- !query output +1606833008 1606833008 NULL + + +-- !query +select UNIX_MILLIS(timestamp'2020-12-01 14:30:08Z'), UNIX_MILLIS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MILLIS(null) +-- !query schema +struct +-- !query output +1606833008000 1606833008999 NULL + + +-- !query +select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MICROS(null) +-- !query schema +struct +-- !query output +1606833008000000 1606833008999999 NULL + + +-- !query +select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 + + +-- !query +select to_timestamp(1) +-- !query schema +struct +-- !query output +1969-12-31 16:00:01 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"2019-10-06 10:11:12.\"" + } +} + + +-- !query +select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"2019-10-06 10:11:12.0\"" + } +} + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"2019-10-06 10:11:12.1\"" + } +} + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"2019-10-06 10:11:12.12\"" + } +} + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"2019-10-06 10:11:12.123UTC\"" + } +} + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"2019-10-06 10:11:12.1234\"" + } +} + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"2019-10-06 10:11:12.12345CST\"" + } +} + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"2019-10-06 10:11:12.123456PST\"" + } +} + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"2019-10-06 10:11:12.1234567PST\"" + } +} + + +-- !query +select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"123456 2019-10-06 10:11:12.123456PST\"" + } +} + + +-- !query +select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"223456 2019-10-06 10:11:12.123456PST\"" + } +} + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"2019-10-06 10:11:12.1234\"" + } +} + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"2019-10-06 10:11:12.123\"" + } +} + + +-- !query +select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"2019-10-06 10:11:12\"" + } +} + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"2019-10-06 10:11:12.12\"" + } +} + + +-- !query +select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"2019-10-06 10:11\"" + } +} + + +-- !query +select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"2019-10-06S10:11:12.12345\"" + } +} + + +-- !query +select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"12.12342019-10-06S10:11\"" + } +} + + +-- !query +select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"12.1232019-10-06S10:11\"" + } +} + + +-- !query +select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"12.1232019-10-06S10:11\"" + } +} + + +-- !query +select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"12.1234019-10-06S10:11\"" + } +} + + +-- !query +select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'") +-- !query schema +struct +-- !query output +2019-10-06 00:00:00 + + +-- !query +select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd") +-- !query schema +struct +-- !query output +2019-10-06 00:00:00 + + +-- !query +select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.012 + + +-- !query +select to_timestamp("2019-10-06T10:11:12'", "yyyy-MM-dd'T'HH:mm:ss''") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("16", "dd") +-- !query schema +struct +-- !query output +1970-01-16 00:00:00 + + +-- !query +select to_timestamp("02-29", "MM-dd") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"02-29\"" + } +} + + +-- !query +select to_timestamp("2019 40", "yyyy mm") +-- !query schema +struct +-- !query output +2019-01-01 00:40:00 + + +-- !query +select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss") +-- !query schema +struct +-- !query output +2019-01-01 10:10:10 + + +-- !query +select timestamp'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query schema +struct<(TIMESTAMP '2011-11-11 11:11:11' - TIMESTAMP '2011-11-11 11:11:10'):interval day to second> +-- !query output +0 00:00:01.000000000 + + +-- !query +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' +-- !query schema +struct<(DATE '2020-01-01' - TIMESTAMP '2019-10-06 10:11:12.345678'):interval day to second> +-- !query output +86 13:48:47.654322000 + + +-- !query +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' +-- !query schema +struct<(TIMESTAMP '2019-10-06 10:11:12.345678' - DATE '2020-01-01'):interval day to second> +-- !query output +-86 13:48:47.654322000 + + +-- !query +select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' +-- !query schema +struct<(TIMESTAMP '2011-11-11 11:11:11' - 2011-11-11 11:11:10):interval day to second> +-- !query output +0 00:00:01.000000000 + + +-- !query +select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query schema +struct<(2011-11-11 11:11:11 - TIMESTAMP '2011-11-11 11:11:10'):interval day to second> +-- !query output +0 00:00:01.000000000 + + +-- !query +select timestamp'2011-11-11 11:11:11' - null +-- !query schema +struct<(TIMESTAMP '2011-11-11 11:11:11' - NULL):interval day to second> +-- !query output +NULL + + +-- !query +select null - timestamp'2011-11-11 11:11:11' +-- !query schema +struct<(NULL - TIMESTAMP '2011-11-11 11:11:11'):interval day to second> +-- !query output +NULL + + +-- !query +create temporary view ts_view as select '2011-11-11 11:11:11' str +-- !query schema +struct<> +-- !query output + + + +-- !query +select str - timestamp'2011-11-11 11:11:11' from ts_view +-- !query schema +struct<(str - TIMESTAMP '2011-11-11 11:11:11'):interval day to second> +-- !query output +0 00:00:00.000000000 + + +-- !query +select timestamp'2011-11-11 11:11:11' - str from ts_view +-- !query schema +struct<(TIMESTAMP '2011-11-11 11:11:11' - str):interval day to second> +-- !query output +0 00:00:00.000000000 + + +-- !query +select timestamp'2011-11-11 11:11:11' + '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "timestamp'2011-11-11 11:11:11' + '1'" + } ] +} + + +-- !query +select '1' + timestamp'2011-11-11 11:11:11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "'1' + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + null +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"VOID\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "timestamp'2011-11-11 11:11:11' + null" + } ] +} + + +-- !query +select null + timestamp'2011-11-11 11:11:11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "null + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + interval '2' day, + timestamp'2011-11-11 11:11:11' - interval '2-2' year to month, + timestamp'2011-11-11 11:11:11' + interval '-2' second, + timestamp'2011-11-11 11:11:11' - interval '12:12:12.123456789' hour to second, + - interval 2 years + timestamp'2011-11-11 11:11:11', + interval '1 12' day to hour + timestamp'2011-11-11 11:11:11' +-- !query schema +struct +-- !query output +2011-11-13 11:11:11 2009-09-11 11:11:11 2011-11-11 11:11:09 2011-11-10 22:58:58.876544 2009-11-11 11:11:11 2011-11-12 23:11:11 + + +-- !query +select date '2012-01-01' - interval 3 hours, + date '2012-01-01' + interval '12:12:12' hour to second, + interval '2' minute + date '2012-01-01' +-- !query schema +struct +-- !query output +2011-12-31 21:00:00 2012-01-01 12:12:12 2012-01-01 00:02:00 + + +-- !query +select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Unparseable date: \"2019-10-06 A\"" + } +} + + +-- !query +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE') +-- !query schema +struct +-- !query output +2020-05-22 00:00:00 + + +-- !query +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') +-- !query schema +struct +-- !query output +2020-05-22 00:00:00 + + +-- !query +select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') +-- !query schema +struct +-- !query output +1590130800 + + +-- !query +select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct> +-- !query output +{"t":2015-10-26 00:00:00} + + +-- !query +select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct> +-- !query output +{"t":2015-10-26 00:00:00} + + +-- !query +select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03') +-- !query schema +struct +-- !query output +2022-01-14 01:02:03 + + +-- !query +select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03') +-- !query schema +struct +-- !query output +2022-02-14 02:00:03 + + +-- !query +select timestampadd(YEAR, 1, date'2022-02-15') +-- !query schema +struct +-- !query output +2023-02-15 00:00:00 + + +-- !query +select timestampadd(SECOND, -1, date'2022-02-15') +-- !query schema +struct +-- !query output +2022-02-14 23:59:59 + + +-- !query +select timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampadd`", + "invalidValue" : "'MONTH'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03')" + } ] +} + + +-- !query +select timestampadd('SECOND', -1, date'2022-02-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampadd`", + "invalidValue" : "'SECOND'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "timestampadd('SECOND', -1, date'2022-02-15')" + } ] +} + + +-- !query +select timestampdiff(MONTH, timestamp'2022-02-14 01:02:03', timestamp'2022-01-14 01:02:03') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select timestampdiff(MINUTE, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03') +-- !query schema +struct +-- !query output +58 + + +-- !query +select timestampdiff(YEAR, date'2022-02-15', date'2023-02-15') +-- !query schema +struct +-- !query output +1 + + +-- !query +select timestampdiff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampdiff`", + "invalidValue" : "'MINUTE'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 94, + "fragment" : "timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03')" + } ] +} + + +-- !query +select timestampdiff('YEAR', date'2022-02-15', date'2023-02-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampdiff`", + "invalidValue" : "'YEAR'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "timestampdiff('YEAR', date'2022-02-15', date'2023-02-15')" + } ] +} + + +-- !query +select timediff(QUARTER, timestamp'2023-08-10 01:02:03', timestamp'2022-01-14 01:02:03') +-- !query schema +struct +-- !query output +-6 + + +-- !query +select timediff(HOUR, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 12:00:03') +-- !query schema +struct +-- !query output +10 + + +-- !query +select timediff(DAY, date'2022-02-15', date'2023-02-15') +-- !query schema +struct +-- !query output +365 + + +-- !query +select timediff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timediff`", + "invalidValue" : "'MINUTE'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 89, + "fragment" : "timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03')" + } ] +} + + +-- !query +select timediff('YEAR', date'2020-02-15', date'2023-02-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timediff`", + "invalidValue" : "'YEAR'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "timediff('YEAR', date'2020-02-15', date'2023-02-15')" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-parsing-invalid.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-parsing-invalid.sql.out new file mode 100644 index 000000000000..280040ba393b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-parsing-invalid.sql.out @@ -0,0 +1,467 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select to_timestamp('294248', 'y') +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select to_timestamp('1', 'yy') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '1' could not be parsed at index 0" + } +} + + +-- !query +select to_timestamp('-12', 'yy') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '-12' could not be parsed at index 0" + } +} + + +-- !query +select to_timestamp('123', 'yy') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '123' could not be parsed, unparsed text found at index 2" + } +} + + +-- !query +select to_timestamp('1', 'yyy') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '1' could not be parsed at index 0" + } +} + + +-- !query +select to_timestamp('1234567', 'yyyyyyy') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'yyyyyyy'" + } +} + + +-- !query +select to_timestamp('366', 'D') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Invalid date 'DayOfYear 366' as '1970' is not a leap year" + } +} + + +-- !query +select to_timestamp('9', 'DD') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '9' could not be parsed at index 0" + } +} + + +-- !query +select to_timestamp('366', 'DD') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Invalid date 'DayOfYear 366' as '1970' is not a leap year" + } +} + + +-- !query +select to_timestamp('9', 'DDD') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '9' could not be parsed at index 0" + } +} + + +-- !query +select to_timestamp('99', 'DDD') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '99' could not be parsed at index 0" + } +} + + +-- !query +select to_timestamp('30-365', 'dd-DDD') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31." + } +} + + +-- !query +select to_timestamp('11-365', 'MM-DDD') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Conflict found: Field MonthOfYear 11 differs from MonthOfYear 12 derived from 1970-12-31." + } +} + + +-- !query +select to_timestamp('2019-366', 'yyyy-DDD') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '2019-366' could not be parsed: Invalid date 'DayOfYear 366' as '2019' is not a leap year" + } +} + + +-- !query +select to_timestamp('12-30-365', 'MM-dd-DDD') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31." + } +} + + +-- !query +select to_timestamp('2020-01-365', 'yyyy-dd-DDD') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '2020-01-365' could not be parsed: Conflict found: Field DayOfMonth 30 differs from DayOfMonth 1 derived from 2020-12-30" + } +} + + +-- !query +select to_timestamp('2020-10-350', 'yyyy-MM-DDD') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '2020-10-350' could not be parsed: Conflict found: Field MonthOfYear 12 differs from MonthOfYear 10 derived from 2020-12-15" + } +} + + +-- !query +select to_timestamp('2020-11-31-366', 'yyyy-MM-dd-DDD') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '2020-11-31-366' could not be parsed: Invalid date 'NOVEMBER 31'" + } +} + + +-- !query +select from_csv('2018-366', 'date Date', map('dateFormat', 'yyyy-DDD')) +-- !query schema +struct> +-- !query output +{"date":null} + + +-- !query +select to_date("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" + } +} + + +-- !query +select to_date("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text 'Unparseable' could not be parsed at index 0" + } +} + + +-- !query +select to_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" + } +} + + +-- !query +select to_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text 'Unparseable' could not be parsed at index 0" + } +} + + +-- !query +select unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" + } +} + + +-- !query +select unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text 'Unparseable' could not be parsed at index 0" + } +} + + +-- !query +select to_unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10" + } +} + + +-- !query +select to_unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text 'Unparseable' could not be parsed at index 0" + } +} + + +-- !query +select cast("Unparseable" as timestamp) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'Unparseable'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "cast(\"Unparseable\" as timestamp)" + } ] +} + + +-- !query +select cast("Unparseable" as date) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'Unparseable'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast(\"Unparseable\" as date)" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-parsing-legacy.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-parsing-legacy.sql.out new file mode 100644 index 000000000000..b041c96bf287 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-parsing-legacy.sql.out @@ -0,0 +1,207 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select to_timestamp('1', 'y') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('009999', 'y') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('00', 'yy') +-- !query schema +struct +-- !query output +2000-01-01 00:00:00 + + +-- !query +select to_timestamp('99', 'yy') +-- !query schema +struct +-- !query output +1999-01-01 00:00:00 + + +-- !query +select to_timestamp('001', 'yyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('009999', 'yyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('0001', 'yyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('9999', 'yyyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('00001', 'yyyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('09999', 'yyyyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('000001', 'yyyyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('009999', 'yyyyyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('9', 'D') +-- !query schema +struct +-- !query output +1970-01-09 00:00:00 + + +-- !query +select to_timestamp('300', 'D') +-- !query schema +struct +-- !query output +1970-10-27 00:00:00 + + +-- !query +select to_timestamp('09', 'DD') +-- !query schema +struct +-- !query output +1970-01-09 00:00:00 + + +-- !query +select to_timestamp('99', 'DD') +-- !query schema +struct +-- !query output +1970-04-09 00:00:00 + + +-- !query +select to_timestamp('100', 'DD') +-- !query schema +struct +-- !query output +1970-04-10 00:00:00 + + +-- !query +select to_timestamp('009', 'DDD') +-- !query schema +struct +-- !query output +1970-01-09 00:00:00 + + +-- !query +select to_timestamp('365', 'DDD') +-- !query schema +struct +-- !query output +1970-12-31 00:00:00 + + +-- !query +select to_timestamp('31-365', 'dd-DDD') +-- !query schema +struct +-- !query output +1970-12-31 00:00:00 + + +-- !query +select to_timestamp('12-365', 'MM-DDD') +-- !query schema +struct +-- !query output +1970-12-31 00:00:00 + + +-- !query +select to_timestamp('2020-365', 'yyyy-DDD') +-- !query schema +struct +-- !query output +2020-12-30 00:00:00 + + +-- !query +select to_timestamp('12-31-365', 'MM-dd-DDD') +-- !query schema +struct +-- !query output +1970-12-31 00:00:00 + + +-- !query +select to_timestamp('2020-30-365', 'yyyy-dd-DDD') +-- !query schema +struct +-- !query output +2020-12-30 00:00:00 + + +-- !query +select to_timestamp('2020-12-350', 'yyyy-MM-DDD') +-- !query schema +struct +-- !query output +2020-12-15 00:00:00 + + +-- !query +select to_timestamp('2020-12-31-366', 'yyyy-MM-dd-DDD') +-- !query schema +struct +-- !query output +2020-12-31 00:00:00 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-parsing.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-parsing.sql.out new file mode 100644 index 000000000000..d45154f97005 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-parsing.sql.out @@ -0,0 +1,207 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select to_timestamp('1', 'y') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('009999', 'y') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('00', 'yy') +-- !query schema +struct +-- !query output +2000-01-01 00:00:00 + + +-- !query +select to_timestamp('99', 'yy') +-- !query schema +struct +-- !query output +2099-01-01 00:00:00 + + +-- !query +select to_timestamp('001', 'yyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('009999', 'yyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('0001', 'yyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('9999', 'yyyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('00001', 'yyyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('09999', 'yyyyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('000001', 'yyyyyy') +-- !query schema +struct +-- !query output +0001-01-01 00:00:00 + + +-- !query +select to_timestamp('009999', 'yyyyyy') +-- !query schema +struct +-- !query output +9999-01-01 00:00:00 + + +-- !query +select to_timestamp('9', 'D') +-- !query schema +struct +-- !query output +1970-01-09 00:00:00 + + +-- !query +select to_timestamp('300', 'D') +-- !query schema +struct +-- !query output +1970-10-27 00:00:00 + + +-- !query +select to_timestamp('09', 'DD') +-- !query schema +struct +-- !query output +1970-01-09 00:00:00 + + +-- !query +select to_timestamp('99', 'DD') +-- !query schema +struct +-- !query output +1970-04-09 00:00:00 + + +-- !query +select to_timestamp('100', 'DD') +-- !query schema +struct +-- !query output +1970-04-10 00:00:00 + + +-- !query +select to_timestamp('009', 'DDD') +-- !query schema +struct +-- !query output +1970-01-09 00:00:00 + + +-- !query +select to_timestamp('365', 'DDD') +-- !query schema +struct +-- !query output +1970-12-31 00:00:00 + + +-- !query +select to_timestamp('31-365', 'dd-DDD') +-- !query schema +struct +-- !query output +1970-12-31 00:00:00 + + +-- !query +select to_timestamp('12-365', 'MM-DDD') +-- !query schema +struct +-- !query output +1970-12-31 00:00:00 + + +-- !query +select to_timestamp('2020-365', 'yyyy-DDD') +-- !query schema +struct +-- !query output +2020-12-30 00:00:00 + + +-- !query +select to_timestamp('12-31-365', 'MM-dd-DDD') +-- !query schema +struct +-- !query output +1970-12-31 00:00:00 + + +-- !query +select to_timestamp('2020-30-365', 'yyyy-dd-DDD') +-- !query schema +struct +-- !query output +2020-12-30 00:00:00 + + +-- !query +select to_timestamp('2020-12-350', 'yyyy-MM-DDD') +-- !query schema +struct +-- !query output +2020-12-15 00:00:00 + + +-- !query +select to_timestamp('2020-12-31-366', 'yyyy-MM-dd-DDD') +-- !query schema +struct +-- !query output +2020-12-31 00:00:00 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-special.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-special.sql.out new file mode 100644 index 000000000000..4f7ecf4b03f4 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/datetime-special.sql.out @@ -0,0 +1,31 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select date'999999-03-18', date'-0001-1-28', date'0015' +-- !query schema +struct +-- !query output ++999999-03-18 -0001-01-28 0015-01-01 + + +-- !query +select make_date(999999, 3, 18), make_date(-1, 1, 28) +-- !query schema +struct +-- !query output ++999999-03-18 -0001-01-28 + + +-- !query +select timestamp'-1969-12-31 16:00:00', timestamp'-0015-03-18 16:00:00', timestamp'-000001', timestamp'99999-03-18T12:03:17' +-- !query schema +struct +-- !query output +-1969-12-31 16:00:00 -0015-03-18 16:00:00 -0001-01-01 00:00:00 +99999-03-18 12:03:17 + + +-- !query +select make_timestamp(-1969, 12, 31, 16, 0, 0.0), make_timestamp(-15, 3, 18, 16, 0, 0.0), make_timestamp(99999, 3, 18, 12, 3, 17.0) +-- !query schema +struct +-- !query output +-1969-12-31 16:00:00 -0015-03-18 16:00:00 +99999-03-18 12:03:17 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/decimalArithmeticOperations.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/decimalArithmeticOperations.sql.out new file mode 100644 index 000000000000..cb52778c420a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/decimalArithmeticOperations.sql.out @@ -0,0 +1,818 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1.0 as a, 0.0 as b +-- !query schema +struct<> +-- !query output + + + +-- !query +select a / b from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "a / b" + } ] +} + + +-- !query +select a % b from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "a % b" + } ] +} + + +-- !query +select pmod(a, b) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "pmod(a, b)" + } ] +} + + +-- !query +create table decimals_test(id int, a decimal(38,18), b decimal(38,18)) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into decimals_test values(1, 100.0, 999.0), (2, 12345.123, 12345.123), + (3, 0.1234567891011, 1234.1), (4, 123456789123456789.0, 1.123456789123456789) +-- !query schema +struct<> +-- !query output + + + +-- !query +select id, a+b, a-b, a*b, a/b from decimals_test order by id +-- !query schema +struct +-- !query output +1 1099.00000000000000000 -899.00000000000000000 99900.000000 0.100100 +2 24690.24600000000000000 0.00000000000000000 152402061.885129 1.000000 +3 1234.22345678910110000 -1233.97654321089890000 152.358023 0.000100 +4 123456789123456790.12345678912345679 123456789123456787.87654321087654321 138698367904130467.515623 109890109097814272.043109 + + +-- !query +select id, a*10, b/10 from decimals_test order by id +-- !query schema +struct +-- !query output +1 1000.000000000000000 99.900000000000000000 +2 123451.230000000000000 1234.512300000000000000 +3 1.234567891011000 123.410000000000000000 +4 1234567891234567890.000000000000000 0.112345678912345679 + + +-- !query +select 10.3 * 3.0 +-- !query schema +struct<(10.3 * 3.0):decimal(6,2)> +-- !query output +30.90 + + +-- !query +select 10.3000 * 3.0 +-- !query schema +struct<(10.3000 * 3.0):decimal(9,5)> +-- !query output +30.90000 + + +-- !query +select 10.30000 * 30.0 +-- !query schema +struct<(10.30000 * 30.0):decimal(11,6)> +-- !query output +309.000000 + + +-- !query +select 10.300000000000000000 * 3.000000000000000000 +-- !query schema +struct<(10.300000000000000000 * 3.000000000000000000):decimal(38,34)> +-- !query output +30.9000000000000000000000000000000000 + + +-- !query +select 10.300000000000000000 * 3.0000000000000000000 +-- !query schema +struct<(10.300000000000000000 * 3.0000000000000000000):decimal(38,34)> +-- !query output +30.9000000000000000000000000000000000 + + +-- !query +select 2.35E10 * 1.0 +-- !query schema +struct<(2.35E10 * 1.0):double> +-- !query output +2.35E10 + + +-- !query +select (5e36BD + 0.1) + 5e36BD +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "1", + "value" : "10000000000000000000000000000000000000.1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "(5e36BD + 0.1) + 5e36BD" + } ] +} + + +-- !query +select (-4e36BD - 0.1) - 7e36BD +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "1", + "value" : "-11000000000000000000000000000000000000.1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "(-4e36BD - 0.1) - 7e36BD" + } ] +} + + +-- !query +select 12345678901234567890.0 * 12345678901234567890.0 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "2", + "value" : "152415787532388367501905199875019052100" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "12345678901234567890.0 * 12345678901234567890.0" + } ] +} + + +-- !query +select 1e35BD / 0.1 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "6", + "value" : "1000000000000000000000000000000000000.000000000000000000000000000000000000000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "1e35BD / 0.1" + } ] +} + + +-- !query +select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "0", + "value" : "15241578753238836750190519987501905210000000000000000000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "1.2345678901234567890E30BD * 1.2345678901234567890E25BD" + } ] +} + + +-- !query +select 12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345 +-- !query schema +struct<(12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345):decimal(38,6)> +-- !query output +10012345678912345678912345678911.246907 + + +-- !query +select 123456789123456789.1234567890 * 1.123456789123456789 +-- !query schema +struct<(123456789123456789.1234567890 * 1.123456789123456789):decimal(38,18)> +-- !query output +138698367904130467.654320988515622621 + + +-- !query +select 12345678912345.123456789123 / 0.000000012345678 +-- !query schema +struct<(12345678912345.123456789123 / 1.2345678E-8):decimal(38,9)> +-- !query output +1000000073899961059796.725866332 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) + CAST(90 AS DECIMAL(3, 1)) +-- !query schema +struct +-- !query output +100.0 +20.0 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) - CAST(-90 AS DECIMAL(3, 1)) +-- !query schema +struct +-- !query output +100.0 +20.0 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) * CAST(10 AS DECIMAL(3, 1)) +-- !query schema +struct +-- !query output +100.00 +20.00 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) / CAST(10 AS DECIMAL(3, 1)) +-- !query schema +struct +-- !query output +1.000000 +20.000000 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(10, 2)) % CAST(3 AS DECIMAL(5, 1)) +-- !query schema +struct +-- !query output +1.00 +20.00 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT pmod(CAST(10 AS DECIMAL(10, 2)), CAST(3 AS DECIMAL(5, 1))) +-- !query schema +struct +-- !query output +1.00 +20.00 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(10, 2)) div CAST(3 AS DECIMAL(5, 1)) +-- !query schema +struct +-- !query output +20.0 +3.0 + + +-- !query +set spark.sql.decimalOperations.allowPrecisionLoss=false +-- !query schema +struct +-- !query output +spark.sql.decimalOperations.allowPrecisionLoss false + + +-- !query +select /*+ COALESCE(1) */ id, a+b, a-b, a*b, a/b from decimals_test order by id +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "36", + "value" : "152.358023429667510000000000000000000000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 41, + "stopIndex" : 43, + "fragment" : "a*b" + } ] +} + + +-- !query +select id, a*10, b/10 from decimals_test order by id +-- !query schema +struct +-- !query output +1 1000.000000000000000000 99.9000000000000000000 +2 123451.230000000000000000 1234.5123000000000000000 +3 1.234567891011000000 123.4100000000000000000 +4 1234567891234567890.000000000000000000 0.1123456789123456789 + + +-- !query +select 10.3 * 3.0 +-- !query schema +struct<(10.3 * 3.0):decimal(6,2)> +-- !query output +30.90 + + +-- !query +select 10.3000 * 3.0 +-- !query schema +struct<(10.3000 * 3.0):decimal(9,5)> +-- !query output +30.90000 + + +-- !query +select 10.30000 * 30.0 +-- !query schema +struct<(10.30000 * 30.0):decimal(11,6)> +-- !query output +309.000000 + + +-- !query +select 10.300000000000000000 * 3.000000000000000000 +-- !query schema +struct<(10.300000000000000000 * 3.000000000000000000):decimal(38,36)> +-- !query output +30.900000000000000000000000000000000000 + + +-- !query +select 10.300000000000000000 * 3.0000000000000000000 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "37", + "value" : "30.9000000000000000000000000000000000000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "10.300000000000000000 * 3.0000000000000000000" + } ] +} + + +-- !query +select 2.35E10 * 1.0 +-- !query schema +struct<(2.35E10 * 1.0):double> +-- !query output +2.35E10 + + +-- !query +select (5e36BD + 0.1) + 5e36BD +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "1", + "value" : "10000000000000000000000000000000000000.1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "(5e36BD + 0.1) + 5e36BD" + } ] +} + + +-- !query +select (-4e36BD - 0.1) - 7e36BD +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "1", + "value" : "-11000000000000000000000000000000000000.1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "(-4e36BD - 0.1) - 7e36BD" + } ] +} + + +-- !query +select 12345678901234567890.0 * 12345678901234567890.0 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "2", + "value" : "152415787532388367501905199875019052100" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "12345678901234567890.0 * 12345678901234567890.0" + } ] +} + + +-- !query +select 1e35BD / 0.1 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "3", + "value" : "1000000000000000000000000000000000000.000000000000000000000000000000000000000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "1e35BD / 0.1" + } ] +} + + +-- !query +select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "0", + "value" : "15241578753238836750190519987501905210000000000000000000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "1.2345678901234567890E30BD * 1.2345678901234567890E25BD" + } ] +} + + +-- !query +select 12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "7", + "value" : "10012345678912345678912345678911.2469067" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 84, + "fragment" : "12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345" + } ] +} + + +-- !query +select 123456789123456789.1234567890 * 1.123456789123456789 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "28", + "value" : "138698367904130467.654320988515622620750" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "123456789123456789.1234567890 * 1.123456789123456789" + } ] +} + + +-- !query +select 12345678912345.123456789123 / 0.000000012345678 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "18", + "value" : "1000000073899961059796.725866331521039184725213147467478092333" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "12345678912345.123456789123 / 0.000000012345678" + } ] +} + + +-- !query +select 1.0123456789012345678901234567890123456e36BD / 0.1 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "2", + "value" : "10123456789012345678901234567890123456.000000000000000000000000000000000000000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "1.0123456789012345678901234567890123456e36BD / 0.1" + } ] +} + + +-- !query +select 1.0123456789012345678901234567890123456e35BD / 1.0 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "3", + "value" : "101234567890123456789012345678901234.560000000000000000000000000000000000000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "1.0123456789012345678901234567890123456e35BD / 1.0" + } ] +} + + +-- !query +select 1.0123456789012345678901234567890123456e34BD / 1.0 +-- !query schema +struct<(10123456789012345678901234567890123.456 / 1.0):decimal(38,3)> +-- !query output +10123456789012345678901234567890123.456 + + +-- !query +select 1.0123456789012345678901234567890123456e33BD / 1.0 +-- !query schema +struct<(1012345678901234567890123456789012.3456 / 1.0):decimal(38,4)> +-- !query output +1012345678901234567890123456789012.3456 + + +-- !query +select 1.0123456789012345678901234567890123456e32BD / 1.0 +-- !query schema +struct<(101234567890123456789012345678901.23456 / 1.0):decimal(38,5)> +-- !query output +101234567890123456789012345678901.23456 + + +-- !query +select 1.0123456789012345678901234567890123456e31BD / 1.0 +-- !query schema +struct<(10123456789012345678901234567890.123456 / 1.0):decimal(38,6)> +-- !query output +10123456789012345678901234567890.123456 + + +-- !query +select 1.0123456789012345678901234567890123456e31BD / 0.1 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "38", + "scale" : "6", + "value" : "101234567890123456789012345678901.234560000000000000000000000000000000000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "1.0123456789012345678901234567890123456e31BD / 0.1" + } ] +} + + +-- !query +select 1.0123456789012345678901234567890123456e31BD / 10.0 +-- !query schema +struct<(10123456789012345678901234567890.123456 / 10.0):decimal(38,7)> +-- !query output +1012345678901234567890123456789.0123456 + + +-- !query +drop table decimals_test +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/describe-part-after-analyze.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/describe-part-after-analyze.sql.out new file mode 100644 index 000000000000..b04ad638dd2c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/describe-part-after-analyze.sql.out @@ -0,0 +1,255 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE t (key STRING, value STRING, ds STRING, hr INT) USING parquet + PARTITIONED BY (ds, hr) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO TABLE t PARTITION (ds='2017-08-01', hr=10) +VALUES ('k1', 100), ('k2', 200), ('k3', 300) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO TABLE t PARTITION (ds='2017-08-01', hr=11) +VALUES ('k1', 101), ('k2', 201), ('k3', 301), ('k4', 401) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO TABLE t PARTITION (ds='2017-09-01', hr=5) +VALUES ('k1', 102), ('k2', 202) +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) +-- !query schema +struct +-- !query output +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-08-01, hr=10] +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 +Created Time [not included in comparison] +Last Access [not included in comparison] + +# Storage Information +Location [not included in comparison]/{warehouse_dir}/t + + +-- !query +ANALYZE TABLE t PARTITION (ds='2017-08-01', hr=10) COMPUTE STATISTICS +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) +-- !query schema +struct +-- !query output +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-08-01, hr=10] +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 +Created Time [not included in comparison] +Last Access [not included in comparison] +Partition Statistics [not included in comparison] bytes, 3 rows + +# Storage Information +Location [not included in comparison]/{warehouse_dir}/t + + +-- !query +ANALYZE TABLE t PARTITION (ds='2017-08-01') COMPUTE STATISTICS +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) +-- !query schema +struct +-- !query output +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-08-01, hr=10] +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 +Created Time [not included in comparison] +Last Access [not included in comparison] +Partition Statistics [not included in comparison] bytes, 3 rows + +# Storage Information +Location [not included in comparison]/{warehouse_dir}/t + + +-- !query +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=11) +-- !query schema +struct +-- !query output +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-08-01, hr=11] +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=11 +Created Time [not included in comparison] +Last Access [not included in comparison] +Partition Statistics [not included in comparison] bytes, 4 rows + +# Storage Information +Location [not included in comparison]/{warehouse_dir}/t + + +-- !query +ANALYZE TABLE t PARTITION (ds, hr) COMPUTE STATISTICS +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=10) +-- !query schema +struct +-- !query output +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-08-01, hr=10] +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=10 +Created Time [not included in comparison] +Last Access [not included in comparison] +Partition Statistics [not included in comparison] bytes, 3 rows + +# Storage Information +Location [not included in comparison]/{warehouse_dir}/t + + +-- !query +DESC EXTENDED t PARTITION (ds='2017-08-01', hr=11) +-- !query schema +struct +-- !query output +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-08-01, hr=11] +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-08-01/hr=11 +Created Time [not included in comparison] +Last Access [not included in comparison] +Partition Statistics [not included in comparison] bytes, 4 rows + +# Storage Information +Location [not included in comparison]/{warehouse_dir}/t + + +-- !query +DESC EXTENDED t PARTITION (ds='2017-09-01', hr=5) +-- !query schema +struct +-- !query output +key string +value string +ds string +hr int +# Partition Information +# col_name data_type comment +ds string +hr int + +# Detailed Partition Information +Database default +Table t +Partition Values [ds=2017-09-01, hr=5] +Location [not included in comparison]/{warehouse_dir}/t/ds=2017-09-01/hr=5 +Created Time [not included in comparison] +Last Access [not included in comparison] +Partition Statistics [not included in comparison] bytes, 2 rows + +# Storage Information +Location [not included in comparison]/{warehouse_dir}/t + + +-- !query +DROP TABLE t +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/describe-query.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/describe-query.sql.out new file mode 100644 index 000000000000..c02d4b4adaf8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/describe-query.sql.out @@ -0,0 +1,199 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE table desc_temp1 (key int COMMENT 'column_comment', val string) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE table desc_temp2 (key int, val string) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC SELECT key, key + 1 as plusone FROM desc_temp1 +-- !query schema +struct +-- !query output +key int column_comment +plusone int + + +-- !query +DESC QUERY SELECT * FROM desc_temp2 +-- !query schema +struct +-- !query output +key int +val string + + +-- !query +DESC SELECT key, COUNT(*) as count FROM desc_temp1 group by key +-- !query schema +struct +-- !query output +key int column_comment +count bigint + + +-- !query +DESC SELECT 10.00D as col1 +-- !query schema +struct +-- !query output +col1 double + + +-- !query +DESC QUERY SELECT key FROM desc_temp1 UNION ALL select CAST(1 AS DOUBLE) +-- !query schema +struct +-- !query output +key double + + +-- !query +DESC QUERY VALUES(1.00D, 'hello') as tab1(col1, col2) +-- !query schema +struct +-- !query output +col1 double +col2 string + + +-- !query +DESC QUERY FROM desc_temp1 a SELECT * +-- !query schema +struct +-- !query output +key int column_comment +val string + + +-- !query +DESC WITH s AS (SELECT 'hello' as col1) SELECT * FROM s +-- !query schema +struct +-- !query output +col1 string + + +-- !query +DESCRIBE QUERY WITH s AS (SELECT * from desc_temp1) SELECT * FROM s +-- !query schema +struct +-- !query output +key int column_comment +val string + + +-- !query +DESCRIBE SELECT * FROM (FROM desc_temp2 select * select *) +-- !query schema +struct +-- !query output +key int +val string + + +-- !query +DESCRIBE INSERT INTO desc_temp1 values (1, 'val1') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'desc_temp1'", + "hint" : "" + } +} + + +-- !query +DESCRIBE INSERT INTO desc_temp1 SELECT * FROM desc_temp2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'desc_temp1'", + "hint" : "" + } +} + + +-- !query +DESCRIBE + FROM desc_temp1 a + insert into desc_temp1 select * + insert into desc_temp2 select * +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'insert'", + "hint" : "" + } +} + + +-- !query +EXPLAIN DESC QUERY SELECT * FROM desc_temp2 WHERE key > 0 +-- !query schema +struct +-- !query output +== Physical Plan == +Execute DescribeQueryCommand + +- DescribeQueryCommand SELECT * FROM desc_temp2 WHERE key > 0 + + +-- !query +EXPLAIN EXTENDED DESC WITH s AS (SELECT 'hello' as col1) SELECT * FROM s +-- !query schema +struct +-- !query output +== Parsed Logical Plan == +DescribeQueryCommand WITH s AS (SELECT 'hello' as col1) SELECT * FROM s + +== Analyzed Logical Plan == +col_name: string, data_type: string, comment: string +DescribeQueryCommand WITH s AS (SELECT 'hello' as col1) SELECT * FROM s + +== Optimized Logical Plan == +DescribeQueryCommand WITH s AS (SELECT 'hello' as col1) SELECT * FROM s + +== Physical Plan == +Execute DescribeQueryCommand + +- DescribeQueryCommand WITH s AS (SELECT 'hello' as col1) SELECT * FROM s + + +-- !query +DROP TABLE desc_temp1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE desc_temp2 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/describe-table-after-alter-table.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/describe-table-after-alter-table.sql.out new file mode 100644 index 000000000000..f7a8a4464607 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/describe-table-after-alter-table.sql.out @@ -0,0 +1,163 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE table_with_comment (a STRING, b INT, c STRING, d STRING) USING parquet COMMENT 'added' +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC FORMATTED table_with_comment +-- !query schema +struct +-- !query output +a string +b int +c string +d string + +# Detailed Table Information +Catalog spark_catalog +Database default +Table table_with_comment +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Comment added +Location [not included in comparison]/{warehouse_dir}/table_with_comment + + +-- !query +ALTER TABLE table_with_comment SET TBLPROPERTIES("comment"= "modified comment", "type"= "parquet") +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC FORMATTED table_with_comment +-- !query schema +struct +-- !query output +a string +b int +c string +d string + +# Detailed Table Information +Catalog spark_catalog +Database default +Table table_with_comment +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Comment modified comment +Table Properties [type=parquet] +Location [not included in comparison]/{warehouse_dir}/table_with_comment + + +-- !query +DROP TABLE table_with_comment +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE table_comment (a STRING, b INT) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC FORMATTED table_comment +-- !query schema +struct +-- !query output +a string +b int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table table_comment +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/table_comment + + +-- !query +ALTER TABLE table_comment SET TBLPROPERTIES(comment = "added comment") +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC formatted table_comment +-- !query schema +struct +-- !query output +a string +b int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table table_comment +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Comment added comment +Location [not included in comparison]/{warehouse_dir}/table_comment + + +-- !query +ALTER TABLE table_comment UNSET TBLPROPERTIES IF EXISTS ('comment') +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC FORMATTED table_comment +-- !query schema +struct +-- !query output +a string +b int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table table_comment +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/table_comment + + +-- !query +DROP TABLE table_comment +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/describe.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/describe.sql.out new file mode 100644 index 000000000000..3659267e8e9f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/describe.sql.out @@ -0,0 +1,988 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet + OPTIONS (a '1', b '2', password 'password') + PARTITIONED BY (c, d) CLUSTERED BY (a) SORTED BY (b ASC) INTO 2 BUCKETS + COMMENT 'table_comment' + TBLPROPERTIES (t 'test', password 'password') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW temp_v AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW temp_Data_Source_View + USING org.apache.spark.sql.sources.DDLScanSource + OPTIONS ( + From '1', + To '10', + Table 'test1') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER TABLE t SET TBLPROPERTIES (e = '3') +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER TABLE t ADD PARTITION (c='Us', d=1) +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE t +-- !query schema +struct +-- !query output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + + +-- !query +DESCRIBE EXTENDED t AS JSON +-- !query schema +struct +-- !query output +{"table_name":"t","catalog_name":"spark_catalog","namespace":["default"],"schema_name":"default","columns":[{"name":"a","type":{"name":"string","collation":"UTF8_BINARY"},"nullable":true},{"name":"b","type":{"name":"int"},"nullable":true},{"name":"c","type":{"name":"string","collation":"UTF8_BINARY"},"nullable":true},{"name":"d","type":{"name":"string","collation":"UTF8_BINARY"},"nullable":true}],"num_buckets":2,"bucket_columns":["a"],"sort_columns":["b"],"location":"file:[not included in comparison]/{warehouse_dir}/t","storage_properties":{"a":"1","b":"2","password":"*********(redacted)"},"created_time [not included in comparison]":"None","last_access [not included in comparison]":"None","created_by [not included in comparison]":"None","type":"MANAGED","provider":"parquet","comment":"table_comment","table_properties":{"e":"3","password":"*********(redacted)","t":"test"},"partition_provider":"Catalog","partition_columns":["c","d"],"collation":"UTF8_BINARY"} + + +-- !query +DESCRIBE t AS JSON +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "DESCRIBE_JSON_NOT_EXTENDED", + "sqlState" : "0A000", + "messageParameters" : { + "tableName" : "t" + } +} + + +-- !query +DESC FORMATTED t a AS JSON +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_FEATURE.DESC_TABLE_COLUMN_JSON", + "sqlState" : "0A000" +} + + +-- !query +DESC default.t +-- !query schema +struct +-- !query output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + + +-- !query +DESC TABLE t +-- !query schema +struct +-- !query output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + + +-- !query +DESC FORMATTED t +-- !query schema +struct +-- !query output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Table Information +Catalog spark_catalog +Database default +Table t +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Comment table_comment +Table Properties [e=3, password=*********(redacted), t=test] +Location [not included in comparison]/{warehouse_dir}/t +Storage Properties [a=1, b=2, password=*********(redacted)] +Partition Provider Catalog + + +-- !query +DESC EXTENDED t +-- !query schema +struct +-- !query output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Table Information +Catalog spark_catalog +Database default +Table t +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Comment table_comment +Table Properties [e=3, password=*********(redacted), t=test] +Location [not included in comparison]/{warehouse_dir}/t +Storage Properties [a=1, b=2, password=*********(redacted)] +Partition Provider Catalog + + +-- !query +ALTER TABLE t UNSET TBLPROPERTIES (e) +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC EXTENDED t +-- !query schema +struct +-- !query output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Table Information +Catalog spark_catalog +Database default +Table t +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Comment table_comment +Table Properties [password=*********(redacted), t=test] +Location [not included in comparison]/{warehouse_dir}/t +Storage Properties [a=1, b=2, password=*********(redacted)] +Partition Provider Catalog + + +-- !query +ALTER TABLE t UNSET TBLPROPERTIES (comment) +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC EXTENDED t +-- !query schema +struct +-- !query output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Table Information +Catalog spark_catalog +Database default +Table t +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Table Properties [password=*********(redacted), t=test] +Location [not included in comparison]/{warehouse_dir}/t +Storage Properties [a=1, b=2, password=*********(redacted)] +Partition Provider Catalog + + +-- !query +DESC t PARTITION (c='Us', d=1) +-- !query schema +struct +-- !query output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + + +-- !query +DESC EXTENDED t PARTITION (c='Us', d=1) AS JSON +-- !query schema +struct +-- !query output +{"table_name":"t","catalog_name":"spark_catalog","namespace":["default"],"schema_name":"default","columns":[{"name":"a","type":{"name":"string","collation":"UTF8_BINARY"},"nullable":true},{"name":"b","type":{"name":"int"},"nullable":true},{"name":"c","type":{"name":"string","collation":"UTF8_BINARY"},"nullable":true},{"name":"d","type":{"name":"string","collation":"UTF8_BINARY"},"nullable":true}],"partition_values":{"c":"Us","d":"1"},"location":"file:[not included in comparison]/{warehouse_dir}/t/c=Us/d=1","storage_properties":{"a":"1","b":"2","password":"*********(redacted)"},"created_time [not included in comparison]":"None","last_access [not included in comparison]":"None","created_by [not included in comparison]":"None","type":"MANAGED","provider":"parquet","num_buckets":2,"bucket_columns":["a"],"sort_columns":["b"],"table_properties":{"password":"*********(redacted)","t":"test"},"partition_provider":"Catalog","partition_columns":["c","d"],"collation":"UTF8_BINARY"} + + +-- !query +DESC EXTENDED t PARTITION (c='Us', d=1) +-- !query schema +struct +-- !query output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Partition Information +Database default +Table t +Partition Values [c=Us, d=1] +Location [not included in comparison]/{warehouse_dir}/t/c=Us/d=1 +Storage Properties [a=1, b=2, password=*********(redacted)] +Created Time [not included in comparison] +Last Access [not included in comparison] + +# Storage Information +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Location [not included in comparison]/{warehouse_dir}/t +Storage Properties [a=1, b=2, password=*********(redacted)] + + +-- !query +DESC FORMATTED t PARTITION (c='Us', d=1) +-- !query schema +struct +-- !query output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Partition Information +Database default +Table t +Partition Values [c=Us, d=1] +Location [not included in comparison]/{warehouse_dir}/t/c=Us/d=1 +Storage Properties [a=1, b=2, password=*********(redacted)] +Created Time [not included in comparison] +Last Access [not included in comparison] + +# Storage Information +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Location [not included in comparison]/{warehouse_dir}/t +Storage Properties [a=1, b=2, password=*********(redacted)] + + +-- !query +DESC EXTENDED t PARTITION (C='Us', D=1) +-- !query schema +struct +-- !query output +a string +b int +c string +d string +# Partition Information +# col_name data_type comment +c string +d string + +# Detailed Partition Information +Database default +Table t +Partition Values [c=Us, d=1] +Location [not included in comparison]/{warehouse_dir}/t/c=Us/d=1 +Storage Properties [a=1, b=2, password=*********(redacted)] +Created Time [not included in comparison] +Last Access [not included in comparison] + +# Storage Information +Num Buckets 2 +Bucket Columns [`a`] +Sort Columns [`b`] +Location [not included in comparison]/{warehouse_dir}/t +Storage Properties [a=1, b=2, password=*********(redacted)] + + +-- !query +DESC t PARTITION (c='Us', d=2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException +{ + "errorClass" : "PARTITIONS_NOT_FOUND", + "sqlState" : "428FT", + "messageParameters" : { + "partitionList" : "PARTITION (`c` = Us, `d` = 2)", + "tableName" : "`default`.`t`" + } +} + + +-- !query +DESC t PARTITION (c='Us') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1232", + "messageParameters" : { + "partitionColumnNames" : "c, d", + "specKeys" : "c", + "tableName" : "`spark_catalog`.`default`.`t`" + } +} + + +-- !query +DESC t PARTITION (c='Us', d) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.EMPTY_PARTITION_VALUE", + "sqlState" : "42000", + "messageParameters" : { + "partKey" : "`d`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 28, + "fragment" : "DESC t PARTITION (c='Us', d)" + } ] +} + + +-- !query +DESC temp_v +-- !query schema +struct +-- !query output +a string +b int +c string +d string + + +-- !query +DESC TABLE temp_v +-- !query schema +struct +-- !query output +a string +b int +c string +d string + + +-- !query +DESC FORMATTED temp_v +-- !query schema +struct +-- !query output +a string +b int +c string +d string + + +-- !query +DESC EXTENDED temp_v +-- !query schema +struct +-- !query output +a string +b int +c string +d string + + +-- !query +DESC temp_Data_Source_View +-- !query schema +struct +-- !query output +intType int test comment test1 +stringType string +dateType date +timestampType timestamp +doubleType double +bigintType bigint +tinyintType tinyint +decimalType decimal(10,0) +fixedDecimalType decimal(5,1) +binaryType binary +booleanType boolean +smallIntType smallint +floatType float +mapType map +arrayType array +structType struct + + +-- !query +DESC temp_v PARTITION (c='Us', d=1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "FORBIDDEN_OPERATION", + "sqlState" : "42809", + "messageParameters" : { + "objectName" : "`temp_v`", + "objectType" : "TEMPORARY VIEW", + "statement" : "DESC PARTITION" + } +} + + +-- !query +DESC v +-- !query schema +struct +-- !query output +a string +b int +c string +d string + + +-- !query +DESC TABLE v +-- !query schema +struct +-- !query output +a string +b int +c string +d string + + +-- !query +DESC FORMATTED v +-- !query schema +struct +-- !query output +a string +b int +c string +d string + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`a`, `b`, `c`, `d`] + + +-- !query +DESC EXTENDED v +-- !query schema +struct +-- !query output +a string +b int +c string +d string + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`a`, `b`, `c`, `d`] + + +-- !query +DESC v PARTITION (c='Us', d=1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "FORBIDDEN_OPERATION", + "sqlState" : "42809", + "messageParameters" : { + "objectName" : "`v`", + "objectType" : "VIEW", + "statement" : "DESC PARTITION" + } +} + + +-- !query +EXPLAIN DESC t +-- !query schema +struct +-- !query output +== Physical Plan == +Execute DescribeTableCommand + +- DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +EXPLAIN DESC EXTENDED t +-- !query schema +struct +-- !query output +== Physical Plan == +Execute DescribeTableCommand + +- DescribeTableCommand `spark_catalog`.`default`.`t`, true, [col_name#x, data_type#x, comment#x] + + +-- !query +EXPLAIN EXTENDED DESC t +-- !query schema +struct +-- !query output +== Parsed Logical Plan == +'DescribeRelation false, [col_name#x, data_type#x, comment#x] ++- 'UnresolvedTableOrView [t], DESCRIBE TABLE, true + +== Analyzed Logical Plan == +col_name: string, data_type: string, comment: string +DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x] + +== Optimized Logical Plan == +DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x] + +== Physical Plan == +Execute DescribeTableCommand + +- DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x] + + +-- !query +EXPLAIN DESCRIBE t b +-- !query schema +struct +-- !query output +== Physical Plan == +Execute DescribeColumnCommand + +- DescribeColumnCommand `spark_catalog`.`default`.`t`, [spark_catalog, default, t, b], false, [info_name#x, info_value#x] + + +-- !query +EXPLAIN DESCRIBE t PARTITION (c='Us', d=2) +-- !query schema +struct +-- !query output +== Physical Plan == +Execute DescribeTableCommand + +- DescribeTableCommand `spark_catalog`.`default`.`t`, [c=Us, d=2], false, [col_name#x, data_type#x, comment#x] + + +-- !query +EXPLAIN DESCRIBE EXTENDED t PARTITION (c='Us', d=2) AS JSON +-- !query schema +struct +-- !query output +== Physical Plan == +Execute DescribeRelationJsonCommand + +- DescribeRelationJsonCommand [c=Us, d=2], true, [json_metadata#x] + +- ResolvedTable V2SessionCatalog(spark_catalog), default.t, V1Table(default.t), [a#x, b#x, c#x, d#x] + + +-- !query +CREATE TABLE d (a STRING DEFAULT 'default-value', b INT DEFAULT 42) USING parquet COMMENT 'table_comment' +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC d +-- !query schema +struct +-- !query output +a string +b int + +# Column Default Values +a string 'default-value' +b int 42 + + +-- !query +DESC EXTENDED d +-- !query schema +struct +-- !query output +a string +b int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table d +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Comment table_comment +Location [not included in comparison]/{warehouse_dir}/d + +# Column Default Values +a string 'default-value' +b int 42 + + +-- !query +DESC TABLE EXTENDED d +-- !query schema +struct +-- !query output +a string +b int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table d +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Comment table_comment +Location [not included in comparison]/{warehouse_dir}/d + +# Column Default Values +a string 'default-value' +b int 42 + + +-- !query +DESC FORMATTED d +-- !query schema +struct +-- !query output +a string +b int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table d +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Comment table_comment +Location [not included in comparison]/{warehouse_dir}/d + +# Column Default Values +a string 'default-value' +b int 42 + + +-- !query +CREATE TABLE e (a STRING DEFAULT CONCAT('a\n b\n ', 'c\n d'), b INT DEFAULT 42) USING parquet COMMENT 'table_comment' +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC e +-- !query schema +struct +-- !query output +a string +b int + +# Column Default Values +a string CONCAT('a\n b\n ', 'c\n d') +b int 42 + + +-- !query +DESC EXTENDED e +-- !query schema +struct +-- !query output +a string +b int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table e +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Comment table_comment +Location [not included in comparison]/{warehouse_dir}/e + +# Column Default Values +a string CONCAT('a\n b\n ', 'c\n d') +b int 42 + + +-- !query +DESC TABLE EXTENDED e +-- !query schema +struct +-- !query output +a string +b int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table e +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Comment table_comment +Location [not included in comparison]/{warehouse_dir}/e + +# Column Default Values +a string CONCAT('a\n b\n ', 'c\n d') +b int 42 + + +-- !query +DESC FORMATTED e +-- !query schema +struct +-- !query output +a string +b int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table e +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Comment table_comment +Location [not included in comparison]/{warehouse_dir}/e + +# Column Default Values +a string CONCAT('a\n b\n ', 'c\n d') +b int 42 + + +-- !query +CREATE TABLE f USING json PARTITIONED BY (B, C) AS SELECT 'APACHE' A, CAST('SPARK' AS BINARY) B, TIMESTAMP'2018-11-17 13:33:33' C +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC FORMATTED f PARTITION (B='SPARK', C=TIMESTAMP'2018-11-17 13:33:33') +-- !query schema +struct +-- !query output +A string +B binary +C timestamp +# Partition Information +# col_name data_type comment +B binary +C timestamp + +# Detailed Partition Information +Database default +Table f +Partition Values [B=SPARK, C=2018-11-17 13:33:33] +Location [not included in comparison]/{warehouse_dir}/f/B=SPARK/C=2018-11-17 13%3A33%3A33 +Partition Parameters [numFiles=1, totalSize=15, transient_lastDdlTime=[not included in comparison]] +Created Time [not included in comparison] +Last Access [not included in comparison] + +# Storage Information +Location [not included in comparison]/{warehouse_dir}/f + + +-- !query +DESC TABLE EXTENDED f PARTITION (B='SPARK', C=TIMESTAMP'2018-11-17 13:33:33') AS JSON +-- !query schema +struct +-- !query output +{"table_name":"f","catalog_name":"spark_catalog","namespace":["default"],"schema_name":"default","columns":[{"name":"A","type":{"name":"string","collation":"UTF8_BINARY"},"nullable":true},{"name":"B","type":{"name":"binary"},"nullable":true},{"name":"C","type":{"name":"timestamp_ltz"},"nullable":true}],"partition_values":{"B":"SPARK","C":"2018-11-17 13:33:33"},"location":"file:[not included in comparison]/{warehouse_dir}/f/B=SPARK/C=2018-11-17 13%3A33%3A33","partition_parameters":{"numFiles":"1","totalSize":"15","transient_lastDdlTime [not included in comparison]":"None"},"created_time [not included in comparison]":"None","last_access [not included in comparison]":"None","created_by [not included in comparison]":"None","type":"MANAGED","provider":"json","partition_provider":"Catalog","partition_columns":["B","C"],"collation":"UTF8_BINARY"} + + +-- !query +DROP VIEW temp_v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW temp_Data_Source_View +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE t +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE d +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE e +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE f +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/double-quoted-identifiers-enabled.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/double-quoted-identifiers-enabled.sql.out new file mode 100644 index 000000000000..2444c399a87e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/double-quoted-identifiers-enabled.sql.out @@ -0,0 +1,483 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT 1 FROM "not_exist" +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 25, + "fragment" : "\"not_exist\"" + } ] +} + + +-- !query +USE SCHEMA "not_exist" +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException +{ + "errorClass" : "SCHEMA_NOT_FOUND", + "sqlState" : "42704", + "messageParameters" : { + "schemaName" : "`spark_catalog`.`not_exist`" + } +} + + +-- !query +ALTER TABLE "not_exist" ADD COLUMN not_exist int +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 23, + "fragment" : "\"not_exist\"" + } ] +} + + +-- !query +ALTER TABLE not_exist ADD COLUMN "not_exist" int +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 21, + "fragment" : "not_exist" + } ] +} + + +-- !query +SELECT 1 AS "not_exist" FROM not_exist +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 38, + "fragment" : "not_exist" + } ] +} + + +-- !query +SELECT 1 FROM not_exist AS X("hello") +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 23, + "fragment" : "not_exist" + } ] +} + + +-- !query +SELECT "not_exist"() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`not_exist`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "\"not_exist\"()" + } ] +} + + +-- !query +SELECT "not_exist".not_exist() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`not_exist`.`not_exist`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "\"not_exist\".not_exist()" + } ] +} + + +-- !query +SELECT 1 FROM `hello` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`hello`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 21, + "fragment" : "`hello`" + } ] +} + + +-- !query +USE SCHEMA `not_exist` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException +{ + "errorClass" : "SCHEMA_NOT_FOUND", + "sqlState" : "42704", + "messageParameters" : { + "schemaName" : "`spark_catalog`.`not_exist`" + } +} + + +-- !query +ALTER TABLE `not_exist` ADD COLUMN not_exist int +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 23, + "fragment" : "`not_exist`" + } ] +} + + +-- !query +ALTER TABLE not_exist ADD COLUMN `not_exist` int +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 21, + "fragment" : "not_exist" + } ] +} + + +-- !query +SELECT 1 AS `not_exist` FROM `not_exist` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 40, + "fragment" : "`not_exist`" + } ] +} + + +-- !query +SELECT 1 FROM not_exist AS X(`hello`) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 23, + "fragment" : "not_exist" + } ] +} + + +-- !query +SELECT `not_exist`() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`not_exist`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "`not_exist`()" + } ] +} + + +-- !query +SELECT `not_exist`.not_exist() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`not_exist`.`not_exist`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "`not_exist`.not_exist()" + } ] +} + + +-- !query +SELECT "hello" +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`hello`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 14, + "fragment" : "\"hello\"" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW v(c1 COMMENT "hello") AS SELECT 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"hello\"'", + "hint" : "" + } +} + + +-- !query +DROP VIEW v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`spark_catalog`.`default`.`v`" + } +} + + +-- !query +SELECT INTERVAL "1" YEAR +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"1\"'", + "hint" : "" + } +} + + +-- !query +SELECT 'hello' +-- !query schema +struct +-- !query output +hello + + +-- !query +CREATE TEMPORARY VIEW v(c1 COMMENT 'hello') AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW v +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT INTERVAL '1' YEAR +-- !query schema +struct +-- !query output +1-0 + + +-- !query +CREATE SCHEMA "myschema" +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW "myview"("c1") AS + WITH "v"("a") AS (SELECT 1) SELECT "a" FROM "v" +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT "a1" AS "a2" FROM "myview" AS "atab"("a1") +-- !query schema +struct +-- !query output +1 + + +-- !query +DROP TABLE "myview" +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP SCHEMA "myschema" +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/double-quoted-identifiers.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/double-quoted-identifiers.sql.out new file mode 100644 index 000000000000..81a98a60590f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/double-quoted-identifiers.sql.out @@ -0,0 +1,443 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT 1 FROM "not_exist" +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +USE SCHEMA "not_exist" +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +ALTER TABLE "not_exist" ADD COLUMN not_exist int +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +ALTER TABLE not_exist ADD COLUMN "not_exist" int +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +SELECT 1 AS "not_exist" FROM not_exist +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +SELECT 1 FROM not_exist AS X("hello") +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"hello\"'", + "hint" : "" + } +} + + +-- !query +SELECT "not_exist"() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +SELECT "not_exist".not_exist() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +SELECT 1 FROM `hello` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`hello`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 21, + "fragment" : "`hello`" + } ] +} + + +-- !query +USE SCHEMA `not_exist` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException +{ + "errorClass" : "SCHEMA_NOT_FOUND", + "sqlState" : "42704", + "messageParameters" : { + "schemaName" : "`spark_catalog`.`not_exist`" + } +} + + +-- !query +ALTER TABLE `not_exist` ADD COLUMN not_exist int +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 23, + "fragment" : "`not_exist`" + } ] +} + + +-- !query +ALTER TABLE not_exist ADD COLUMN `not_exist` int +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 21, + "fragment" : "not_exist" + } ] +} + + +-- !query +SELECT 1 AS `not_exist` FROM `not_exist` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 40, + "fragment" : "`not_exist`" + } ] +} + + +-- !query +SELECT 1 FROM not_exist AS X(`hello`) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 23, + "fragment" : "not_exist" + } ] +} + + +-- !query +SELECT `not_exist`() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`not_exist`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "`not_exist`()" + } ] +} + + +-- !query +SELECT `not_exist`.not_exist() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`not_exist`.`not_exist`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "`not_exist`.not_exist()" + } ] +} + + +-- !query +SELECT "hello" +-- !query schema +struct +-- !query output +hello + + +-- !query +CREATE TEMPORARY VIEW v(c1 COMMENT "hello") AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW v +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT INTERVAL "1" YEAR +-- !query schema +struct +-- !query output +1-0 + + +-- !query +SELECT 'hello' +-- !query schema +struct +-- !query output +hello + + +-- !query +CREATE TEMPORARY VIEW v(c1 COMMENT 'hello') AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW v +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT INTERVAL '1' YEAR +-- !query schema +struct +-- !query output +1-0 + + +-- !query +CREATE SCHEMA "myschema" +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"myschema\"'", + "hint" : "" + } +} + + +-- !query +CREATE TEMPORARY VIEW "myview"("c1") AS + WITH "v"("a") AS (SELECT 1) SELECT "a" FROM "v" +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"myview\"'", + "hint" : "" + } +} + + +-- !query +SELECT "a1" AS "a2" FROM "myview" AS "atab"("a1") +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"a2\"'", + "hint" : "" + } +} + + +-- !query +DROP TABLE "myview" +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"myview\"'", + "hint" : "" + } +} + + +-- !query +DROP SCHEMA "myschema" +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"myschema\"'", + "hint" : "" + } +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/except-all.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/except-all.sql.out new file mode 100644 index 000000000000..437a5553d385 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/except-all.sql.out @@ -0,0 +1,377 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (0), (1), (2), (2), (2), (2), (3), (null), (null) AS tab1(c1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1), (2), (2), (3), (5), (5), (null) AS tab2(c1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW tab3 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (2, 3), + (2, 2) + AS tab3(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW tab4 AS SELECT * FROM VALUES + (1, 2), + (2, 3), + (2, 2), + (2, 2), + (2, 20) + AS tab4(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM tab1 +EXCEPT ALL +SELECT * FROM tab2 +-- !query schema +struct +-- !query output +0 +2 +2 +NULL + + +-- !query +SELECT * FROM tab1 +MINUS ALL +SELECT * FROM tab2 +-- !query schema +struct +-- !query output +0 +2 +2 +NULL + + +-- !query +SELECT * FROM tab1 +EXCEPT ALL +SELECT * FROM tab2 WHERE c1 IS NOT NULL +-- !query schema +struct +-- !query output +0 +2 +2 +NULL +NULL + + +-- !query +SELECT * FROM tab1 WHERE c1 > 5 +EXCEPT ALL +SELECT * FROM tab2 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM tab1 +EXCEPT ALL +SELECT * FROM tab2 WHERE c1 > 6 +-- !query schema +struct +-- !query output +0 +1 +2 +2 +2 +2 +3 +NULL +NULL + + +-- !query +SELECT * FROM tab1 +EXCEPT ALL +SELECT CAST(1 AS BIGINT) +-- !query schema +struct +-- !query output +0 +2 +2 +2 +2 +3 +NULL +NULL + + +-- !query +SELECT * FROM tab1 +EXCEPT ALL +SELECT array(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"ARRAY\"", + "dataType2" : "\"INT\"", + "hint" : "", + "operator" : "EXCEPT ALL", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 45, + "fragment" : "SELECT * FROM tab1\nEXCEPT ALL\nSELECT array(1)" + } ] +} + + +-- !query +SELECT * FROM tab3 +EXCEPT ALL +SELECT * FROM tab4 +-- !query schema +struct +-- !query output +1 2 +1 3 + + +-- !query +SELECT * FROM tab4 +EXCEPT ALL +SELECT * FROM tab3 +-- !query schema +struct +-- !query output +2 2 +2 20 + + +-- !query +SELECT * FROM tab4 +EXCEPT ALL +SELECT * FROM tab3 +INTERSECT DISTINCT +SELECT * FROM tab4 +-- !query schema +struct +-- !query output +2 2 +2 20 + + +-- !query +SELECT * FROM tab4 +EXCEPT ALL +SELECT * FROM tab3 +EXCEPT DISTINCT +SELECT * FROM tab4 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM tab3 +EXCEPT ALL +SELECT * FROM tab4 +UNION ALL +SELECT * FROM tab3 +EXCEPT DISTINCT +SELECT * FROM tab4 +-- !query schema +struct +-- !query output +1 3 + + +-- !query +SELECT k FROM tab3 +EXCEPT ALL +SELECT k, v FROM tab4 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NUM_COLUMNS_MISMATCH", + "sqlState" : "42826", + "messageParameters" : { + "firstNumColumns" : "1", + "invalidNumColumns" : "2", + "invalidOrdinalNum" : "second", + "operator" : "EXCEPT ALL" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 51, + "fragment" : "SELECT k FROM tab3\nEXCEPT ALL\nSELECT k, v FROM tab4" + } ] +} + + +-- !query +SELECT * FROM tab3 +EXCEPT ALL +SELECT * FROM tab4 +UNION +SELECT * FROM tab3 +EXCEPT DISTINCT +SELECT * FROM tab4 +-- !query schema +struct +-- !query output +1 3 + + +-- !query +SELECT * FROM tab3 +MINUS ALL +SELECT * FROM tab4 +UNION +SELECT * FROM tab3 +MINUS DISTINCT +SELECT * FROM tab4 +-- !query schema +struct +-- !query output +1 3 + + +-- !query +SELECT * FROM tab3 +EXCEPT ALL +SELECT * FROM tab4 +EXCEPT DISTINCT +SELECT * FROM tab3 +EXCEPT DISTINCT +SELECT * FROM tab4 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM (SELECT tab3.k, + tab4.v + FROM tab3 + JOIN tab4 + ON tab3.k = tab4.k) +EXCEPT ALL +SELECT * +FROM (SELECT tab3.k, + tab4.v + FROM tab3 + JOIN tab4 + ON tab3.k = tab4.k) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM (SELECT tab3.k, + tab4.v + FROM tab3 + JOIN tab4 + ON tab3.k = tab4.k) +EXCEPT ALL +SELECT * +FROM (SELECT tab4.v AS k, + tab3.k AS v + FROM tab3 + JOIN tab4 + ON tab3.k = tab4.k) +-- !query schema +struct +-- !query output +1 2 +1 2 +1 2 +2 20 +2 20 +2 3 +2 3 + + +-- !query +SELECT v FROM tab3 GROUP BY v +EXCEPT ALL +SELECT k FROM tab4 GROUP BY k +-- !query schema +struct +-- !query output +3 + + +-- !query +DROP VIEW IF EXISTS tab1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS tab2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS tab3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS tab4 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/except.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/except.sql.out new file mode 100644 index 000000000000..db5be40ad571 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/except.sql.out @@ -0,0 +1,158 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", NULL) + as t1(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5), + ("one", NULL), + (NULL, 5) + as t2(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM t1 EXCEPT SELECT * FROM t2 +-- !query schema +struct +-- !query output +three 3 +two 2 + + +-- !query +SELECT * FROM t1 EXCEPT SELECT * FROM t1 where v <> 1 and v <> 2 +-- !query schema +struct +-- !query output +one 1 +one NULL +two 2 + + +-- !query +SELECT * FROM t1 where v <> 1 and v <> 22 EXCEPT SELECT * FROM t1 where v <> 2 and v >= 3 +-- !query schema +struct +-- !query output +two 2 + + +-- !query +SELECT t1.* FROM t1, t2 where t1.k = t2.k +EXCEPT +SELECT t1.* FROM t1, t2 where t1.k = t2.k and t1.k != 'one' +-- !query schema +struct +-- !query output +one 1 +one NULL + + +-- !query +SELECT * FROM t2 where v >= 1 and v <> 22 EXCEPT SELECT * FROM t1 +-- !query schema +struct +-- !query output +NULL 5 +one 5 + + +-- !query +SELECT (SELECT min(k) FROM t2 WHERE t2.k = t1.k) min_t2 FROM t1 +MINUS +SELECT (SELECT min(k) FROM t2) abs_min_t2 FROM t1 WHERE t1.k = 'one' +-- !query schema +struct +-- !query output +NULL +two + + +-- !query +SELECT t1.k +FROM t1 +WHERE t1.v <= (SELECT max(t2.v) + FROM t2 + WHERE t2.k = t1.k) +MINUS +SELECT t1.k +FROM t1 +WHERE t1.v >= (SELECT min(t2.v) + FROM t2 + WHERE t2.k = t1.k) +-- !query schema +struct +-- !query output +two + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t3 AS VALUES (decimal(1)) tbl(v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t.v FROM ( + SELECT v FROM t3 + EXCEPT + SELECT v + v AS v FROM t3 +) t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT SUM(t.v) FROM ( + SELECT v FROM t3 + EXCEPT + SELECT v + v AS v FROM t3 +) t +-- !query schema +struct +-- !query output +1 + + +-- !query +DROP VIEW IF EXISTS t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS t3 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/execute-immediate.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/execute-immediate.sql.out new file mode 100644 index 000000000000..9249d7eb3e51 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/execute-immediate.sql.out @@ -0,0 +1,792 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES + (10, 'name1', named_struct('f1', 1, 's2', named_struct('f2', 101, 'f3', 'a'))), + (20, 'name2', named_struct('f1', 2, 's2', named_struct('f2', 202, 'f3', 'b'))), + (30, 'name3', named_struct('f1', 3, 's2', named_struct('f2', 303, 'f3', 'c'))), + (40, 'name4', named_struct('f1', 4, 's2', named_struct('f2', 404, 'f3', 'd'))), + (50, 'name5', named_struct('f1', 5, 's2', named_struct('f2', 505, 'f3', 'e'))), + (60, 'name6', named_struct('f1', 6, 's2', named_struct('f2', 606, 'f3', 'f'))), + (70, 'name7', named_struct('f1', 7, 's2', named_struct('f2', 707, 'f3', 'g'))) +AS tbl_view(id, name, data) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE x (id INT) USING csv +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE sql_string STRING +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VAR sql_string = 'SELECT * from tbl_view where name = \'name1\'' +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SET spark.sql.ansi.enabled=true' +-- !query schema +struct +-- !query output +spark.sql.ansi.enabled true + + +-- !query +EXECUTE IMMEDIATE 'CREATE TEMPORARY VIEW IDENTIFIER(:tblName) AS SELECT id, name FROM tbl_view' USING 'tbl_view_tmp' as tblName +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM tbl_view_tmp' +-- !query schema +struct +-- !query output +10 name1 +20 name2 +30 name3 +40 name4 +50 name5 +60 name6 +70 name7 + + +-- !query +EXECUTE IMMEDIATE 'REFRESH TABLE IDENTIFIER(:tblName)' USING 'x' as tblName +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE sql_string +-- !query schema +struct>> +-- !query output +10 name1 {"f1":1,"s2":{"f2":101,"f3":"a"}} + + +-- !query +EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = \'name1\'' +-- !query schema +struct>> +-- !query output +10 name1 {"f1":1,"s2":{"f2":101,"f3":"a"}} + + +-- !query +SET VAR sql_string = 'SELECT * from tbl_view where name = ? or name = ?' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE a STRING +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VAR a = 'name1' +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE sql_string USING 'name1', 'name3' +-- !query schema +struct>> +-- !query output +10 name1 {"f1":1,"s2":{"f2":101,"f3":"a"}} +30 name3 {"f1":3,"s2":{"f2":303,"f3":"c"}} + + +-- !query +EXECUTE IMMEDIATE sql_string USING a, 'name2' +-- !query schema +struct>> +-- !query output +10 name1 {"f1":1,"s2":{"f2":101,"f3":"a"}} +20 name2 {"f1":2,"s2":{"f2":202,"f3":"b"}} + + +-- !query +EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING 'name1', 'name3' +-- !query schema +struct>> +-- !query output +10 name1 {"f1":1,"s2":{"f2":101,"f3":"a"}} +30 name3 {"f1":3,"s2":{"f2":303,"f3":"c"}} + + +-- !query +EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING a, 'name2' +-- !query schema +struct>> +-- !query output +10 name1 {"f1":1,"s2":{"f2":101,"f3":"a"}} +20 name2 {"f1":2,"s2":{"f2":202,"f3":"b"}} + + +-- !query +EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING (a, 'name2') +-- !query schema +struct>> +-- !query output +10 name1 {"f1":1,"s2":{"f2":101,"f3":"a"}} +20 name2 {"f1":2,"s2":{"f2":202,"f3":"b"}} + + +-- !query +EXECUTE IMMEDIATE 'INSERT INTO x VALUES(?)' USING 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * from x +-- !query schema +struct +-- !query output +1 + + +-- !query +SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE b INT +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VAR b = 40 +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE sql_string USING 40 as second, 'name7' as first +-- !query schema +struct>> +-- !query output +40 name4 {"f1":4,"s2":{"f2":404,"f3":"d"}} +70 name7 {"f1":7,"s2":{"f2":707,"f3":"g"}} + + +-- !query +EXECUTE IMMEDIATE sql_string USING b as second, 'name7' as first +-- !query schema +struct>> +-- !query output +40 name4 {"f1":4,"s2":{"f2":404,"f3":"d"}} +70 name7 {"f1":7,"s2":{"f2":707,"f3":"g"}} + + +-- !query +EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 40 as second, 'name7' as first +-- !query schema +struct>> +-- !query output +40 name4 {"f1":4,"s2":{"f2":404,"f3":"d"}} +70 name7 {"f1":7,"s2":{"f2":707,"f3":"g"}} + + +-- !query +EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 'name7' as first, b as second +-- !query schema +struct>> +-- !query output +40 name4 {"f1":4,"s2":{"f2":404,"f3":"d"}} +70 name7 {"f1":7,"s2":{"f2":707,"f3":"g"}} + + +-- !query +EXECUTE IMMEDIATE 'SELECT tbl_view.*, :first as p FROM tbl_view WHERE name = :first' USING 'name7' as first +-- !query schema +struct>,p:string> +-- !query output +70 name7 {"f1":7,"s2":{"f2":707,"f3":"g"}} name7 + + +-- !query +EXECUTE IMMEDIATE 'SET VAR sql_string = ?' USING 'SELECT id from tbl_view where name = :first' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT sql_string +-- !query schema +struct +-- !query output +SELECT id from tbl_view where name = :first + + +-- !query +DECLARE res_id INT +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE sql_string INTO res_id USING 'name7' as first +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT res_id +-- !query schema +struct +-- !query output +70 + + +-- !query +EXECUTE IMMEDIATE sql_string INTO res_id USING a as first +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT res_id +-- !query schema +struct +-- !query output +10 + + +-- !query +SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second' +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT 42' INTO res_id +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT res_id +-- !query schema +struct +-- !query output +42 + + +-- !query +EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO b, a USING 10 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT b, a +-- !query schema +struct +-- !query output +10 name1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where id = ? AND name = ?' USING b as first, a +-- !query schema +struct>> +-- !query output +10 name1 {"f1":1,"s2":{"f2":101,"f3":"a"}} + + +-- !query +EXECUTE IMMEDIATE 'SELECT 42 WHERE 2 = 1' INTO res_id +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT res_id +-- !query schema +struct +-- !query output +NULL + + +-- !query +EXECUTE IMMEDIATE 'SELECT \'1707\'' INTO res_id +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT res_id +-- !query schema +struct +-- !query output +1707 + + +-- !query +EXECUTE IMMEDIATE 'SELECT \'invalid_cast_error_expected\'' INTO res_id +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'invalid_cast_error_expected'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 70, + "fragment" : "EXECUTE IMMEDIATE 'SELECT \\'invalid_cast_error_expected\\'' INTO res_id" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'INSERT INTO x VALUES (?)' INTO res_id USING 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_STATEMENT_FOR_EXECUTE_INTO", + "sqlState" : "07501", + "messageParameters" : { + "sqlString" : "INSERT INTO X VALUES (?)" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM tbl_view WHERE ? = id' USING id +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_VARIABLE", + "sqlState" : "42883", + "messageParameters" : { + "searchPath" : "`system`.`session`", + "variableName" : "`id`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 63, + "stopIndex" : 64, + "fragment" : "id" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where ? = id and :first = name' USING 1, 'name2' as first +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_QUERY_MIXED_QUERY_PARAMETERS", + "sqlState" : "42613" +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where :first = name' USING 1, 'name2' as first +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "ALL_PARAMETERS_MUST_BE_NAMED", + "sqlState" : "07001", + "messageParameters" : { + "exprs" : "\"1\"" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELCT Fa' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'SELCT'", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 28, + "fragment" : "EXECUTE IMMEDIATE 'SELCT Fa'" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELCT Fa' INTO res_id +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'SELCT'", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 40, + "fragment" : "EXECUTE IMMEDIATE 'SELCT Fa' INTO res_id" + } ] +} + + +-- !query +EXECUTE IMMEDIATE b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", + "sqlState" : "42K09", + "messageParameters" : { + "varType" : "\"INT\"" + } +} + + +-- !query +SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second' +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VAR a = 'na' +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first' USING CONCAT(a , "me1") as first +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_EXPR_FOR_PARAMETER", + "sqlState" : "42K0E", + "messageParameters" : { + "invalidExprSql" : "\"CONCAT(a, me1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 70, + "stopIndex" : 86, + "fragment" : "CONCAT(a , \"me1\")" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first' USING (SELECT 42) as first, 'name2' as second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_EXPR_FOR_PARAMETER", + "sqlState" : "42K0E", + "messageParameters" : { + "invalidExprSql" : "\"scalarsubquery()\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 70, + "stopIndex" : 80, + "fragment" : "(SELECT 42)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO a, b USING 10 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'name1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 81, + "fragment" : "EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO a, b USING 10" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO (a, b) USING 10 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT id FROM tbl_view' INTO res_id +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +{ + "errorClass" : "ROW_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000" +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT id, data.f1 FROM tbl_view' INTO res_id +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "ASSIGNMENT_ARITY_MISMATCH", + "sqlState" : "42802", + "messageParameters" : { + "numExpr" : "2", + "numTarget" : "1" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT id FROM tbl_view' INTO res_id, b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "ASSIGNMENT_ARITY_MISMATCH", + "sqlState" : "42802", + "messageParameters" : { + "numExpr" : "1", + "numTarget" : "2" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT id FROM tbl_view WHERE id = :first' USING 10 as first, 20 as first +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "EXEC_IMMEDIATE_DUPLICATE_ARGUMENT_ALIASES", + "sqlState" : "42701", + "messageParameters" : { + "aliases" : "`first`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 63, + "stopIndex" : 92, + "fragment" : "USING 10 as first, 20 as first" + } ] +} + + +-- !query +DECLARE p = 10 +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT id FROM tbl_view WHERE id = :p' USING p +-- !query schema +struct +-- !query output +10 + + +-- !query +EXECUTE IMMEDIATE 'SELECT id FROM tbl_view WHERE id = :p' USING p, 'p' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "ALL_PARAMETERS_MUST_BE_NAMED", + "sqlState" : "07001", + "messageParameters" : { + "exprs" : "\"p\"" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT id, data.f1 FROM tbl_view WHERE id = 10' INTO res_id, res_id +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DUPLICATE_ASSIGNMENTS", + "sqlState" : "42701", + "messageParameters" : { + "nameList" : "`res_id`" + } +} + + +-- !query +EXECUTE IMMEDIATE 'EXECUTE IMMEDIATE \'SELECT id FROM tbl_view WHERE id = ? USING 10\'' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NESTED_EXECUTE_IMMEDIATE", + "sqlState" : "07501", + "messageParameters" : { + "sqlString" : "EXECUTE IMMEDIATE 'SELECT ID FROM TBL_VIEW WHERE ID = ? USING 10'" + } +} + + +-- !query +SET VAR sql_string = null +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE sql_string +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NULL_QUERY_STRING_EXECUTE_IMMEDIATE", + "sqlState" : "22004", + "messageParameters" : { + "varName" : "`sql_string`" + } +} + + +-- !query +DROP TABLE x +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/explain-aqe.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/explain-aqe.sql.out new file mode 100644 index 000000000000..16077a78f389 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/explain-aqe.sql.out @@ -0,0 +1,1232 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE table explain_temp1 (key int, val int) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE table explain_temp2 (key int, val int) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE table explain_temp3 (key int, val int) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE table explain_temp4 (key int, val string) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE table explain_temp5 (key int) USING PARQUET PARTITIONED BY(val string) +-- !query schema +struct<> +-- !query output + + + +-- !query +SET spark.sql.codegen.wholeStage = true +-- !query schema +struct +-- !query output +spark.sql.codegen.wholeStage true + + +-- !query +EXPLAIN EXTENDED + SELECT sum(distinct val) + FROM explain_temp1 +-- !query schema +struct +-- !query output +== Parsed Logical Plan == +'Project [unresolvedalias('sum(distinct 'val))] ++- 'UnresolvedRelation [explain_temp1], [], false + +== Analyzed Logical Plan == +sum(DISTINCT val): bigint +Aggregate [sum(distinct val#x) AS sum(DISTINCT val)#xL] ++- SubqueryAlias spark_catalog.default.explain_temp1 + +- Relation spark_catalog.default.explain_temp1[key#x,val#x] parquet + +== Optimized Logical Plan == +Aggregate [sum(distinct val#x) AS sum(DISTINCT val)#xL] ++- Project [val#x] + +- Relation spark_catalog.default.explain_temp1[key#x,val#x] parquet + +== Physical Plan == +AdaptiveSparkPlan isFinalPlan=false ++- HashAggregate(keys=[], functions=[sum(distinct val#x)], output=[sum(DISTINCT val)#xL]) + +- Exchange SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +- HashAggregate(keys=[], functions=[partial_sum(distinct val#x)], output=[sum#xL]) + +- HashAggregate(keys=[val#x], functions=[], output=[val#x]) + +- Exchange hashpartitioning(val#x, 4), ENSURE_REQUIREMENTS, [plan_id=x] + +- HashAggregate(keys=[val#x], functions=[], output=[val#x]) + +- FileScan parquet spark_catalog.default.explain_temp1[val#x] Batched: true, DataFilters: [], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp1], PartitionFilters: [], PushedFilters: [], ReadSchema: struct + + +-- !query +EXPLAIN FORMATTED + SELECT key, max(val) + FROM explain_temp1 + WHERE key > 0 + GROUP BY key + ORDER BY key +-- !query schema +struct +-- !query output +== Physical Plan == +AdaptiveSparkPlan (8) ++- Sort (7) + +- Exchange (6) + +- HashAggregate (5) + +- Exchange (4) + +- HashAggregate (3) + +- Filter (2) + +- Scan parquet spark_catalog.default.explain_temp1 (1) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,0)] +ReadSchema: struct + +(2) Filter +Input [2]: [key#x, val#x] +Condition : (isnotnull(key#x) AND (key#x > 0)) + +(3) HashAggregate +Input [2]: [key#x, val#x] +Keys [1]: [key#x] +Functions [1]: [partial_max(val#x)] +Aggregate Attributes [1]: [max#x] +Results [2]: [key#x, max#x] + +(4) Exchange +Input [2]: [key#x, max#x] +Arguments: hashpartitioning(key#x, 4), ENSURE_REQUIREMENTS, [plan_id=x] + +(5) HashAggregate +Input [2]: [key#x, max#x] +Keys [1]: [key#x] +Functions [1]: [max(val#x)] +Aggregate Attributes [1]: [max(val#x)#x] +Results [2]: [key#x, max(val#x)#x AS max(val)#x] + +(6) Exchange +Input [2]: [key#x, max(val)#x] +Arguments: rangepartitioning(key#x ASC NULLS FIRST, 4), ENSURE_REQUIREMENTS, [plan_id=x] + +(7) Sort +Input [2]: [key#x, max(val)#x] +Arguments: [key#x ASC NULLS FIRST], true, 0 + +(8) AdaptiveSparkPlan +Output [2]: [key#x, max(val)#x] +Arguments: isFinalPlan=false + + +-- !query +EXPLAIN FORMATTED + SELECT key, max(val) + FROM explain_temp1 + WHERE key > 0 + GROUP BY key + HAVING max(val) > 0 +-- !query schema +struct +-- !query output +== Physical Plan == +AdaptiveSparkPlan (7) ++- Filter (6) + +- HashAggregate (5) + +- Exchange (4) + +- HashAggregate (3) + +- Filter (2) + +- Scan parquet spark_catalog.default.explain_temp1 (1) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,0)] +ReadSchema: struct + +(2) Filter +Input [2]: [key#x, val#x] +Condition : (isnotnull(key#x) AND (key#x > 0)) + +(3) HashAggregate +Input [2]: [key#x, val#x] +Keys [1]: [key#x] +Functions [1]: [partial_max(val#x)] +Aggregate Attributes [1]: [max#x] +Results [2]: [key#x, max#x] + +(4) Exchange +Input [2]: [key#x, max#x] +Arguments: hashpartitioning(key#x, 4), ENSURE_REQUIREMENTS, [plan_id=x] + +(5) HashAggregate +Input [2]: [key#x, max#x] +Keys [1]: [key#x] +Functions [1]: [max(val#x)] +Aggregate Attributes [1]: [max(val#x)#x] +Results [2]: [key#x, max(val#x)#x AS max(val)#x] + +(6) Filter +Input [2]: [key#x, max(val)#x] +Condition : (isnotnull(max(val)#x) AND (max(val)#x > 0)) + +(7) AdaptiveSparkPlan +Output [2]: [key#x, max(val)#x] +Arguments: isFinalPlan=false + + +-- !query +EXPLAIN FORMATTED + SELECT key, val FROM explain_temp1 WHERE key > 0 + UNION + SELECT key, val FROM explain_temp1 WHERE key > 1 +-- !query schema +struct +-- !query output +== Physical Plan == +AdaptiveSparkPlan (9) ++- HashAggregate (8) + +- Exchange (7) + +- HashAggregate (6) + +- Union (5) + :- Filter (2) + : +- Scan parquet spark_catalog.default.explain_temp1 (1) + +- Filter (4) + +- Scan parquet spark_catalog.default.explain_temp1 (3) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,0)] +ReadSchema: struct + +(2) Filter +Input [2]: [key#x, val#x] +Condition : (isnotnull(key#x) AND (key#x > 0)) + +(3) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,1)] +ReadSchema: struct + +(4) Filter +Input [2]: [key#x, val#x] +Condition : (isnotnull(key#x) AND (key#x > 1)) + +(5) Union + +(6) HashAggregate +Input [2]: [key#x, val#x] +Keys [2]: [key#x, val#x] +Functions: [] +Aggregate Attributes: [] +Results [2]: [key#x, val#x] + +(7) Exchange +Input [2]: [key#x, val#x] +Arguments: hashpartitioning(key#x, val#x, 4), ENSURE_REQUIREMENTS, [plan_id=x] + +(8) HashAggregate +Input [2]: [key#x, val#x] +Keys [2]: [key#x, val#x] +Functions: [] +Aggregate Attributes: [] +Results [2]: [key#x, val#x] + +(9) AdaptiveSparkPlan +Output [2]: [key#x, val#x] +Arguments: isFinalPlan=false + + +-- !query +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 a, + explain_temp2 b + WHERE a.key = b.key +-- !query schema +struct +-- !query output +== Physical Plan == +AdaptiveSparkPlan (7) ++- BroadcastHashJoin Inner BuildRight (6) + :- Filter (2) + : +- Scan parquet spark_catalog.default.explain_temp1 (1) + +- BroadcastExchange (5) + +- Filter (4) + +- Scan parquet spark_catalog.default.explain_temp2 (3) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key)] +ReadSchema: struct + +(2) Filter +Input [2]: [key#x, val#x] +Condition : isnotnull(key#x) + +(3) Scan parquet spark_catalog.default.explain_temp2 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp2] +PushedFilters: [IsNotNull(key)] +ReadSchema: struct + +(4) Filter +Input [2]: [key#x, val#x] +Condition : isnotnull(key#x) + +(5) BroadcastExchange +Input [2]: [key#x, val#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=x] + +(6) BroadcastHashJoin +Left keys [1]: [key#x] +Right keys [1]: [key#x] +Join type: Inner +Join condition: None + +(7) AdaptiveSparkPlan +Output [4]: [key#x, val#x, key#x, val#x] +Arguments: isFinalPlan=false + + +-- !query +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 a + LEFT OUTER JOIN explain_temp2 b + ON a.key = b.key +-- !query schema +struct +-- !query output +== Physical Plan == +AdaptiveSparkPlan (6) ++- BroadcastHashJoin LeftOuter BuildRight (5) + :- Scan parquet spark_catalog.default.explain_temp1 (1) + +- BroadcastExchange (4) + +- Filter (3) + +- Scan parquet spark_catalog.default.explain_temp2 (2) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct + +(2) Scan parquet spark_catalog.default.explain_temp2 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp2] +PushedFilters: [IsNotNull(key)] +ReadSchema: struct + +(3) Filter +Input [2]: [key#x, val#x] +Condition : isnotnull(key#x) + +(4) BroadcastExchange +Input [2]: [key#x, val#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=x] + +(5) BroadcastHashJoin +Left keys [1]: [key#x] +Right keys [1]: [key#x] +Join type: LeftOuter +Join condition: None + +(6) AdaptiveSparkPlan +Output [4]: [key#x, val#x, key#x, val#x] +Arguments: isFinalPlan=false + + +-- !query +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 + WHERE key = (SELECT max(key) + FROM explain_temp2 + WHERE key = (SELECT max(key) + FROM explain_temp3 + WHERE val > 0) + AND val = 2) + AND val > 3 +-- !query schema +struct +-- !query output +== Physical Plan == +AdaptiveSparkPlan (3) ++- Filter (2) + +- Scan parquet spark_catalog.default.explain_temp1 (1) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), IsNotNull(val), EqualTo(key,ScalarSubquery#x), GreaterThan(val,3)] +ReadSchema: struct + +(2) Filter +Input [2]: [key#x, val#x] +Condition : (((isnotnull(key#x) AND isnotnull(val#x)) AND (key#x = Subquery subquery#x, [id=#x])) AND (val#x > 3)) + +(3) AdaptiveSparkPlan +Output [2]: [key#x, val#x] +Arguments: isFinalPlan=false + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 2 Hosting Expression = Subquery subquery#x, [id=#x] +AdaptiveSparkPlan (10) ++- HashAggregate (9) + +- Exchange (8) + +- HashAggregate (7) + +- Project (6) + +- Filter (5) + +- Scan parquet spark_catalog.default.explain_temp2 (4) + + +(4) Scan parquet spark_catalog.default.explain_temp2 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp2] +PushedFilters: [IsNotNull(key), IsNotNull(val), EqualTo(key,ScalarSubquery#x), EqualTo(val,2)] +ReadSchema: struct + +(5) Filter +Input [2]: [key#x, val#x] +Condition : (((isnotnull(key#x) AND isnotnull(val#x)) AND (key#x = Subquery subquery#x, [id=#x])) AND (val#x = 2)) + +(6) Project +Output [1]: [key#x] +Input [2]: [key#x, val#x] + +(7) HashAggregate +Input [1]: [key#x] +Keys: [] +Functions [1]: [partial_max(key#x)] +Aggregate Attributes [1]: [max#x] +Results [1]: [max#x] + +(8) Exchange +Input [1]: [max#x] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +(9) HashAggregate +Input [1]: [max#x] +Keys: [] +Functions [1]: [max(key#x)] +Aggregate Attributes [1]: [max(key#x)#x] +Results [1]: [max(key#x)#x AS max(key)#x] + +(10) AdaptiveSparkPlan +Output [1]: [max(key)#x] +Arguments: isFinalPlan=false + +Subquery:2 Hosting operator id = 5 Hosting Expression = Subquery subquery#x, [id=#x] +AdaptiveSparkPlan (17) ++- HashAggregate (16) + +- Exchange (15) + +- HashAggregate (14) + +- Project (13) + +- Filter (12) + +- Scan parquet spark_catalog.default.explain_temp3 (11) + + +(11) Scan parquet spark_catalog.default.explain_temp3 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp3] +PushedFilters: [IsNotNull(val), GreaterThan(val,0)] +ReadSchema: struct + +(12) Filter +Input [2]: [key#x, val#x] +Condition : (isnotnull(val#x) AND (val#x > 0)) + +(13) Project +Output [1]: [key#x] +Input [2]: [key#x, val#x] + +(14) HashAggregate +Input [1]: [key#x] +Keys: [] +Functions [1]: [partial_max(key#x)] +Aggregate Attributes [1]: [max#x] +Results [1]: [max#x] + +(15) Exchange +Input [1]: [max#x] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +(16) HashAggregate +Input [1]: [max#x] +Keys: [] +Functions [1]: [max(key#x)] +Aggregate Attributes [1]: [max(key#x)#x] +Results [1]: [max(key#x)#x AS max(key)#x] + +(17) AdaptiveSparkPlan +Output [1]: [max(key)#x] +Arguments: isFinalPlan=false + +Subquery:3 Hosting operator id = 4 Hosting Expression = Subquery subquery#x, [id=#x] +AdaptiveSparkPlan (17) ++- HashAggregate (16) + +- Exchange (15) + +- HashAggregate (14) + +- Project (13) + +- Filter (12) + +- Scan parquet spark_catalog.default.explain_temp3 (11) + + +Subquery:4 Hosting operator id = 1 Hosting Expression = Subquery subquery#x, [id=#x] +AdaptiveSparkPlan (10) ++- HashAggregate (9) + +- Exchange (8) + +- HashAggregate (7) + +- Project (6) + +- Filter (5) + +- Scan parquet spark_catalog.default.explain_temp2 (4) + + +Subquery:5 Hosting operator id = 5 Hosting Expression = Subquery subquery#x, [id=#x] +AdaptiveSparkPlan (17) ++- HashAggregate (16) + +- Exchange (15) + +- HashAggregate (14) + +- Project (13) + +- Filter (12) + +- Scan parquet spark_catalog.default.explain_temp3 (11) + + +Subquery:6 Hosting operator id = 4 Hosting Expression = Subquery subquery#x, [id=#x] +AdaptiveSparkPlan (17) ++- HashAggregate (16) + +- Exchange (15) + +- HashAggregate (14) + +- Project (13) + +- Filter (12) + +- Scan parquet spark_catalog.default.explain_temp3 (11) + + +-- !query +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 + WHERE key = (SELECT max(key) + FROM explain_temp2 + WHERE val > 0) + OR + key = (SELECT avg(key) + FROM explain_temp3 + WHERE val > 0) +-- !query schema +struct +-- !query output +== Physical Plan == +AdaptiveSparkPlan (3) ++- Filter (2) + +- Scan parquet spark_catalog.default.explain_temp1 (1) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct + +(2) Filter +Input [2]: [key#x, val#x] +Condition : ((key#x = Subquery subquery#x, [id=#x]) OR (cast(key#x as double) = Subquery subquery#x, [id=#x])) + +(3) AdaptiveSparkPlan +Output [2]: [key#x, val#x] +Arguments: isFinalPlan=false + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 2 Hosting Expression = Subquery subquery#x, [id=#x] +AdaptiveSparkPlan (10) ++- HashAggregate (9) + +- Exchange (8) + +- HashAggregate (7) + +- Project (6) + +- Filter (5) + +- Scan parquet spark_catalog.default.explain_temp2 (4) + + +(4) Scan parquet spark_catalog.default.explain_temp2 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp2] +PushedFilters: [IsNotNull(val), GreaterThan(val,0)] +ReadSchema: struct + +(5) Filter +Input [2]: [key#x, val#x] +Condition : (isnotnull(val#x) AND (val#x > 0)) + +(6) Project +Output [1]: [key#x] +Input [2]: [key#x, val#x] + +(7) HashAggregate +Input [1]: [key#x] +Keys: [] +Functions [1]: [partial_max(key#x)] +Aggregate Attributes [1]: [max#x] +Results [1]: [max#x] + +(8) Exchange +Input [1]: [max#x] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +(9) HashAggregate +Input [1]: [max#x] +Keys: [] +Functions [1]: [max(key#x)] +Aggregate Attributes [1]: [max(key#x)#x] +Results [1]: [max(key#x)#x AS max(key)#x] + +(10) AdaptiveSparkPlan +Output [1]: [max(key)#x] +Arguments: isFinalPlan=false + +Subquery:2 Hosting operator id = 2 Hosting Expression = Subquery subquery#x, [id=#x] +AdaptiveSparkPlan (17) ++- HashAggregate (16) + +- Exchange (15) + +- HashAggregate (14) + +- Project (13) + +- Filter (12) + +- Scan parquet spark_catalog.default.explain_temp3 (11) + + +(11) Scan parquet spark_catalog.default.explain_temp3 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp3] +PushedFilters: [IsNotNull(val), GreaterThan(val,0)] +ReadSchema: struct + +(12) Filter +Input [2]: [key#x, val#x] +Condition : (isnotnull(val#x) AND (val#x > 0)) + +(13) Project +Output [1]: [key#x] +Input [2]: [key#x, val#x] + +(14) HashAggregate +Input [1]: [key#x] +Keys: [] +Functions [1]: [partial_avg(key#x)] +Aggregate Attributes [2]: [sum#x, count#xL] +Results [2]: [sum#x, count#xL] + +(15) Exchange +Input [2]: [sum#x, count#xL] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +(16) HashAggregate +Input [2]: [sum#x, count#xL] +Keys: [] +Functions [1]: [avg(key#x)] +Aggregate Attributes [1]: [avg(key#x)#x] +Results [1]: [avg(key#x)#x AS avg(key)#x] + +(17) AdaptiveSparkPlan +Output [1]: [avg(key)#x] +Arguments: isFinalPlan=false + +Subquery:3 Hosting operator id = 1 Hosting Expression = Subquery subquery#x, [id=#x] +AdaptiveSparkPlan (10) ++- HashAggregate (9) + +- Exchange (8) + +- HashAggregate (7) + +- Project (6) + +- Filter (5) + +- Scan parquet spark_catalog.default.explain_temp2 (4) + + +Subquery:4 Hosting operator id = 1 Hosting Expression = Subquery subquery#x, [id=#x] +AdaptiveSparkPlan (17) ++- HashAggregate (16) + +- Exchange (15) + +- HashAggregate (14) + +- Project (13) + +- Filter (12) + +- Scan parquet spark_catalog.default.explain_temp3 (11) + + +-- !query +EXPLAIN FORMATTED + SELECT (SELECT Avg(key) FROM explain_temp1) + (SELECT Avg(key) FROM explain_temp1) + FROM explain_temp1 +-- !query schema +struct +-- !query output +== Physical Plan == +AdaptiveSparkPlan (3) ++- Project (2) + +- Scan parquet spark_catalog.default.explain_temp1 (1) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output: [] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct<> + +(2) Project +Output [1]: [(Subquery subquery#x, [id=#x] + Subquery subquery#x, [id=#x]) AS (scalarsubquery() + scalarsubquery())#x] +Input: [] + +(3) AdaptiveSparkPlan +Output [1]: [(scalarsubquery() + scalarsubquery())#x] +Arguments: isFinalPlan=false + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 2 Hosting Expression = Subquery subquery#x, [id=#x] +AdaptiveSparkPlan (8) ++- HashAggregate (7) + +- Exchange (6) + +- HashAggregate (5) + +- Scan parquet spark_catalog.default.explain_temp1 (4) + + +(4) Scan parquet spark_catalog.default.explain_temp1 +Output [1]: [key#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct + +(5) HashAggregate +Input [1]: [key#x] +Keys: [] +Functions [1]: [partial_avg(key#x)] +Aggregate Attributes [2]: [sum#x, count#xL] +Results [2]: [sum#x, count#xL] + +(6) Exchange +Input [2]: [sum#x, count#xL] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +(7) HashAggregate +Input [2]: [sum#x, count#xL] +Keys: [] +Functions [1]: [avg(key#x)] +Aggregate Attributes [1]: [avg(key#x)#x] +Results [1]: [avg(key#x)#x AS avg(key)#x] + +(8) AdaptiveSparkPlan +Output [1]: [avg(key)#x] +Arguments: isFinalPlan=false + +Subquery:2 Hosting operator id = 2 Hosting Expression = Subquery subquery#x, [id=#x] +AdaptiveSparkPlan (13) ++- HashAggregate (12) + +- Exchange (11) + +- HashAggregate (10) + +- Scan parquet spark_catalog.default.explain_temp1 (9) + + +(9) Scan parquet spark_catalog.default.explain_temp1 +Output [1]: [key#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct + +(10) HashAggregate +Input [1]: [key#x] +Keys: [] +Functions [1]: [partial_avg(key#x)] +Aggregate Attributes [2]: [sum#x, count#xL] +Results [2]: [sum#x, count#xL] + +(11) Exchange +Input [2]: [sum#x, count#xL] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +(12) HashAggregate +Input [2]: [sum#x, count#xL] +Keys: [] +Functions [1]: [avg(key#x)] +Aggregate Attributes [1]: [avg(key#x)#x] +Results [1]: [avg(key#x)#x AS avg(key)#x] + +(13) AdaptiveSparkPlan +Output [1]: [avg(key)#x] +Arguments: isFinalPlan=false + + +-- !query +EXPLAIN FORMATTED + WITH cte1 AS ( + SELECT * + FROM explain_temp1 + WHERE key > 10 + ) + SELECT * FROM cte1 a, cte1 b WHERE a.key = b.key +-- !query schema +struct +-- !query output +== Physical Plan == +AdaptiveSparkPlan (7) ++- BroadcastHashJoin Inner BuildRight (6) + :- Filter (2) + : +- Scan parquet spark_catalog.default.explain_temp1 (1) + +- BroadcastExchange (5) + +- Filter (4) + +- Scan parquet spark_catalog.default.explain_temp1 (3) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,10)] +ReadSchema: struct + +(2) Filter +Input [2]: [key#x, val#x] +Condition : (isnotnull(key#x) AND (key#x > 10)) + +(3) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,10)] +ReadSchema: struct + +(4) Filter +Input [2]: [key#x, val#x] +Condition : (isnotnull(key#x) AND (key#x > 10)) + +(5) BroadcastExchange +Input [2]: [key#x, val#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=x] + +(6) BroadcastHashJoin +Left keys [1]: [key#x] +Right keys [1]: [key#x] +Join type: Inner +Join condition: None + +(7) AdaptiveSparkPlan +Output [4]: [key#x, val#x, key#x, val#x] +Arguments: isFinalPlan=false + + +-- !query +EXPLAIN FORMATTED + WITH cte1 AS ( + SELECT key, max(val) + FROM explain_temp1 + WHERE key > 10 + GROUP BY key + ) + SELECT * FROM cte1 a, cte1 b WHERE a.key = b.key +-- !query schema +struct +-- !query output +== Physical Plan == +AdaptiveSparkPlan (13) ++- BroadcastHashJoin Inner BuildRight (12) + :- HashAggregate (5) + : +- Exchange (4) + : +- HashAggregate (3) + : +- Filter (2) + : +- Scan parquet spark_catalog.default.explain_temp1 (1) + +- BroadcastExchange (11) + +- HashAggregate (10) + +- Exchange (9) + +- HashAggregate (8) + +- Filter (7) + +- Scan parquet spark_catalog.default.explain_temp1 (6) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,10)] +ReadSchema: struct + +(2) Filter +Input [2]: [key#x, val#x] +Condition : (isnotnull(key#x) AND (key#x > 10)) + +(3) HashAggregate +Input [2]: [key#x, val#x] +Keys [1]: [key#x] +Functions [1]: [partial_max(val#x)] +Aggregate Attributes [1]: [max#x] +Results [2]: [key#x, max#x] + +(4) Exchange +Input [2]: [key#x, max#x] +Arguments: hashpartitioning(key#x, 4), ENSURE_REQUIREMENTS, [plan_id=x] + +(5) HashAggregate +Input [2]: [key#x, max#x] +Keys [1]: [key#x] +Functions [1]: [max(val#x)] +Aggregate Attributes [1]: [max(val#x)#x] +Results [2]: [key#x, max(val#x)#x AS max(val)#x] + +(6) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,10)] +ReadSchema: struct + +(7) Filter +Input [2]: [key#x, val#x] +Condition : (isnotnull(key#x) AND (key#x > 10)) + +(8) HashAggregate +Input [2]: [key#x, val#x] +Keys [1]: [key#x] +Functions [1]: [partial_max(val#x)] +Aggregate Attributes [1]: [max#x] +Results [2]: [key#x, max#x] + +(9) Exchange +Input [2]: [key#x, max#x] +Arguments: hashpartitioning(key#x, 4), ENSURE_REQUIREMENTS, [plan_id=x] + +(10) HashAggregate +Input [2]: [key#x, max#x] +Keys [1]: [key#x] +Functions [1]: [max(val#x)] +Aggregate Attributes [1]: [max(val#x)#x] +Results [2]: [key#x, max(val#x)#x AS max(val)#x] + +(11) BroadcastExchange +Input [2]: [key#x, max(val)#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=x] + +(12) BroadcastHashJoin +Left keys [1]: [key#x] +Right keys [1]: [key#x] +Join type: Inner +Join condition: None + +(13) AdaptiveSparkPlan +Output [4]: [key#x, max(val)#x, key#x, max(val)#x] +Arguments: isFinalPlan=false + + +-- !query +EXPLAIN FORMATTED + CREATE VIEW explain_view AS + SELECT key, val FROM explain_temp1 +-- !query schema +struct +-- !query output +== Physical Plan == +Execute CreateViewCommand (1) + +- CreateViewCommand (2) + +- Project (5) + +- SubqueryAlias (4) + +- LogicalRelation (3) + + +(1) Execute CreateViewCommand +Output: [] + +(2) CreateViewCommand +Arguments: `spark_catalog`.`default`.`explain_view`, SELECT key, val FROM explain_temp1, false, false, PersistedView, COMPENSATION, true + +(3) LogicalRelation +Arguments: parquet, [key#x, val#x], `spark_catalog`.`default`.`explain_temp1`, false + +(4) SubqueryAlias +Arguments: spark_catalog.default.explain_temp1 + +(5) Project +Arguments: [key#x, val#x] + + +-- !query +EXPLAIN FORMATTED + SELECT + COUNT(val) + SUM(key) as TOTAL, + COUNT(key) FILTER (WHERE val > 1) + FROM explain_temp1 +-- !query schema +struct +-- !query output +== Physical Plan == +AdaptiveSparkPlan (5) ++- HashAggregate (4) + +- Exchange (3) + +- HashAggregate (2) + +- Scan parquet spark_catalog.default.explain_temp1 (1) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct + +(2) HashAggregate +Input [2]: [key#x, val#x] +Keys: [] +Functions [3]: [partial_count(val#x), partial_sum(key#x), partial_count(key#x) FILTER (WHERE (val#x > 1))] +Aggregate Attributes [3]: [count#xL, sum#xL, count#xL] +Results [3]: [count#xL, sum#xL, count#xL] + +(3) Exchange +Input [3]: [count#xL, sum#xL, count#xL] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +(4) HashAggregate +Input [3]: [count#xL, sum#xL, count#xL] +Keys: [] +Functions [3]: [count(val#x), sum(key#x), count(key#x)] +Aggregate Attributes [3]: [count(val#x)#xL, sum(key#x)#xL, count(key#x)#xL] +Results [2]: [(count(val#x)#xL + sum(key#x)#xL) AS TOTAL#xL, count(key#x)#xL AS count(key) FILTER (WHERE (val > 1))#xL] + +(5) AdaptiveSparkPlan +Output [2]: [TOTAL#xL, count(key) FILTER (WHERE (val > 1))#xL] +Arguments: isFinalPlan=false + + +-- !query +EXPLAIN FORMATTED + SELECT key, sort_array(collect_set(val))[0] + FROM explain_temp4 + GROUP BY key +-- !query schema +struct +-- !query output +== Physical Plan == +AdaptiveSparkPlan (5) ++- ObjectHashAggregate (4) + +- Exchange (3) + +- ObjectHashAggregate (2) + +- Scan parquet spark_catalog.default.explain_temp4 (1) + + +(1) Scan parquet spark_catalog.default.explain_temp4 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp4] +ReadSchema: struct + +(2) ObjectHashAggregate +Input [2]: [key#x, val#x] +Keys [1]: [key#x] +Functions [1]: [partial_collect_set(val#x, 0, 0)] +Aggregate Attributes [1]: [buf#x] +Results [2]: [key#x, buf#x] + +(3) Exchange +Input [2]: [key#x, buf#x] +Arguments: hashpartitioning(key#x, 4), ENSURE_REQUIREMENTS, [plan_id=x] + +(4) ObjectHashAggregate +Input [2]: [key#x, buf#x] +Keys [1]: [key#x] +Functions [1]: [collect_set(val#x, 0, 0)] +Aggregate Attributes [1]: [collect_set(val#x, 0, 0)#x] +Results [2]: [key#x, sort_array(collect_set(val#x, 0, 0)#x, true)[0] AS sort_array(collect_set(val), true)[0]#x] + +(5) AdaptiveSparkPlan +Output [2]: [key#x, sort_array(collect_set(val), true)[0]#x] +Arguments: isFinalPlan=false + + +-- !query +EXPLAIN FORMATTED + SELECT key, MIN(val) + FROM explain_temp4 + GROUP BY key +-- !query schema +struct +-- !query output +== Physical Plan == +AdaptiveSparkPlan (7) ++- SortAggregate (6) + +- Sort (5) + +- Exchange (4) + +- SortAggregate (3) + +- Sort (2) + +- Scan parquet spark_catalog.default.explain_temp4 (1) + + +(1) Scan parquet spark_catalog.default.explain_temp4 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp4] +ReadSchema: struct + +(2) Sort +Input [2]: [key#x, val#x] +Arguments: [key#x ASC NULLS FIRST], false, 0 + +(3) SortAggregate +Input [2]: [key#x, val#x] +Keys [1]: [key#x] +Functions [1]: [partial_min(val#x)] +Aggregate Attributes [1]: [min#x] +Results [2]: [key#x, min#x] + +(4) Exchange +Input [2]: [key#x, min#x] +Arguments: hashpartitioning(key#x, 4), ENSURE_REQUIREMENTS, [plan_id=x] + +(5) Sort +Input [2]: [key#x, min#x] +Arguments: [key#x ASC NULLS FIRST], false, 0 + +(6) SortAggregate +Input [2]: [key#x, min#x] +Keys [1]: [key#x] +Functions [1]: [min(val#x)] +Aggregate Attributes [1]: [min(val#x)#x] +Results [2]: [key#x, min(val#x)#x AS min(val)#x] + +(7) AdaptiveSparkPlan +Output [2]: [key#x, min(val)#x] +Arguments: isFinalPlan=false + + +-- !query +EXPLAIN EXTENDED INSERT INTO TABLE explain_temp5 SELECT * FROM explain_temp4 +-- !query schema +struct +-- !query output +== Parsed Logical Plan == +'InsertIntoStatement 'UnresolvedRelation [explain_temp5], [__required_write_privileges__=INSERT], false, false, false, false ++- 'Project [*] + +- 'UnresolvedRelation [explain_temp4], [], false + +== Analyzed Logical Plan == +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/explain_temp5, false, [val#x], Parquet, [path=file:[not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/explain_temp5), [key, val] ++- Project [key#x, val#x] + +- SubqueryAlias spark_catalog.default.explain_temp4 + +- Relation spark_catalog.default.explain_temp4[key#x,val#x] parquet + +== Optimized Logical Plan == +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/explain_temp5, false, [val#x], Parquet, [path=file:[not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/explain_temp5), [key, val] ++- WriteFiles + +- Sort [val#x ASC NULLS FIRST], false + +- Project [key#x, empty2null(val#x) AS val#x] + +- Relation spark_catalog.default.explain_temp4[key#x,val#x] parquet + +== Physical Plan == +Execute InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/explain_temp5, false, [val#x], Parquet, [path=file:[not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/explain_temp5), [key, val] ++- WriteFiles + +- *Sort [val#x ASC NULLS FIRST], false, 0 + +- *Project [key#x, empty2null(val#x) AS val#x] + +- *ColumnarToRow + +- FileScan parquet spark_catalog.default.explain_temp4[key#x,val#x] Batched: true, DataFilters: [], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp4], PartitionFilters: [], PushedFilters: [], ReadSchema: struct + + +-- !query +DROP TABLE explain_temp1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE explain_temp2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE explain_temp3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE explain_temp4 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE explain_temp5 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE table t(v array) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +EXPLAIN SELECT * FROM t WHERE v IN (array('a'), null) +-- !query schema +struct +-- !query output +== Physical Plan == +*Filter v#x IN ([a],null) ++- *ColumnarToRow + +- FileScan parquet spark_catalog.default.t[v#x] Batched: true, DataFilters: [v#x IN ([a],null)], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/t], PartitionFilters: [], PushedFilters: [In(v, [[a],null])], ReadSchema: struct> + + +-- !query +DROP TABLE t +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/explain-cbo.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/explain-cbo.sql.out new file mode 100644 index 000000000000..634c6b93d878 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/explain-cbo.sql.out @@ -0,0 +1,103 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE explain_temp1(a INT, b INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE explain_temp2(c INT, d INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +ANALYZE TABLE explain_temp1 COMPUTE STATISTICS FOR ALL COLUMNS +-- !query schema +struct<> +-- !query output + + + +-- !query +ANALYZE TABLE explain_temp2 COMPUTE STATISTICS FOR ALL COLUMNS +-- !query schema +struct<> +-- !query output + + + +-- !query +EXPLAIN COST WITH max_store_sales AS +( + SELECT max(csales) tpcds_cmax + FROM ( + SELECT sum(b) csales + FROM explain_temp1 WHERE a < 100 + ) x +), +best_ss_customer AS +( + SELECT c + FROM explain_temp2 + WHERE d > (SELECT * FROM max_store_sales) +) +SELECT c FROM best_ss_customer +-- !query schema +struct +-- !query output +== Optimized Logical Plan == +Project [c#x], Statistics(sizeInBytes=1.0 B, rowCount=0) ++- Filter (isnotnull(d#x) AND (cast(d#x as bigint) > scalar-subquery#x [])), Statistics(sizeInBytes=1.0 B, rowCount=0) + : +- Aggregate [max(csales#xL) AS tpcds_cmax#xL], Statistics(sizeInBytes=16.0 B, rowCount=1) + : +- Aggregate [sum(b#x) AS csales#xL], Statistics(sizeInBytes=16.0 B, rowCount=1) + : +- Project [b#x], Statistics(sizeInBytes=1.0 B, rowCount=0) + : +- Filter (isnotnull(a#x) AND (a#x < 100)), Statistics(sizeInBytes=1.0 B, rowCount=0) + : +- Relation spark_catalog.default.explain_temp1[a#x,b#x] parquet, Statistics(sizeInBytes=1.0 B, rowCount=0) + +- Relation spark_catalog.default.explain_temp2[c#x,d#x] parquet, Statistics(sizeInBytes=1.0 B, rowCount=0) + +== Physical Plan == +AdaptiveSparkPlan isFinalPlan=false ++- Project [c#x] + +- Filter (isnotnull(d#x) AND (cast(d#x as bigint) > Subquery subquery#x, [id=#x])) + : +- Subquery subquery#x, [id=#x] + : +- AdaptiveSparkPlan isFinalPlan=false + : +- HashAggregate(keys=[], functions=[max(csales#xL)], output=[tpcds_cmax#xL]) + : +- HashAggregate(keys=[], functions=[partial_max(csales#xL)], output=[max#xL]) + : +- HashAggregate(keys=[], functions=[sum(b#x)], output=[csales#xL]) + : +- Exchange SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + : +- HashAggregate(keys=[], functions=[partial_sum(b#x)], output=[sum#xL]) + : +- Project [b#x] + : +- Filter (isnotnull(a#x) AND (a#x < 100)) + : +- FileScan parquet spark_catalog.default.explain_temp1[a#x,b#x] Batched: true, DataFilters: [isnotnull(a#x), (a#x < 100)], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp1], PartitionFilters: [], PushedFilters: [IsNotNull(a), LessThan(a,100)], ReadSchema: struct + +- FileScan parquet spark_catalog.default.explain_temp2[c#x,d#x] Batched: true, DataFilters: [isnotnull(d#x), (cast(d#x as bigint) > Subquery subquery#x, [id=#x])], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp2], PartitionFilters: [], PushedFilters: [IsNotNull(d)], ReadSchema: struct + +- Subquery subquery#x, [id=#x] + +- AdaptiveSparkPlan isFinalPlan=false + +- HashAggregate(keys=[], functions=[max(csales#xL)], output=[tpcds_cmax#xL]) + +- HashAggregate(keys=[], functions=[partial_max(csales#xL)], output=[max#xL]) + +- HashAggregate(keys=[], functions=[sum(b#x)], output=[csales#xL]) + +- Exchange SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +- HashAggregate(keys=[], functions=[partial_sum(b#x)], output=[sum#xL]) + +- Project [b#x] + +- Filter (isnotnull(a#x) AND (a#x < 100)) + +- FileScan parquet spark_catalog.default.explain_temp1[a#x,b#x] Batched: true, DataFilters: [isnotnull(a#x), (a#x < 100)], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp1], PartitionFilters: [], PushedFilters: [IsNotNull(a), LessThan(a,100)], ReadSchema: struct + + +-- !query +DROP TABLE explain_temp1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE explain_temp2 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/explain.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/explain.sql.out new file mode 100644 index 000000000000..9d25b829e03f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/explain.sql.out @@ -0,0 +1,1124 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE table explain_temp1 (key int, val int) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE table explain_temp2 (key int, val int) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE table explain_temp3 (key int, val int) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE table explain_temp4 (key int, val string) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE table explain_temp5 (key int) USING PARQUET PARTITIONED BY(val string) +-- !query schema +struct<> +-- !query output + + + +-- !query +SET spark.sql.codegen.wholeStage = true +-- !query schema +struct +-- !query output +spark.sql.codegen.wholeStage true + + +-- !query +EXPLAIN EXTENDED + SELECT sum(distinct val) + FROM explain_temp1 +-- !query schema +struct +-- !query output +== Parsed Logical Plan == +'Project [unresolvedalias('sum(distinct 'val))] ++- 'UnresolvedRelation [explain_temp1], [], false + +== Analyzed Logical Plan == +sum(DISTINCT val): bigint +Aggregate [sum(distinct val#x) AS sum(DISTINCT val)#xL] ++- SubqueryAlias spark_catalog.default.explain_temp1 + +- Relation spark_catalog.default.explain_temp1[key#x,val#x] parquet + +== Optimized Logical Plan == +Aggregate [sum(distinct val#x) AS sum(DISTINCT val)#xL] ++- Project [val#x] + +- Relation spark_catalog.default.explain_temp1[key#x,val#x] parquet + +== Physical Plan == +*HashAggregate(keys=[], functions=[sum(distinct val#x)], output=[sum(DISTINCT val)#xL]) ++- Exchange SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +- *HashAggregate(keys=[], functions=[partial_sum(distinct val#x)], output=[sum#xL]) + +- *HashAggregate(keys=[val#x], functions=[], output=[val#x]) + +- Exchange hashpartitioning(val#x, 4), ENSURE_REQUIREMENTS, [plan_id=x] + +- *HashAggregate(keys=[val#x], functions=[], output=[val#x]) + +- *ColumnarToRow + +- FileScan parquet spark_catalog.default.explain_temp1[val#x] Batched: true, DataFilters: [], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp1], PartitionFilters: [], PushedFilters: [], ReadSchema: struct + + +-- !query +EXPLAIN FORMATTED + SELECT key, max(val) + FROM explain_temp1 + WHERE key > 0 + GROUP BY key + ORDER BY key +-- !query schema +struct +-- !query output +== Physical Plan == +* Sort (8) ++- Exchange (7) + +- * HashAggregate (6) + +- Exchange (5) + +- * HashAggregate (4) + +- * Filter (3) + +- * ColumnarToRow (2) + +- Scan parquet spark_catalog.default.explain_temp1 (1) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,0)] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 1] +Input [2]: [key#x, val#x] + +(3) Filter [codegen id : 1] +Input [2]: [key#x, val#x] +Condition : (isnotnull(key#x) AND (key#x > 0)) + +(4) HashAggregate [codegen id : 1] +Input [2]: [key#x, val#x] +Keys [1]: [key#x] +Functions [1]: [partial_max(val#x)] +Aggregate Attributes [1]: [max#x] +Results [2]: [key#x, max#x] + +(5) Exchange +Input [2]: [key#x, max#x] +Arguments: hashpartitioning(key#x, 4), ENSURE_REQUIREMENTS, [plan_id=x] + +(6) HashAggregate [codegen id : 2] +Input [2]: [key#x, max#x] +Keys [1]: [key#x] +Functions [1]: [max(val#x)] +Aggregate Attributes [1]: [max(val#x)#x] +Results [2]: [key#x, max(val#x)#x AS max(val)#x] + +(7) Exchange +Input [2]: [key#x, max(val)#x] +Arguments: rangepartitioning(key#x ASC NULLS FIRST, 4), ENSURE_REQUIREMENTS, [plan_id=x] + +(8) Sort [codegen id : 3] +Input [2]: [key#x, max(val)#x] +Arguments: [key#x ASC NULLS FIRST], true, 0 + + +-- !query +EXPLAIN FORMATTED + SELECT key, max(val) + FROM explain_temp1 + WHERE key > 0 + GROUP BY key + HAVING max(val) > 0 +-- !query schema +struct +-- !query output +== Physical Plan == +* Filter (7) ++- * HashAggregate (6) + +- Exchange (5) + +- * HashAggregate (4) + +- * Filter (3) + +- * ColumnarToRow (2) + +- Scan parquet spark_catalog.default.explain_temp1 (1) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,0)] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 1] +Input [2]: [key#x, val#x] + +(3) Filter [codegen id : 1] +Input [2]: [key#x, val#x] +Condition : (isnotnull(key#x) AND (key#x > 0)) + +(4) HashAggregate [codegen id : 1] +Input [2]: [key#x, val#x] +Keys [1]: [key#x] +Functions [1]: [partial_max(val#x)] +Aggregate Attributes [1]: [max#x] +Results [2]: [key#x, max#x] + +(5) Exchange +Input [2]: [key#x, max#x] +Arguments: hashpartitioning(key#x, 4), ENSURE_REQUIREMENTS, [plan_id=x] + +(6) HashAggregate [codegen id : 2] +Input [2]: [key#x, max#x] +Keys [1]: [key#x] +Functions [1]: [max(val#x)] +Aggregate Attributes [1]: [max(val#x)#x] +Results [2]: [key#x, max(val#x)#x AS max(val)#x] + +(7) Filter [codegen id : 2] +Input [2]: [key#x, max(val)#x] +Condition : (isnotnull(max(val)#x) AND (max(val)#x > 0)) + + +-- !query +EXPLAIN FORMATTED + SELECT key, val FROM explain_temp1 WHERE key > 0 + UNION + SELECT key, val FROM explain_temp1 WHERE key > 1 +-- !query schema +struct +-- !query output +== Physical Plan == +* HashAggregate (10) ++- Exchange (9) + +- * HashAggregate (8) + +- Union (7) + :- * Filter (3) + : +- * ColumnarToRow (2) + : +- Scan parquet spark_catalog.default.explain_temp1 (1) + +- * Filter (6) + +- * ColumnarToRow (5) + +- Scan parquet spark_catalog.default.explain_temp1 (4) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,0)] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 1] +Input [2]: [key#x, val#x] + +(3) Filter [codegen id : 1] +Input [2]: [key#x, val#x] +Condition : (isnotnull(key#x) AND (key#x > 0)) + +(4) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,1)] +ReadSchema: struct + +(5) ColumnarToRow [codegen id : 2] +Input [2]: [key#x, val#x] + +(6) Filter [codegen id : 2] +Input [2]: [key#x, val#x] +Condition : (isnotnull(key#x) AND (key#x > 1)) + +(7) Union + +(8) HashAggregate [codegen id : 3] +Input [2]: [key#x, val#x] +Keys [2]: [key#x, val#x] +Functions: [] +Aggregate Attributes: [] +Results [2]: [key#x, val#x] + +(9) Exchange +Input [2]: [key#x, val#x] +Arguments: hashpartitioning(key#x, val#x, 4), ENSURE_REQUIREMENTS, [plan_id=x] + +(10) HashAggregate [codegen id : 4] +Input [2]: [key#x, val#x] +Keys [2]: [key#x, val#x] +Functions: [] +Aggregate Attributes: [] +Results [2]: [key#x, val#x] + + +-- !query +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 a, + explain_temp2 b + WHERE a.key = b.key +-- !query schema +struct +-- !query output +== Physical Plan == +* BroadcastHashJoin Inner BuildRight (8) +:- * Filter (3) +: +- * ColumnarToRow (2) +: +- Scan parquet spark_catalog.default.explain_temp1 (1) ++- BroadcastExchange (7) + +- * Filter (6) + +- * ColumnarToRow (5) + +- Scan parquet spark_catalog.default.explain_temp2 (4) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key)] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 2] +Input [2]: [key#x, val#x] + +(3) Filter [codegen id : 2] +Input [2]: [key#x, val#x] +Condition : isnotnull(key#x) + +(4) Scan parquet spark_catalog.default.explain_temp2 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp2] +PushedFilters: [IsNotNull(key)] +ReadSchema: struct + +(5) ColumnarToRow [codegen id : 1] +Input [2]: [key#x, val#x] + +(6) Filter [codegen id : 1] +Input [2]: [key#x, val#x] +Condition : isnotnull(key#x) + +(7) BroadcastExchange +Input [2]: [key#x, val#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=x] + +(8) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [key#x] +Right keys [1]: [key#x] +Join type: Inner +Join condition: None + + +-- !query +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 a + LEFT OUTER JOIN explain_temp2 b + ON a.key = b.key +-- !query schema +struct +-- !query output +== Physical Plan == +* BroadcastHashJoin LeftOuter BuildRight (7) +:- * ColumnarToRow (2) +: +- Scan parquet spark_catalog.default.explain_temp1 (1) ++- BroadcastExchange (6) + +- * Filter (5) + +- * ColumnarToRow (4) + +- Scan parquet spark_catalog.default.explain_temp2 (3) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 2] +Input [2]: [key#x, val#x] + +(3) Scan parquet spark_catalog.default.explain_temp2 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp2] +PushedFilters: [IsNotNull(key)] +ReadSchema: struct + +(4) ColumnarToRow [codegen id : 1] +Input [2]: [key#x, val#x] + +(5) Filter [codegen id : 1] +Input [2]: [key#x, val#x] +Condition : isnotnull(key#x) + +(6) BroadcastExchange +Input [2]: [key#x, val#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=x] + +(7) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [key#x] +Right keys [1]: [key#x] +Join type: LeftOuter +Join condition: None + + +-- !query +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 + WHERE key = (SELECT max(key) + FROM explain_temp2 + WHERE key = (SELECT max(key) + FROM explain_temp3 + WHERE val > 0) + AND val = 2) + AND val > 3 +-- !query schema +struct +-- !query output +== Physical Plan == +* Filter (3) ++- * ColumnarToRow (2) + +- Scan parquet spark_catalog.default.explain_temp1 (1) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), IsNotNull(val), EqualTo(key,ScalarSubquery#x), GreaterThan(val,3)] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 1] +Input [2]: [key#x, val#x] + +(3) Filter [codegen id : 1] +Input [2]: [key#x, val#x] +Condition : (((isnotnull(key#x) AND isnotnull(val#x)) AND (key#x = ReusedSubquery Subquery scalar-subquery#x, [id=#x])) AND (val#x > 3)) + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#x, [id=#x] + +Subquery:2 Hosting operator id = 1 Hosting Expression = Subquery scalar-subquery#x, [id=#x] +* HashAggregate (10) ++- Exchange (9) + +- * HashAggregate (8) + +- * Project (7) + +- * Filter (6) + +- * ColumnarToRow (5) + +- Scan parquet spark_catalog.default.explain_temp2 (4) + + +(4) Scan parquet spark_catalog.default.explain_temp2 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp2] +PushedFilters: [IsNotNull(key), IsNotNull(val), EqualTo(key,ScalarSubquery#x), EqualTo(val,2)] +ReadSchema: struct + +(5) ColumnarToRow [codegen id : 1] +Input [2]: [key#x, val#x] + +(6) Filter [codegen id : 1] +Input [2]: [key#x, val#x] +Condition : (((isnotnull(key#x) AND isnotnull(val#x)) AND (key#x = ReusedSubquery Subquery scalar-subquery#x, [id=#x])) AND (val#x = 2)) + +(7) Project [codegen id : 1] +Output [1]: [key#x] +Input [2]: [key#x, val#x] + +(8) HashAggregate [codegen id : 1] +Input [1]: [key#x] +Keys: [] +Functions [1]: [partial_max(key#x)] +Aggregate Attributes [1]: [max#x] +Results [1]: [max#x] + +(9) Exchange +Input [1]: [max#x] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +(10) HashAggregate [codegen id : 2] +Input [1]: [max#x] +Keys: [] +Functions [1]: [max(key#x)] +Aggregate Attributes [1]: [max(key#x)#x] +Results [1]: [max(key#x)#x AS max(key)#x] + +Subquery:3 Hosting operator id = 6 Hosting Expression = ReusedSubquery Subquery scalar-subquery#x, [id=#x] + +Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#x, [id=#x] +* HashAggregate (17) ++- Exchange (16) + +- * HashAggregate (15) + +- * Project (14) + +- * Filter (13) + +- * ColumnarToRow (12) + +- Scan parquet spark_catalog.default.explain_temp3 (11) + + +(11) Scan parquet spark_catalog.default.explain_temp3 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp3] +PushedFilters: [IsNotNull(val), GreaterThan(val,0)] +ReadSchema: struct + +(12) ColumnarToRow [codegen id : 1] +Input [2]: [key#x, val#x] + +(13) Filter [codegen id : 1] +Input [2]: [key#x, val#x] +Condition : (isnotnull(val#x) AND (val#x > 0)) + +(14) Project [codegen id : 1] +Output [1]: [key#x] +Input [2]: [key#x, val#x] + +(15) HashAggregate [codegen id : 1] +Input [1]: [key#x] +Keys: [] +Functions [1]: [partial_max(key#x)] +Aggregate Attributes [1]: [max#x] +Results [1]: [max#x] + +(16) Exchange +Input [1]: [max#x] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +(17) HashAggregate [codegen id : 2] +Input [1]: [max#x] +Keys: [] +Functions [1]: [max(key#x)] +Aggregate Attributes [1]: [max(key#x)#x] +Results [1]: [max(key#x)#x AS max(key)#x] + + +-- !query +EXPLAIN FORMATTED + SELECT * + FROM explain_temp1 + WHERE key = (SELECT max(key) + FROM explain_temp2 + WHERE val > 0) + OR + key = (SELECT avg(key) + FROM explain_temp3 + WHERE val > 0) +-- !query schema +struct +-- !query output +== Physical Plan == +* Filter (3) ++- * ColumnarToRow (2) + +- Scan parquet spark_catalog.default.explain_temp1 (1) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 1] +Input [2]: [key#x, val#x] + +(3) Filter [codegen id : 1] +Input [2]: [key#x, val#x] +Condition : ((key#x = ReusedSubquery Subquery scalar-subquery#x, [id=#x]) OR (cast(key#x as double) = ReusedSubquery Subquery scalar-subquery#x, [id=#x])) + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#x, [id=#x] + +Subquery:2 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#x, [id=#x] + +Subquery:3 Hosting operator id = 1 Hosting Expression = Subquery scalar-subquery#x, [id=#x] +* HashAggregate (10) ++- Exchange (9) + +- * HashAggregate (8) + +- * Project (7) + +- * Filter (6) + +- * ColumnarToRow (5) + +- Scan parquet spark_catalog.default.explain_temp2 (4) + + +(4) Scan parquet spark_catalog.default.explain_temp2 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp2] +PushedFilters: [IsNotNull(val), GreaterThan(val,0)] +ReadSchema: struct + +(5) ColumnarToRow [codegen id : 1] +Input [2]: [key#x, val#x] + +(6) Filter [codegen id : 1] +Input [2]: [key#x, val#x] +Condition : (isnotnull(val#x) AND (val#x > 0)) + +(7) Project [codegen id : 1] +Output [1]: [key#x] +Input [2]: [key#x, val#x] + +(8) HashAggregate [codegen id : 1] +Input [1]: [key#x] +Keys: [] +Functions [1]: [partial_max(key#x)] +Aggregate Attributes [1]: [max#x] +Results [1]: [max#x] + +(9) Exchange +Input [1]: [max#x] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +(10) HashAggregate [codegen id : 2] +Input [1]: [max#x] +Keys: [] +Functions [1]: [max(key#x)] +Aggregate Attributes [1]: [max(key#x)#x] +Results [1]: [max(key#x)#x AS max(key)#x] + +Subquery:4 Hosting operator id = 1 Hosting Expression = Subquery scalar-subquery#x, [id=#x] +* HashAggregate (17) ++- Exchange (16) + +- * HashAggregate (15) + +- * Project (14) + +- * Filter (13) + +- * ColumnarToRow (12) + +- Scan parquet spark_catalog.default.explain_temp3 (11) + + +(11) Scan parquet spark_catalog.default.explain_temp3 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp3] +PushedFilters: [IsNotNull(val), GreaterThan(val,0)] +ReadSchema: struct + +(12) ColumnarToRow [codegen id : 1] +Input [2]: [key#x, val#x] + +(13) Filter [codegen id : 1] +Input [2]: [key#x, val#x] +Condition : (isnotnull(val#x) AND (val#x > 0)) + +(14) Project [codegen id : 1] +Output [1]: [key#x] +Input [2]: [key#x, val#x] + +(15) HashAggregate [codegen id : 1] +Input [1]: [key#x] +Keys: [] +Functions [1]: [partial_avg(key#x)] +Aggregate Attributes [2]: [sum#x, count#xL] +Results [2]: [sum#x, count#xL] + +(16) Exchange +Input [2]: [sum#x, count#xL] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +(17) HashAggregate [codegen id : 2] +Input [2]: [sum#x, count#xL] +Keys: [] +Functions [1]: [avg(key#x)] +Aggregate Attributes [1]: [avg(key#x)#x] +Results [1]: [avg(key#x)#x AS avg(key)#x] + + +-- !query +EXPLAIN FORMATTED + SELECT (SELECT Avg(key) FROM explain_temp1) + (SELECT Avg(key) FROM explain_temp1) + FROM explain_temp1 +-- !query schema +struct +-- !query output +== Physical Plan == +* Project (3) ++- * ColumnarToRow (2) + +- Scan parquet spark_catalog.default.explain_temp1 (1) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output: [] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct<> + +(2) ColumnarToRow [codegen id : 1] +Input: [] + +(3) Project [codegen id : 1] +Output [1]: [(Subquery scalar-subquery#x, [id=#x] + ReusedSubquery Subquery scalar-subquery#x, [id=#x]) AS (scalarsubquery() + scalarsubquery())#x] +Input: [] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 3 Hosting Expression = Subquery scalar-subquery#x, [id=#x] +* HashAggregate (8) ++- Exchange (7) + +- * HashAggregate (6) + +- * ColumnarToRow (5) + +- Scan parquet spark_catalog.default.explain_temp1 (4) + + +(4) Scan parquet spark_catalog.default.explain_temp1 +Output [1]: [key#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct + +(5) ColumnarToRow [codegen id : 1] +Input [1]: [key#x] + +(6) HashAggregate [codegen id : 1] +Input [1]: [key#x] +Keys: [] +Functions [1]: [partial_avg(key#x)] +Aggregate Attributes [2]: [sum#x, count#xL] +Results [2]: [sum#x, count#xL] + +(7) Exchange +Input [2]: [sum#x, count#xL] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +(8) HashAggregate [codegen id : 2] +Input [2]: [sum#x, count#xL] +Keys: [] +Functions [1]: [avg(key#x)] +Aggregate Attributes [1]: [avg(key#x)#x] +Results [1]: [avg(key#x)#x AS avg(key)#x] + +Subquery:2 Hosting operator id = 3 Hosting Expression = ReusedSubquery Subquery scalar-subquery#x, [id=#x] + + +-- !query +EXPLAIN FORMATTED + WITH cte1 AS ( + SELECT * + FROM explain_temp1 + WHERE key > 10 + ) + SELECT * FROM cte1 a, cte1 b WHERE a.key = b.key +-- !query schema +struct +-- !query output +== Physical Plan == +* BroadcastHashJoin Inner BuildRight (8) +:- * Filter (3) +: +- * ColumnarToRow (2) +: +- Scan parquet spark_catalog.default.explain_temp1 (1) ++- BroadcastExchange (7) + +- * Filter (6) + +- * ColumnarToRow (5) + +- Scan parquet spark_catalog.default.explain_temp1 (4) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,10)] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 2] +Input [2]: [key#x, val#x] + +(3) Filter [codegen id : 2] +Input [2]: [key#x, val#x] +Condition : (isnotnull(key#x) AND (key#x > 10)) + +(4) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,10)] +ReadSchema: struct + +(5) ColumnarToRow [codegen id : 1] +Input [2]: [key#x, val#x] + +(6) Filter [codegen id : 1] +Input [2]: [key#x, val#x] +Condition : (isnotnull(key#x) AND (key#x > 10)) + +(7) BroadcastExchange +Input [2]: [key#x, val#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=x] + +(8) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [key#x] +Right keys [1]: [key#x] +Join type: Inner +Join condition: None + + +-- !query +EXPLAIN FORMATTED + WITH cte1 AS ( + SELECT key, max(val) + FROM explain_temp1 + WHERE key > 10 + GROUP BY key + ) + SELECT * FROM cte1 a, cte1 b WHERE a.key = b.key +-- !query schema +struct +-- !query output +== Physical Plan == +* BroadcastHashJoin Inner BuildRight (10) +:- * HashAggregate (6) +: +- Exchange (5) +: +- * HashAggregate (4) +: +- * Filter (3) +: +- * ColumnarToRow (2) +: +- Scan parquet spark_catalog.default.explain_temp1 (1) ++- BroadcastExchange (9) + +- * HashAggregate (8) + +- ReusedExchange (7) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +PushedFilters: [IsNotNull(key), GreaterThan(key,10)] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 1] +Input [2]: [key#x, val#x] + +(3) Filter [codegen id : 1] +Input [2]: [key#x, val#x] +Condition : (isnotnull(key#x) AND (key#x > 10)) + +(4) HashAggregate [codegen id : 1] +Input [2]: [key#x, val#x] +Keys [1]: [key#x] +Functions [1]: [partial_max(val#x)] +Aggregate Attributes [1]: [max#x] +Results [2]: [key#x, max#x] + +(5) Exchange +Input [2]: [key#x, max#x] +Arguments: hashpartitioning(key#x, 4), ENSURE_REQUIREMENTS, [plan_id=x] + +(6) HashAggregate [codegen id : 4] +Input [2]: [key#x, max#x] +Keys [1]: [key#x] +Functions [1]: [max(val#x)] +Aggregate Attributes [1]: [max(val#x)#x] +Results [2]: [key#x, max(val#x)#x AS max(val)#x] + +(7) ReusedExchange [Reuses operator id: 5] +Output [2]: [key#x, max#x] + +(8) HashAggregate [codegen id : 3] +Input [2]: [key#x, max#x] +Keys [1]: [key#x] +Functions [1]: [max(val#x)] +Aggregate Attributes [1]: [max(val#x)#x] +Results [2]: [key#x, max(val#x)#x AS max(val)#x] + +(9) BroadcastExchange +Input [2]: [key#x, max(val)#x] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=x] + +(10) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [key#x] +Right keys [1]: [key#x] +Join type: Inner +Join condition: None + + +-- !query +EXPLAIN FORMATTED + CREATE VIEW explain_view AS + SELECT key, val FROM explain_temp1 +-- !query schema +struct +-- !query output +== Physical Plan == +Execute CreateViewCommand (1) + +- CreateViewCommand (2) + +- Project (5) + +- SubqueryAlias (4) + +- LogicalRelation (3) + + +(1) Execute CreateViewCommand +Output: [] + +(2) CreateViewCommand +Arguments: `spark_catalog`.`default`.`explain_view`, SELECT key, val FROM explain_temp1, false, false, PersistedView, COMPENSATION, true + +(3) LogicalRelation +Arguments: parquet, [key#x, val#x], `spark_catalog`.`default`.`explain_temp1`, false + +(4) SubqueryAlias +Arguments: spark_catalog.default.explain_temp1 + +(5) Project +Arguments: [key#x, val#x] + + +-- !query +EXPLAIN FORMATTED + SELECT + COUNT(val) + SUM(key) as TOTAL, + COUNT(key) FILTER (WHERE val > 1) + FROM explain_temp1 +-- !query schema +struct +-- !query output +== Physical Plan == +* HashAggregate (5) ++- Exchange (4) + +- * HashAggregate (3) + +- * ColumnarToRow (2) + +- Scan parquet spark_catalog.default.explain_temp1 (1) + + +(1) Scan parquet spark_catalog.default.explain_temp1 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp1] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 1] +Input [2]: [key#x, val#x] + +(3) HashAggregate [codegen id : 1] +Input [2]: [key#x, val#x] +Keys: [] +Functions [3]: [partial_count(val#x), partial_sum(key#x), partial_count(key#x) FILTER (WHERE (val#x > 1))] +Aggregate Attributes [3]: [count#xL, sum#xL, count#xL] +Results [3]: [count#xL, sum#xL, count#xL] + +(4) Exchange +Input [3]: [count#xL, sum#xL, count#xL] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=x] + +(5) HashAggregate [codegen id : 2] +Input [3]: [count#xL, sum#xL, count#xL] +Keys: [] +Functions [3]: [count(val#x), sum(key#x), count(key#x)] +Aggregate Attributes [3]: [count(val#x)#xL, sum(key#x)#xL, count(key#x)#xL] +Results [2]: [(count(val#x)#xL + sum(key#x)#xL) AS TOTAL#xL, count(key#x)#xL AS count(key) FILTER (WHERE (val > 1))#xL] + + +-- !query +EXPLAIN FORMATTED + SELECT key, sort_array(collect_set(val))[0] + FROM explain_temp4 + GROUP BY key +-- !query schema +struct +-- !query output +== Physical Plan == +ObjectHashAggregate (5) ++- Exchange (4) + +- ObjectHashAggregate (3) + +- * ColumnarToRow (2) + +- Scan parquet spark_catalog.default.explain_temp4 (1) + + +(1) Scan parquet spark_catalog.default.explain_temp4 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp4] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 1] +Input [2]: [key#x, val#x] + +(3) ObjectHashAggregate +Input [2]: [key#x, val#x] +Keys [1]: [key#x] +Functions [1]: [partial_collect_set(val#x, 0, 0)] +Aggregate Attributes [1]: [buf#x] +Results [2]: [key#x, buf#x] + +(4) Exchange +Input [2]: [key#x, buf#x] +Arguments: hashpartitioning(key#x, 4), ENSURE_REQUIREMENTS, [plan_id=x] + +(5) ObjectHashAggregate +Input [2]: [key#x, buf#x] +Keys [1]: [key#x] +Functions [1]: [collect_set(val#x, 0, 0)] +Aggregate Attributes [1]: [collect_set(val#x, 0, 0)#x] +Results [2]: [key#x, sort_array(collect_set(val#x, 0, 0)#x, true)[0] AS sort_array(collect_set(val), true)[0]#x] + + +-- !query +EXPLAIN FORMATTED + SELECT key, MIN(val) + FROM explain_temp4 + GROUP BY key +-- !query schema +struct +-- !query output +== Physical Plan == +SortAggregate (7) ++- * Sort (6) + +- Exchange (5) + +- SortAggregate (4) + +- * Sort (3) + +- * ColumnarToRow (2) + +- Scan parquet spark_catalog.default.explain_temp4 (1) + + +(1) Scan parquet spark_catalog.default.explain_temp4 +Output [2]: [key#x, val#x] +Batched: true +Location [not included in comparison]/{warehouse_dir}/explain_temp4] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 1] +Input [2]: [key#x, val#x] + +(3) Sort [codegen id : 1] +Input [2]: [key#x, val#x] +Arguments: [key#x ASC NULLS FIRST], false, 0 + +(4) SortAggregate +Input [2]: [key#x, val#x] +Keys [1]: [key#x] +Functions [1]: [partial_min(val#x)] +Aggregate Attributes [1]: [min#x] +Results [2]: [key#x, min#x] + +(5) Exchange +Input [2]: [key#x, min#x] +Arguments: hashpartitioning(key#x, 4), ENSURE_REQUIREMENTS, [plan_id=x] + +(6) Sort [codegen id : 2] +Input [2]: [key#x, min#x] +Arguments: [key#x ASC NULLS FIRST], false, 0 + +(7) SortAggregate +Input [2]: [key#x, min#x] +Keys [1]: [key#x] +Functions [1]: [min(val#x)] +Aggregate Attributes [1]: [min(val#x)#x] +Results [2]: [key#x, min(val#x)#x AS min(val)#x] + + +-- !query +EXPLAIN EXTENDED INSERT INTO TABLE explain_temp5 SELECT * FROM explain_temp4 +-- !query schema +struct +-- !query output +== Parsed Logical Plan == +'InsertIntoStatement 'UnresolvedRelation [explain_temp5], [__required_write_privileges__=INSERT], false, false, false, false ++- 'Project [*] + +- 'UnresolvedRelation [explain_temp4], [], false + +== Analyzed Logical Plan == +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/explain_temp5, false, [val#x], Parquet, [path=file:[not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/explain_temp5), [key, val] ++- Project [key#x, val#x] + +- SubqueryAlias spark_catalog.default.explain_temp4 + +- Relation spark_catalog.default.explain_temp4[key#x,val#x] parquet + +== Optimized Logical Plan == +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/explain_temp5, false, [val#x], Parquet, [path=file:[not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/explain_temp5), [key, val] ++- WriteFiles + +- Sort [val#x ASC NULLS FIRST], false + +- Project [key#x, empty2null(val#x) AS val#x] + +- Relation spark_catalog.default.explain_temp4[key#x,val#x] parquet + +== Physical Plan == +Execute InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/explain_temp5, false, [val#x], Parquet, [path=file:[not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/explain_temp5), [key, val] ++- WriteFiles + +- *Sort [val#x ASC NULLS FIRST], false, 0 + +- *Project [key#x, empty2null(val#x) AS val#x] + +- *ColumnarToRow + +- FileScan parquet spark_catalog.default.explain_temp4[key#x,val#x] Batched: true, DataFilters: [], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp4], PartitionFilters: [], PushedFilters: [], ReadSchema: struct + + +-- !query +DROP TABLE explain_temp1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE explain_temp2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE explain_temp3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE explain_temp4 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE explain_temp5 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE table t(v array) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +EXPLAIN SELECT * FROM t WHERE v IN (array('a'), null) +-- !query schema +struct +-- !query output +== Physical Plan == +*Filter v#x IN ([a],null) ++- *ColumnarToRow + +- FileScan parquet spark_catalog.default.t[v#x] Batched: true, DataFilters: [v#x IN ([a],null)], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/t], PartitionFilters: [], PushedFilters: [In(v, [[a],null])], ReadSchema: struct> + + +-- !query +DROP TABLE t +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/extract.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/extract.sql.out new file mode 100644 index 000000000000..02a7315be7a6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/extract.sql.out @@ -0,0 +1,1198 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS select '2011-05-06 07:08:09.1234567' as c, to_timestamp_ntz('2011-05-06 07:08:09.1234567') as ntz, interval 10 year 20 month as i, interval 30 day 40 hour 50 minute 6.7890 second as j +-- !query schema +struct<> +-- !query output + + + +-- !query +select extract(year from c), extract(year from ntz), extract(year from i) from t +-- !query schema +struct +-- !query output +2011 2011 11 + + +-- !query +select extract(y from c), extract(y from ntz), extract(y from i) from t +-- !query schema +struct +-- !query output +2011 2011 11 + + +-- !query +select extract(years from c), extract(years from ntz), extract(years from i) from t +-- !query schema +struct +-- !query output +2011 2011 11 + + +-- !query +select extract(yr from c), extract(yr from ntz), extract(yr from i) from t +-- !query schema +struct +-- !query output +2011 2011 11 + + +-- !query +select extract(yrs from c), extract(yrs from ntz), extract(yrs from i) from t +-- !query schema +struct +-- !query output +2011 2011 11 + + +-- !query +select extract(yearofweek from c), extract(yearofweek from ntz) from t +-- !query schema +struct +-- !query output +2011 2011 + + +-- !query +select extract(quarter from c), extract(quarter from ntz) from t +-- !query schema +struct +-- !query output +2 2 + + +-- !query +select extract(qtr from c), extract(qtr from ntz) from t +-- !query schema +struct +-- !query output +2 2 + + +-- !query +select extract(month from c), extract(month from ntz), extract(month from i) from t +-- !query schema +struct +-- !query output +5 5 8 + + +-- !query +select extract(mon from c), extract(mon from ntz), extract(mon from i) from t +-- !query schema +struct +-- !query output +5 5 8 + + +-- !query +select extract(mons from c), extract(mons from ntz), extract(mons from i) from t +-- !query schema +struct +-- !query output +5 5 8 + + +-- !query +select extract(months from c), extract(months from ntz), extract(months from i) from t +-- !query schema +struct +-- !query output +5 5 8 + + +-- !query +select extract(week from c), extract(week from ntz) from t +-- !query schema +struct +-- !query output +18 18 + + +-- !query +select extract(w from c), extract(w from ntz) from t +-- !query schema +struct +-- !query output +18 18 + + +-- !query +select extract(weeks from c), extract(weeks from ntz) from t +-- !query schema +struct +-- !query output +18 18 + + +-- !query +select extract(day from c), extract(day from ntz), extract(day from j) from t +-- !query schema +struct +-- !query output +6 6 31 + + +-- !query +select extract(d from c), extract(d from ntz), extract(d from j) from t +-- !query schema +struct +-- !query output +6 6 31 + + +-- !query +select extract(days from c), extract(days from ntz), extract(days from j) from t +-- !query schema +struct +-- !query output +6 6 31 + + +-- !query +select extract(dayofweek from c), extract(dayofweek from ntz) from t +-- !query schema +struct +-- !query output +6 6 + + +-- !query +select extract(dow from c), extract(dow from ntz) from t +-- !query schema +struct +-- !query output +6 6 + + +-- !query +select extract(dayofweek_iso from c), extract(dayofweek_iso from ntz) from t +-- !query schema +struct +-- !query output +5 5 + + +-- !query +select extract(dow_iso from c), extract(dow_iso from ntz) from t +-- !query schema +struct +-- !query output +5 5 + + +-- !query +select extract(doy from c), extract(doy from ntz) from t +-- !query schema +struct +-- !query output +126 126 + + +-- !query +select extract(hour from c), extract(hour from ntz), extract(hour from j) from t +-- !query schema +struct +-- !query output +7 7 16 + + +-- !query +select extract(h from c), extract(h from ntz), extract(h from j) from t +-- !query schema +struct +-- !query output +7 7 16 + + +-- !query +select extract(hours from c), extract(hours from ntz), extract(hours from j) from t +-- !query schema +struct +-- !query output +7 7 16 + + +-- !query +select extract(hr from c), extract(hr from ntz), extract(hr from j) from t +-- !query schema +struct +-- !query output +7 7 16 + + +-- !query +select extract(hrs from c), extract(hrs from ntz), extract(hrs from j) from t +-- !query schema +struct +-- !query output +7 7 16 + + +-- !query +select extract(minute from c), extract(minute from ntz), extract(minute from j) from t +-- !query schema +struct +-- !query output +8 8 50 + + +-- !query +select extract(m from c), extract(m from ntz), extract(m from j) from t +-- !query schema +struct +-- !query output +8 8 50 + + +-- !query +select extract(min from c), extract(min from ntz), extract(min from j) from t +-- !query schema +struct +-- !query output +8 8 50 + + +-- !query +select extract(mins from c), extract(mins from ntz), extract(mins from j) from t +-- !query schema +struct +-- !query output +8 8 50 + + +-- !query +select extract(minutes from c), extract(minutes from ntz), extract(minutes from j) from t +-- !query schema +struct +-- !query output +8 8 50 + + +-- !query +select extract(second from c), extract(second from ntz), extract(second from j) from t +-- !query schema +struct +-- !query output +9.123456 9.123456 6.789000 + + +-- !query +select extract(s from c), extract(s from ntz), extract(s from j) from t +-- !query schema +struct +-- !query output +9.123456 9.123456 6.789000 + + +-- !query +select extract(sec from c), extract(sec from ntz), extract(sec from j) from t +-- !query schema +struct +-- !query output +9.123456 9.123456 6.789000 + + +-- !query +select extract(seconds from c), extract(seconds from ntz), extract(seconds from j) from t +-- !query schema +struct +-- !query output +9.123456 9.123456 6.789000 + + +-- !query +select extract(secs from c), extract(secs from ntz), extract(secs from j) from t +-- !query schema +struct +-- !query output +9.123456 9.123456 6.789000 + + +-- !query +select extract(not_supported from c) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_FIELD", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "\"c\"", + "field" : "`not_supported`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "extract(not_supported from c)" + } ] +} + + +-- !query +select extract(not_supported from i) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_FIELD", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "\"i\"", + "field" : "`not_supported`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "extract(not_supported from i)" + } ] +} + + +-- !query +select extract(not_supported from j) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_FIELD", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "\"j\"", + "field" : "`not_supported`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "extract(not_supported from j)" + } ] +} + + +-- !query +select date_part('year', c), date_part('year', ntz), date_part('year', i) from t +-- !query schema +struct +-- !query output +2011 2011 11 + + +-- !query +select date_part('y', c), date_part('y', ntz), date_part('y', i) from t +-- !query schema +struct +-- !query output +2011 2011 11 + + +-- !query +select date_part('years', c), date_part('years', ntz), date_part('years', i) from t +-- !query schema +struct +-- !query output +2011 2011 11 + + +-- !query +select date_part('yr', c), date_part('yr', ntz), date_part('yr', i) from t +-- !query schema +struct +-- !query output +2011 2011 11 + + +-- !query +select date_part('yrs', c), date_part('yrs', ntz), date_part('yrs', i) from t +-- !query schema +struct +-- !query output +2011 2011 11 + + +-- !query +select date_part('yearofweek', c), date_part('yearofweek', ntz) from t +-- !query schema +struct +-- !query output +2011 2011 + + +-- !query +select date_part('quarter', c), date_part('quarter', ntz) from t +-- !query schema +struct +-- !query output +2 2 + + +-- !query +select date_part('qtr', c), date_part('qtr', ntz) from t +-- !query schema +struct +-- !query output +2 2 + + +-- !query +select date_part('month', c), date_part('month', ntz), date_part('month', i) from t +-- !query schema +struct +-- !query output +5 5 8 + + +-- !query +select date_part('mon', c), date_part('mon', ntz), date_part('mon', i) from t +-- !query schema +struct +-- !query output +5 5 8 + + +-- !query +select date_part('mons', c), date_part('mons', ntz), date_part('mons', i) from t +-- !query schema +struct +-- !query output +5 5 8 + + +-- !query +select date_part('months', c), date_part('months', ntz), date_part('months', i) from t +-- !query schema +struct +-- !query output +5 5 8 + + +-- !query +select date_part('week', c), date_part('week', ntz) from t +-- !query schema +struct +-- !query output +18 18 + + +-- !query +select date_part('w', c), date_part('w', ntz) from t +-- !query schema +struct +-- !query output +18 18 + + +-- !query +select date_part('weeks', c), date_part('weeks', ntz) from t +-- !query schema +struct +-- !query output +18 18 + + +-- !query +select date_part('day', c), date_part('day', ntz), date_part('day', j) from t +-- !query schema +struct +-- !query output +6 6 31 + + +-- !query +select date_part('d', c), date_part('d', ntz), date_part('d', j) from t +-- !query schema +struct +-- !query output +6 6 31 + + +-- !query +select date_part('days', c), date_part('days', ntz), date_part('days', j) from t +-- !query schema +struct +-- !query output +6 6 31 + + +-- !query +select date_part('dayofweek', c), date_part('dayofweek', ntz) from t +-- !query schema +struct +-- !query output +6 6 + + +-- !query +select date_part('dow', c), date_part('dow', ntz) from t +-- !query schema +struct +-- !query output +6 6 + + +-- !query +select date_part('dayofweek_iso', c), date_part('dayofweek_iso', ntz) from t +-- !query schema +struct +-- !query output +5 5 + + +-- !query +select date_part('dow_iso', c), date_part('dow_iso', ntz) from t +-- !query schema +struct +-- !query output +5 5 + + +-- !query +select date_part('doy', c), date_part('doy', ntz) from t +-- !query schema +struct +-- !query output +126 126 + + +-- !query +select date_part('hour', c), date_part('hour', ntz), date_part('hour', j) from t +-- !query schema +struct +-- !query output +7 7 16 + + +-- !query +select date_part('h', c), date_part('h', ntz), date_part('h', j) from t +-- !query schema +struct +-- !query output +7 7 16 + + +-- !query +select date_part('hours', c), date_part('hours', ntz), date_part('hours', j) from t +-- !query schema +struct +-- !query output +7 7 16 + + +-- !query +select date_part('hr', c), date_part('hr', ntz), date_part('hr', j) from t +-- !query schema +struct +-- !query output +7 7 16 + + +-- !query +select date_part('hrs', c), date_part('hrs', ntz), date_part('hrs', j) from t +-- !query schema +struct +-- !query output +7 7 16 + + +-- !query +select date_part('minute', c), date_part('minute', ntz), date_part('minute', j) from t +-- !query schema +struct +-- !query output +8 8 50 + + +-- !query +select date_part('m', c), date_part('m', ntz), date_part('m', j) from t +-- !query schema +struct +-- !query output +8 8 50 + + +-- !query +select date_part('min', c), date_part('min', ntz), date_part('min', j) from t +-- !query schema +struct +-- !query output +8 8 50 + + +-- !query +select date_part('mins', c), date_part('mins', ntz), date_part('mins', j) from t +-- !query schema +struct +-- !query output +8 8 50 + + +-- !query +select date_part('minutes', c), date_part('minutes', ntz), date_part('minutes', j) from t +-- !query schema +struct +-- !query output +8 8 50 + + +-- !query +select date_part('second', c), date_part('second', ntz), date_part('second', j) from t +-- !query schema +struct +-- !query output +9.123456 9.123456 6.789000 + + +-- !query +select date_part('s', c), date_part('s', ntz), date_part('s', j) from t +-- !query schema +struct +-- !query output +9.123456 9.123456 6.789000 + + +-- !query +select date_part('sec', c), date_part('sec', ntz), date_part('sec', j) from t +-- !query schema +struct +-- !query output +9.123456 9.123456 6.789000 + + +-- !query +select date_part('seconds', c), date_part('seconds', ntz), date_part('seconds', j) from t +-- !query schema +struct +-- !query output +9.123456 9.123456 6.789000 + + +-- !query +select date_part('secs', c), date_part('secs', ntz), date_part('secs', j) from t +-- !query schema +struct +-- !query output +9.123456 9.123456 6.789000 + + +-- !query +select date_part('not_supported', c) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_FIELD", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "\"c\"", + "field" : "`not_supported`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "date_part('not_supported', c)" + } ] +} + + +-- !query +select date_part(c, c) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NON_FOLDABLE_ARGUMENT", + "sqlState" : "42K08", + "messageParameters" : { + "funcName" : "`date_part`", + "paramName" : "`field`", + "paramType" : "\"STRING\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "date_part(c, c)" + } ] +} + + +-- !query +select date_part(null, c) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_part(i, i) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NON_FOLDABLE_ARGUMENT", + "sqlState" : "42K08", + "messageParameters" : { + "funcName" : "`date_part`", + "paramName" : "`field`", + "paramType" : "\"STRING\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "date_part(i, i)" + } ] +} + + +-- !query +select date_part(null, i) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select extract('year', c) from t +-- !query schema +struct +-- !query output +2011 + + +-- !query +select extract('quarter', c) from t +-- !query schema +struct +-- !query output +2 + + +-- !query +select extract('month', c) from t +-- !query schema +struct +-- !query output +5 + + +-- !query +select extract('week', c) from t +-- !query schema +struct +-- !query output +18 + + +-- !query +select extract('day', c) from t +-- !query schema +struct +-- !query output +6 + + +-- !query +select extract('days', c) from t +-- !query schema +struct +-- !query output +6 + + +-- !query +select extract('dayofweek', c) from t +-- !query schema +struct +-- !query output +6 + + +-- !query +select extract('dow', c) from t +-- !query schema +struct +-- !query output +6 + + +-- !query +select extract('doy', c) from t +-- !query schema +struct +-- !query output +126 + + +-- !query +select extract('hour', c) from t +-- !query schema +struct +-- !query output +7 + + +-- !query +select extract('minute', c) from t +-- !query schema +struct +-- !query output +8 + + +-- !query +select extract('second', c) from t +-- !query schema +struct +-- !query output +9.123456 + + +-- !query +select c - j from t +-- !query schema +struct +-- !query output +2011-04-04 14:18:02.334456 + + +-- !query +select day(c - j) from t +-- !query schema +struct +-- !query output +4 + + +-- !query +select extract(day from c - j) from t +-- !query schema +struct +-- !query output +4 + + +-- !query +select extract(month from to_timestamp(c) - i) from t +-- !query schema +struct +-- !query output +9 + + +-- !query +select extract(second from to_timestamp(c) - j) from t +-- !query schema +struct +-- !query output +2.334456 + + +-- !query +select extract(YEAR from interval '2-1' YEAR TO MONTH) +-- !query schema +struct +-- !query output +2 + + +-- !query +select date_part('YEAR', interval '2-1' YEAR TO MONTH) +-- !query schema +struct +-- !query output +2 + + +-- !query +select extract(YEAR from -interval '2-1' YEAR TO MONTH) +-- !query schema +struct +-- !query output +-2 + + +-- !query +select extract(MONTH from interval '2-1' YEAR TO MONTH) +-- !query schema +struct +-- !query output +1 + + +-- !query +select date_part('MONTH', interval '2-1' YEAR TO MONTH) +-- !query schema +struct +-- !query output +1 + + +-- !query +select extract(MONTH from -interval '2-1' YEAR TO MONTH) +-- !query schema +struct +-- !query output +-1 + + +-- !query +select date_part(NULL, interval '2-1' YEAR TO MONTH) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select extract(DAY from interval '2-1' YEAR TO MONTH) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_FIELD", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "\"INTERVAL '2-1' YEAR TO MONTH\"", + "field" : "`DAY`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "extract(DAY from interval '2-1' YEAR TO MONTH)" + } ] +} + + +-- !query +select date_part('DAY', interval '2-1' YEAR TO MONTH) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_FIELD", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "\"INTERVAL '2-1' YEAR TO MONTH\"", + "field" : "`DAY`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "date_part('DAY', interval '2-1' YEAR TO MONTH)" + } ] +} + + +-- !query +select date_part('not_supported', interval '2-1' YEAR TO MONTH) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_FIELD", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "\"INTERVAL '2-1' YEAR TO MONTH\"", + "field" : "`not_supported`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 63, + "fragment" : "date_part('not_supported', interval '2-1' YEAR TO MONTH)" + } ] +} + + +-- !query +select extract(DAY from interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query schema +struct +-- !query output +123 + + +-- !query +select date_part('DAY', interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query schema +struct +-- !query output +123 + + +-- !query +select extract(DAY from -interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query schema +struct +-- !query output +-123 + + +-- !query +select extract(HOUR from interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query schema +struct +-- !query output +12 + + +-- !query +select date_part('HOUR', interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query schema +struct +-- !query output +12 + + +-- !query +select extract(HOUR from -interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query schema +struct +-- !query output +-12 + + +-- !query +select extract(MINUTE from interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query schema +struct +-- !query output +34 + + +-- !query +select date_part('MINUTE', interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query schema +struct +-- !query output +34 + + +-- !query +select extract(MINUTE from -interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query schema +struct +-- !query output +-34 + + +-- !query +select extract(SECOND from interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query schema +struct +-- !query output +56.789123 + + +-- !query +select date_part('SECOND', interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query schema +struct +-- !query output +56.789123 + + +-- !query +select extract(SECOND from -interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query schema +struct +-- !query output +-56.789123 + + +-- !query +select date_part(NULL, interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select extract(MONTH from interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_FIELD", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "\"INTERVAL '123 12:34:56.789123' DAY TO SECOND\"", + "field" : "`MONTH`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "extract(MONTH from interval '123 12:34:56.789123123' DAY TO SECOND)" + } ] +} + + +-- !query +select date_part('not_supported', interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EXTRACT_FIELD", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "\"INTERVAL '123 12:34:56.789123' DAY TO SECOND\"", + "field" : "`not_supported`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "date_part('not_supported', interval '123 12:34:56.789123123' DAY TO SECOND)" + } ] +} + + +-- !query +select datepart('year', c), datepart('year', ntz), datepart('year', i) from t +-- !query schema +struct +-- !query output +2011 2011 11 + + +-- !query +select datepart('DAY', interval '123 12:34:56.789123123' DAY TO SECOND) +-- !query schema +struct +-- !query output +123 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-analytics.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-analytics.sql.out new file mode 100644 index 000000000000..f7f76242a4e6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-analytics.sql.out @@ -0,0 +1,1356 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH CUBE +-- !query schema +struct<(a + b):int,b:int,sum((a - b)):bigint> +-- !query output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL 1 3 +NULL 2 0 +NULL NULL 3 + + +-- !query +SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH CUBE +-- !query schema +struct +-- !query output +1 1 1 +1 2 2 +1 NULL 3 +2 1 1 +2 2 2 +2 NULL 3 +3 1 1 +3 2 2 +3 NULL 3 +NULL 1 3 +NULL 2 6 +NULL NULL 9 + + +-- !query +SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP +-- !query schema +struct<(a + b):int,b:int,sum((a - b)):bigint> +-- !query output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL NULL 3 + + +-- !query +SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH ROLLUP +-- !query schema +struct +-- !query output +1 1 1 +1 2 2 +1 NULL 3 +2 1 1 +2 2 2 +2 NULL 3 +3 1 1 +3 2 2 +3 NULL 3 +NULL NULL 9 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW courseSales AS SELECT * FROM VALUES +("dotNET", 2012, 10000), ("Java", 2012, 20000), ("dotNET", 2012, 5000), ("dotNET", 2013, 48000), ("Java", 2013, 30000) +AS courseSales(course, year, earnings) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY course, year +-- !query schema +struct +-- !query output +NULL NULL 113000 +Java NULL 50000 +Java 2012 20000 +Java 2013 30000 +dotNET NULL 63000 +dotNET 2012 15000 +dotNET 2013 48000 + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year, (course, year)) ORDER BY course, year +-- !query schema +struct +-- !query output +NULL NULL 113000 +Java NULL 50000 +Java 2012 20000 +Java 2012 20000 +Java 2013 30000 +Java 2013 30000 +dotNET NULL 63000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2013 48000 +dotNET 2013 48000 + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year, (course, year), ()) ORDER BY course, year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0051", + "messageParameters" : { + "element" : "ROLLUP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 53, + "stopIndex" : 101, + "fragment" : "GROUP BY ROLLUP(course, year, (course, year), ())" + } ] +} + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, year +-- !query schema +struct +-- !query output +NULL NULL 113000 +NULL 2012 35000 +NULL 2013 78000 +Java NULL 50000 +Java 2012 20000 +Java 2013 30000 +dotNET NULL 63000 +dotNET 2012 15000 +dotNET 2013 48000 + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year, (course, year)) ORDER BY course, year +-- !query schema +struct +-- !query output +NULL NULL 113000 +NULL 2012 35000 +NULL 2013 78000 +Java NULL 50000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +dotNET NULL 63000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year, (course, year), ()) ORDER BY course, year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0051", + "messageParameters" : { + "element" : "CUBE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 53, + "stopIndex" : 99, + "fragment" : "GROUP BY CUBE(course, year, (course, year), ())" + } ] +} + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year) +-- !query schema +struct +-- !query output +Java NULL 50000 +NULL 2012 35000 +NULL 2013 78000 +dotNET NULL 63000 + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year, ()) +-- !query schema +struct +-- !query output +Java NULL 50000 +NULL 2012 35000 +NULL 2013 78000 +NULL NULL 113000 +dotNET NULL 63000 + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course) +-- !query schema +struct +-- !query output +Java NULL 50000 +dotNET NULL 63000 + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year) +-- !query schema +struct +-- !query output +NULL 2012 35000 +NULL 2013 78000 + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY course, CUBE(course, year) ORDER BY course, year +-- !query schema +struct +-- !query output +Java NULL 50000 +Java NULL 50000 +Java 2012 20000 +Java 2012 20000 +Java 2013 30000 +Java 2013 30000 +dotNET NULL 63000 +dotNET NULL 63000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2013 48000 +dotNET 2013 48000 + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year), ROLLUP(course, year) ORDER BY course, year +-- !query schema +struct +-- !query output +NULL NULL 113000 +NULL 2012 35000 +NULL 2013 78000 +Java NULL 50000 +Java NULL 50000 +Java NULL 50000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +dotNET NULL 63000 +dotNET NULL 63000 +dotNET NULL 63000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year), ROLLUP(course, year), GROUPING SETS(course, year) ORDER BY course, year +-- !query schema +struct +-- !query output +NULL 2012 35000 +NULL 2012 35000 +NULL 2013 78000 +NULL 2013 78000 +Java NULL 50000 +Java NULL 50000 +Java NULL 50000 +Java NULL 50000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2012 20000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +Java 2013 30000 +dotNET NULL 63000 +dotNET NULL 63000 +dotNET NULL 63000 +dotNET NULL 63000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2012 15000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 +dotNET 2013 48000 + + +-- !query +SELECT course, SUM(earnings) AS sum FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum +-- !query schema +struct +-- !query output +NULL 113000 +Java 20000 +Java 30000 +Java 50000 +dotNET 5000 +dotNET 10000 +dotNET 48000 +dotNET 63000 + + +-- !query +SELECT course, SUM(earnings) AS sum, GROUPING_ID(course, earnings) FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum +-- !query schema +struct +-- !query output +NULL 113000 3 +Java 20000 0 +Java 30000 0 +Java 50000 1 +dotNET 5000 0 +dotNET 10000 0 +dotNET 48000 0 +dotNET 63000 1 + + +-- !query +SELECT course, year, GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales +GROUP BY CUBE(course, year) +-- !query schema +struct +-- !query output +Java 2012 0 0 0 +Java 2013 0 0 0 +Java NULL 0 1 1 +NULL 2012 1 0 2 +NULL 2013 1 0 2 +NULL NULL 1 1 3 +dotNET 2012 0 0 0 +dotNET 2013 0 0 0 +dotNET NULL 0 1 1 + + +-- !query +SELECT course, year, GROUPING(course) FROM courseSales GROUP BY course, year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", + "sqlState" : "42K0E", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 37, + "fragment" : "GROUPING(course)" + } ] +} + + +-- !query +SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY course, year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", + "sqlState" : "42K0E", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 46, + "fragment" : "GROUPING_ID(course, year)" + } ] +} + + +-- !query +SELECT course, year, grouping__id FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, year +-- !query schema +struct +-- !query output +Java 2012 0 +Java 2013 0 +dotNET 2012 0 +dotNET 2013 0 +Java NULL 1 +dotNET NULL 1 +NULL 2012 2 +NULL 2013 2 +NULL NULL 3 + + +-- !query +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) +HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0 ORDER BY course, year +-- !query schema +struct +-- !query output +NULL NULL +Java NULL +dotNET NULL + + +-- !query +SELECT course, year FROM courseSales GROUP BY course, year HAVING GROUPING(course) > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", + "sqlState" : "42K0E" +} + + +-- !query +SELECT course, year FROM courseSales GROUP BY course, year HAVING GROUPING_ID(course) > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", + "sqlState" : "42K0E" +} + + +-- !query +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) HAVING grouping__id > 0 +-- !query schema +struct +-- !query output +Java NULL +NULL 2012 +NULL 2013 +NULL NULL +dotNET NULL + + +-- !query +SELECT course, year, GROUPING(course), GROUPING(year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, year +-- !query schema +struct +-- !query output +Java 2012 0 0 +Java 2013 0 0 +dotNET 2012 0 0 +dotNET 2013 0 0 +Java NULL 0 1 +dotNET NULL 0 1 +NULL 2012 1 0 +NULL 2013 1 0 +NULL NULL 1 1 + + +-- !query +SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, year +-- !query schema +struct +-- !query output +Java 2012 0 +Java 2013 0 +dotNET 2012 0 +dotNET 2013 0 +Java NULL 1 +dotNET NULL 1 +NULL 2012 2 +NULL 2013 2 +NULL NULL 3 + + +-- !query +SELECT course, year FROM courseSales GROUP BY course, year ORDER BY GROUPING(course) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", + "sqlState" : "42K0E" +} + + +-- !query +SELECT course, year FROM courseSales GROUP BY course, year ORDER BY GROUPING_ID(course) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", + "sqlState" : "42K0E" +} + + +-- !query +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, year +-- !query schema +struct +-- !query output +Java 2012 +Java 2013 +dotNET 2012 +dotNET 2013 +Java NULL +dotNET NULL +NULL 2012 +NULL 2013 +NULL NULL + + +-- !query +SELECT a + b AS k1, b AS k2, SUM(a - b) FROM testData GROUP BY CUBE(k1, k2) +-- !query schema +struct +-- !query output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL 1 3 +NULL 2 0 +NULL NULL 3 + + +-- !query +SELECT a + b AS k, b, SUM(a - b) FROM testData GROUP BY ROLLUP(k, b) +-- !query schema +struct +-- !query output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL NULL 3 + + +-- !query +SELECT a + b, b AS k, SUM(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k) +-- !query schema +struct<(a + b):int,k:int,sum((a - b)):bigint> +-- !query output +NULL 1 3 +NULL 2 0 + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, b, CUBE(a, b) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, b, ROLLUP(a, b) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY CUBE(a, b), ROLLUP(a, b) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 NULL 2 +1 NULL 2 +1 NULL 2 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 NULL 2 +2 NULL 2 +2 NULL 2 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 NULL 2 +3 NULL 2 +3 NULL 2 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, CUBE(a, b), ROLLUP(b) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 NULL 2 +1 NULL 2 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 NULL 2 +2 NULL 2 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 NULL 2 +3 NULL 2 + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS((a, b), (a), ()) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +3 NULL 2 + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, CUBE(a, b), GROUPING SETS((a, b), (a), ()) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 NULL 2 +1 NULL 2 +1 NULL 2 +1 NULL 2 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 NULL 2 +2 NULL 2 +2 NULL 2 +2 NULL 2 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 NULL 2 +3 NULL 2 +3 NULL 2 +3 NULL 2 + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, CUBE(a, b), ROLLUP(a, b), GROUPING SETS((a, b), (a), ()) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 NULL 2 +1 NULL 2 +1 NULL 2 +1 NULL 2 +1 NULL 2 +1 NULL 2 +1 NULL 2 +1 NULL 2 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 NULL 2 +2 NULL 2 +2 NULL 2 +2 NULL 2 +2 NULL 2 +2 NULL 2 +2 NULL 2 +2 NULL 2 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 NULL 2 +3 NULL 2 +3 NULL 2 +3 NULL 2 +3 NULL 2 +3 NULL 2 +3 NULL 2 +3 NULL 2 + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(ROLLUP(a, b)) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +3 NULL 2 + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b), (a), ())) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +3 NULL 2 + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS((a, b), GROUPING SETS(ROLLUP(a, b))) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 NULL 2 +1 NULL 2 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 NULL 2 +2 NULL 2 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 NULL 2 +3 NULL 2 + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS((a, b, a, b), (a, b, a), (a, b)) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b, a, b), (a, b, a), (a, b))) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(ROLLUP(a, b), CUBE(a, b)) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +1 NULL 2 +1 NULL 2 +1 NULL 2 +1 NULL 2 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +2 NULL 2 +2 NULL 2 +2 NULL 2 +2 NULL 2 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 +3 NULL 2 +3 NULL 2 +3 NULL 2 +3 NULL 2 + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b), (a), ()), GROUPING SETS((a, b), (a), (b), ())) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +1 NULL 2 +1 NULL 2 +1 NULL 2 +1 NULL 2 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +2 NULL 2 +2 NULL 2 +2 NULL 2 +2 NULL 2 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 +3 NULL 2 +3 NULL 2 +3 NULL 2 +3 NULL 2 + + +-- !query +SELECT a, b, count(1) FROM testData GROUP BY a, GROUPING SETS((a, b), (a), (), (a, b), (a), (b), ()) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +1 NULL 2 +1 NULL 2 +1 NULL 2 +1 NULL 2 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +2 NULL 2 +2 NULL 2 +2 NULL 2 +2 NULL 2 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 +3 NULL 2 +3 NULL 2 +3 NULL 2 +3 NULL 2 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by-alias.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by-alias.sql.out new file mode 100644 index 000000000000..fd740c88bc55 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by-alias.sql.out @@ -0,0 +1,374 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT a from testData GROUP BY A +-- !query schema +struct +-- !query output +1 +2 +3 +NULL + + +-- !query +SELECT a AS k, COUNT(b) FROM testData GROUP BY k +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1 +-- !query schema +struct +-- !query output +2 2 +3 2 + + +-- !query +SELECT col1 AS k, SUM(col2) FROM testData AS t(col1, col2) GROUP BY k +-- !query schema +struct +-- !query output +1 3 +2 3 +3 3 +NULL 1 + + +-- !query +SELECT a as alias FROM testData GROUP BY ALIAS +-- !query schema +struct +-- !query output +1 +2 +3 +NULL + + +-- !query +SELECT a AS k FROM testData GROUP BY 'k' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT 1 AS k FROM testData GROUP BY 'k' +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT concat_ws(' ', a, b) FROM testData GROUP BY `concat_ws( , a, b)` +-- !query schema +struct +-- !query output + +1 +1 1 +1 2 +2 1 +2 2 +3 +3 1 +3 2 + + +-- !query +SELECT 1 AS a FROM testData GROUP BY a +-- !query schema +struct +-- !query output +1 +1 +1 +1 + + +-- !query +SELECT 1 AS a FROM testData GROUP BY `a` +-- !query schema +struct +-- !query output +1 +1 +1 +1 + + +-- !query +SELECT 1 GROUP BY `1` +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT (SELECT a FROM testData) + (SELECT b FROM testData) group by `(scalarsubquery() + scalarsubquery())` +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +{ + "errorClass" : "SCALAR_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "(SELECT a FROM testData)" + } ] +} + + +-- !query +SELECT a, count(*) FROM testData GROUP BY (SELECT b FROM testData) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT a, count(*) FROM testData GROUP BY a, (SELECT b FROM testData) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +{ + "errorClass" : "SCALAR_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 46, + "stopIndex" : 69, + "fragment" : "(SELECT b FROM testData)" + } ] +} + + +-- !query +SELECT a, count(*) FROM testData GROUP BY a, (SELECT b FROM testData LIMIT 1) +-- !query schema +struct +-- !query output +1 2 +2 2 +3 3 +NULL 2 + + +-- !query +SELECT a, count(*) FROM testData GROUP BY a, b IN (SELECT a FROM testData) +-- !query schema +struct +-- !query output +1 2 +2 2 +3 1 +3 2 +NULL 1 +NULL 1 + + +-- !query +SELECT a, count(*) FROM testData GROUP BY a, a IN (SELECT b FROM testData) +-- !query schema +struct +-- !query output +1 2 +2 2 +3 3 +NULL 2 + + +-- !query +SELECT a, count(*) FROM testData GROUP BY a, EXISTS(SELECT b FROM testData) +-- !query schema +struct +-- !query output +1 2 +2 2 +3 3 +NULL 2 + + +-- !query +SELECT a AS k, COUNT(non_existing) FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`non_existing`", + "proposal" : "`a`, `b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 33, + "fragment" : "non_existing" + } ] +} + + +-- !query +SELECT COUNT(b) AS k FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "GROUP_BY_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "sqlExpr" : "count(testdata.b)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "COUNT(b)" + } ] +} + + +-- !query +SELECT MAX(col1), 3 as abc FROM VALUES(1),(2),(3),(4) GROUP BY col1 % abc +-- !query schema +struct +-- !query output +2 3 +3 3 +4 3 + + +-- !query +set spark.sql.groupByAliases=false +-- !query schema +struct +-- !query output +spark.sql.groupByAliases false + + +-- !query +SELECT a AS k, COUNT(b) FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`k`", + "proposal" : "`a`, `b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 48, + "stopIndex" : 48, + "fragment" : "k" + } ] +} + + +-- !query +SELECT 1 GROUP BY `1` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 21, + "fragment" : "`1`" + } ] +} + + +-- !query +SELECT 1 AS col FROM testData GROUP BY `col` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`col`", + "proposal" : "`a`, `b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 40, + "stopIndex" : 44, + "fragment" : "`col`" + } ] +} + + +-- !query +SELECT 1 AS a FROM testData GROUP BY `a` +-- !query schema +struct +-- !query output +1 +1 +1 +1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by-all-duckdb.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by-all-duckdb.sql.out new file mode 100644 index 000000000000..30b5846ceff8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by-all-duckdb.sql.out @@ -0,0 +1,119 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view integers as select * from values + (0, 1), + (0, 2), + (1, 3), + (1, NULL) + as integers(g, i) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT g, SUM(i) FROM integers GROUP BY ALL ORDER BY 1 +-- !query schema +struct +-- !query output +0 3 +1 3 + + +-- !query +SELECT g, SUM(i), COUNT(*), COUNT(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1 +-- !query schema +struct +-- !query output +0 3 2 2 0 +1 3 2 1 2 + + +-- !query +SELECT i%2, SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1 +-- !query schema +struct<(i % 2):int,sum(i):bigint,sum(g):bigint> +-- !query output +NULL NULL 1 +0 2 0 +1 4 1 + + +-- !query +SELECT (g+i)%2, SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1 +-- !query schema +struct<((g + i) % 2):int,sum(i):bigint,sum(g):bigint> +-- !query output +NULL NULL 1 +0 5 1 +1 1 0 + + +-- !query +SELECT (g+i)%2 + SUM(i), SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 55, + "stopIndex" : 66, + "fragment" : "GROUP BY ALL" + } ] +} + + +-- !query +SELECT g, i, g%2, SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1, 2, 3, 4 +-- !query schema +struct +-- !query output +0 1 0 1 0 +0 2 0 2 0 +1 NULL 1 NULL 1 +1 3 1 3 1 + + +-- !query +SELECT c0 FROM (SELECT 1 c0) t0 GROUP BY ALL HAVING c0>0 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT c0 FROM (SELECT 1 c0, 1 c1 UNION ALL SELECT 1, 2) t0 GROUP BY ALL ORDER BY c0 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT c0 FROM (SELECT 1 c0, 1 c1 UNION ALL SELECT 1, 2) t0 GROUP BY ALL HAVING c1>0 ORDER BY c0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`c1`", + "proposal" : "`c0`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 81, + "stopIndex" : 82, + "fragment" : "c1" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by-all-mosha.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by-all-mosha.sql.out new file mode 100644 index 000000000000..f0708c56224f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by-all-mosha.sql.out @@ -0,0 +1,142 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view stuff as select * from values + (42, 9.75, 'hello world', '1970-08-07', '13.37', array(1,20,300)), + (1337, 1.2345, 'oh no', '2000-01-01', '42.0', array(4000,50000,600000)), + (42, 13.37, 'test', '1970-08-07', '1234567890', array(7000000,80000000,900000000)) + as stuff(i, f, s, t, d, a) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 100 * SUM(i) + SUM(f) / COUNT(s) AS f1, i AS f2 FROM stuff GROUP BY ALL ORDER BY f2 +-- !query schema +struct +-- !query output +8411.56000000000000000 42 +133701.23450000000000000 1337 + + +-- !query +SELECT i + 1 AS i1, COUNT(i - 2) ci, f / i AS fi, SUM(i + f) sif FROM stuff GROUP BY ALL ORDER BY 1, 3 +-- !query schema +struct +-- !query output +43 1 0.232142857142857 51.7500 +43 1 0.318333333333333 55.3700 +1338 1 0.000923335826477 1338.2345 + + +-- !query +SELECT i AS i, COUNT(i) ci, f AS f, SUM(i + f) sif FROM stuff GROUP BY ALL ORDER BY 1, i, 2, ci, 3, f, 4, sif +-- !query schema +struct +-- !query output +42 1 9.7500 51.7500 +42 1 13.3700 55.3700 +1337 1 1.2345 1338.2345 + + +-- !query +SELECT i + 1, f / i, substring(s, 2, 3), extract(year from t), octet_length(d), size(a) FROM stuff +GROUP BY ALL ORDER BY 1, 3, 4, 5, 6, 2 +-- !query schema +struct<(i + 1):int,(f / i):decimal(17,15),substring(s, 2, 3):string,extract(year FROM t):int,octet_length(d):int,size(a):int> +-- !query output +43 0.232142857142857 ell 1970 5 3 +43 0.318333333333333 est 1970 10 3 +1338 0.000923335826477 h n 2000 4 3 + + +-- !query +SELECT i + SUM(f) FROM stuff GROUP BY ALL +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 41, + "fragment" : "GROUP BY ALL" + } ] +} + + +-- !query +SELECT s AS s, COUNT(*) c FROM stuff GROUP BY ALL HAVING SUM(f) > 0 ORDER BY s +-- !query schema +struct +-- !query output +hello world 1 +oh no 1 +test 1 + + +-- !query +SELECT SUM(i) si FROM stuff GROUP BY ALL HAVING si > 2 +-- !query schema +struct +-- !query output +1421 + + +-- !query +SELECT SUM(i) si FROM stuff GROUP BY ALL HAVING si < 2 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT SUM(i) si FROM stuff GROUP BY ALL HAVING i > 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`i`", + "proposal" : "`si`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 49, + "stopIndex" : 49, + "fragment" : "i" + } ] +} + + +-- !query +SELECT SUM(i) si FROM stuff GROUP BY ALL ORDER BY i DESC +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`i`", + "proposal" : "`si`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 51, + "fragment" : "i" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by-all.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by-all.sql.out new file mode 100644 index 000000000000..bf3c1a3c21b8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by-all.sql.out @@ -0,0 +1,290 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view data as select * from values + ("USA", "San Francisco", "Reynold", 1, 11.0), + ("USA", "San Francisco", "Matei", 2, 12.0), + ("USA", "Berkeley", "Xiao", 3, 13.0), + ("China", "Hangzhou", "Wenchen", 4, 14.0), + ("China", "Shanghai", "Shanghaiese", 5, 15.0), + ("Korea", "Seoul", "Hyukjin", 6, 16.0), + ("UK", "London", "Sean", 7, 17.0) + as data(country, city, name, id, power) +-- !query schema +struct<> +-- !query output + + + +-- !query +select country, count(*) from data group by ALL +-- !query schema +struct +-- !query output +China 2 +Korea 1 +UK 1 +USA 3 + + +-- !query +select country, count(*) from data group by aLl +-- !query schema +struct +-- !query output +China 2 +Korea 1 +UK 1 +USA 3 + + +-- !query +select all, city, count(*) from (select country as all, city, id from data) group by all, city +-- !query schema +struct +-- !query output +China Hangzhou 1 +China Shanghai 1 +Korea Seoul 1 +UK London 1 +USA Berkeley 1 +USA San Francisco 2 + + +-- !query +SELECT count(1) FROM VALUES(1), (2), (3) AS T(all) GROUP BY all +-- !query schema +struct +-- !query output +1 +1 +1 + + +-- !query +select country, city, count(*), sum(power) from data group by all +-- !query schema +struct +-- !query output +China Hangzhou 1 14.0 +China Shanghai 1 15.0 +Korea Seoul 1 16.0 +UK London 1 17.0 +USA Berkeley 1 13.0 +USA San Francisco 2 23.0 + + +-- !query +select count(*), country, city, sum(power) from data group by all +-- !query schema +struct +-- !query output +1 China Hangzhou 14.0 +1 China Shanghai 15.0 +1 Korea Seoul 16.0 +1 UK London 17.0 +1 USA Berkeley 13.0 +2 USA San Francisco 23.0 + + +-- !query +select country as con, count(*) from data group by all +-- !query schema +struct +-- !query output +China 2 +Korea 1 +UK 1 +USA 3 + + +-- !query +select country, count(*) as cnt from data group by all +-- !query schema +struct +-- !query output +China 2 +Korea 1 +UK 1 +USA 3 + + +-- !query +select upper(country), count(*) as powerup from data group by all +-- !query schema +struct +-- !query output +CHINA 2 +KOREA 1 +UK 1 +USA 3 + + +-- !query +select country, sum(power) + 10 as powerup from data group by all +-- !query schema +struct +-- !query output +China 39.0 +Korea 26.0 +UK 27.0 +USA 46.0 + + +-- !query +select country, city from data group by all +-- !query schema +struct +-- !query output +China Hangzhou +China Shanghai +Korea Seoul +UK London +USA Berkeley +USA San Francisco + + +-- !query +select con, powerup from + (select country as con, sum(power) + 10 as powerup from data group by all) +-- !query schema +struct +-- !query output +China 39.0 +Korea 26.0 +UK 27.0 +USA 46.0 + + +-- !query +select country, count(id) as cnt from data group by all having cnt > 1 +-- !query schema +struct +-- !query output +China 2 +USA 3 + + +-- !query +select count(id) from data group by all +-- !query schema +struct +-- !query output +7 + + +-- !query +select count(id + power / 2) * 3 from data group by all +-- !query schema +struct<(count((id + (power / 2))) * 3):bigint> +-- !query output +21 + + +-- !query +select count(*) from (select * from data where country = "DNS") group by all +-- !query schema +struct +-- !query output +0 + + +-- !query +select id + count(*) from data group by all +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 43, + "fragment" : "group by all" + } ] +} + + +-- !query +select (id + id) / 2 + count(*) * 2 from data group by all +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 47, + "stopIndex" : 58, + "fragment" : "group by all" + } ] +} + + +-- !query +select country, (select count(*) from data) as cnt, count(id) as cnt_id from data group by all +-- !query schema +struct +-- !query output +China 7 2 +Korea 7 1 +UK 7 1 +USA 7 3 + + +-- !query +select country, (select count(*) from data d1 where d1.country = d2.country), count(id) from data d2 group by all +-- !query schema +struct +-- !query output +China 2 2 +Korea 1 1 +UK 1 1 +USA 3 3 + + +-- !query +select (select count(*) from data d1 where d1.country = d2.country) + count(id) from data d2 group by all +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 94, + "stopIndex" : 105, + "fragment" : "group by all" + } ] +} + + +-- !query +select non_exist from data group by all +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`non_exist`", + "proposal" : "`id`, `name`, `power`, `city`, `country`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "non_exist" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by-filter.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by-filter.sql.out new file mode 100644 index 000000000000..57d4681e2dd9 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by-filter.sql.out @@ -0,0 +1,984 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW FilterExpressionTestData AS SELECT * FROM VALUES + (1, 2, "asd"), + (3, 4, "fgh"), + (5, 6, "jkl") +AS FilterExpressionTestData(num1, num2, str) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData" + } ] +} + + +-- !query +SELECT COUNT(a) FILTER (WHERE a = 1), COUNT(b) FILTER (WHERE a > 1) FROM testData +-- !query schema +struct 1)):bigint> +-- !query output +2 4 + + +-- !query +SELECT COUNT(id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT COUNT(id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT COUNT(id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")) FROM emp +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT COUNT(id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") = "2001-01-01") FROM emp +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") = "2001-01-01 00:00:00") FROM emp +-- !query schema +struct +-- !query output +8 2 + + +-- !query +SELECT COUNT(DISTINCT id) FILTER (WHERE hiredate = to_timestamp("2001-01-01 00:00:00")), COUNT(DISTINCT id) FILTER (WHERE hiredate = to_date('2001-01-01 00:00:00')) FROM emp +-- !query schema +struct +-- !query output +2 2 + + +-- !query +SELECT SUM(salary), COUNT(DISTINCT id), COUNT(DISTINCT id) FILTER (WHERE hiredate = date "2001-01-01") FROM emp +-- !query schema +struct +-- !query output +2450.0 8 2 + + +-- !query +SELECT COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(DISTINCT id) FILTER (WHERE true) FROM emp +-- !query schema +struct +-- !query output +8 + + +-- !query +SELECT COUNT(DISTINCT id) FILTER (WHERE false) FROM emp +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id = 40) FROM emp +-- !query schema +struct +-- !query output +1 0 + + +-- !query +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id = 40) FROM emp +-- !query schema +struct +-- !query output +1 0 + + +-- !query +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id > 0) FROM emp +-- !query schema +struct 0)):bigint> +-- !query output +1 1 + + +-- !query +SELECT COUNT(DISTINCT 2), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id > 0) FROM emp +-- !query schema +struct 0)):bigint> +-- !query output +1 1 + + +-- !query +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id = 40) FROM emp +-- !query schema +struct +-- !query output +8 0 + + +-- !query +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id = 40) FROM emp +-- !query schema +struct +-- !query output +8 0 + + +-- !query +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 2,3) FILTER (WHERE dept_id > 0) FROM emp +-- !query schema +struct 0)):bigint> +-- !query output +8 1 + + +-- !query +SELECT COUNT(DISTINCT id), COUNT(DISTINCT 3,2) FILTER (WHERE dept_id > 0) FROM emp +-- !query schema +struct 0)):bigint> +-- !query output +8 1 + + +-- !query +SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData GROUP BY a +-- !query schema +struct= 2)):bigint> +-- !query output +1 0 +2 2 +3 2 +NULL 0 + + +-- !query +SELECT a, COUNT(b) FILTER (WHERE a != 2) FROM testData GROUP BY b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT COUNT(a) FILTER (WHERE a >= 0), COUNT(b) FILTER (WHERE a >= 3) FROM testData GROUP BY a +-- !query schema +struct= 0)):bigint,count(b) FILTER (WHERE (a >= 3)):bigint> +-- !query output +0 0 +2 0 +2 0 +3 2 + + +-- !query +SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > date "2003-01-01") FROM emp GROUP BY dept_id +-- !query schema +struct DATE '2003-01-01')):double> +-- !query output +10 200.0 +100 400.0 +20 NULL +30 400.0 +70 150.0 +NULL NULL + + +-- !query +SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > to_date("2003-01-01")) FROM emp GROUP BY dept_id +-- !query schema +struct to_date(2003-01-01))):double> +-- !query output +10 200.0 +100 400.0 +20 NULL +30 400.0 +70 150.0 +NULL NULL + + +-- !query +SELECT dept_id, SUM(salary) FILTER (WHERE hiredate > to_timestamp("2003-01-01 00:00:00")) FROM emp GROUP BY dept_id +-- !query schema +struct to_timestamp(2003-01-01 00:00:00))):double> +-- !query output +10 200.0 +100 400.0 +20 NULL +30 400.0 +70 150.0 +NULL NULL + + +-- !query +SELECT dept_id, SUM(salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") > "2003-01-01") FROM emp GROUP BY dept_id +-- !query schema +struct 2003-01-01)):double> +-- !query output +10 200.0 +100 400.0 +20 NULL +30 400.0 +70 150.0 +NULL NULL + + +-- !query +SELECT dept_id, SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id +-- !query schema +struct 2001-01-01 00:00:00)):double> +-- !query output +10 300.0 +100 400.0 +20 300.0 +30 400.0 +70 150.0 +NULL NULL + + +-- !query +SELECT dept_id, SUM(DISTINCT salary), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id +-- !query schema +struct 2001-01-01 00:00:00)):double> +-- !query output +10 300.0 300.0 +100 400.0 400.0 +20 300.0 300.0 +30 400.0 400.0 +70 150.0 150.0 +NULL 400.0 NULL + + +-- !query +SELECT dept_id, SUM(DISTINCT salary) FILTER (WHERE hiredate > date "2001-01-01"), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd HH:mm:ss") > "2001-01-01 00:00:00") FROM emp GROUP BY dept_id +-- !query schema +struct DATE '2001-01-01')):double,sum(DISTINCT salary) FILTER (WHERE (date_format(hiredate, yyyy-MM-dd HH:mm:ss) > 2001-01-01 00:00:00)):double> +-- !query output +10 300.0 300.0 +100 400.0 400.0 +20 300.0 300.0 +30 400.0 400.0 +70 150.0 150.0 +NULL NULL NULL + + +-- !query +SELECT dept_id, COUNT(id), SUM(DISTINCT salary), SUM(DISTINCT salary) FILTER (WHERE date_format(hiredate, "yyyy-MM-dd") > "2001-01-01") FROM emp GROUP BY dept_id +-- !query schema +struct 2001-01-01)):double> +-- !query output +10 3 300.0 300.0 +100 2 400.0 400.0 +20 1 300.0 300.0 +30 1 400.0 400.0 +70 1 150.0 150.0 +NULL 1 400.0 NULL + + +-- !query +SELECT b, COUNT(DISTINCT 1) FILTER (WHERE a = 1) FROM testData GROUP BY b +-- !query schema +struct +-- !query output +1 1 +2 1 +NULL 0 + + +-- !query +SELECT 'foo', COUNT(a) FILTER (WHERE b <= 2) FROM testData GROUP BY 1 +-- !query schema +struct +-- !query output +foo 6 + + +-- !query +SELECT 'foo', SUM(salary) FILTER (WHERE hiredate >= date "2003-01-01") FROM emp GROUP BY 1 +-- !query schema +struct= DATE '2003-01-01')):double> +-- !query output +foo 1350.0 + + +-- !query +SELECT 'foo', SUM(salary) FILTER (WHERE hiredate >= to_date("2003-01-01")) FROM emp GROUP BY 1 +-- !query schema +struct= to_date(2003-01-01))):double> +-- !query output +foo 1350.0 + + +-- !query +SELECT 'foo', SUM(salary) FILTER (WHERE hiredate >= to_timestamp("2003-01-01")) FROM emp GROUP BY 1 +-- !query schema +struct= to_timestamp(2003-01-01))):double> +-- !query output +foo 1350.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query schema +struct 200)):double> +-- !query output +10 2 2 400.0 NULL +100 2 2 800.0 800.0 +20 1 1 300.0 300.0 +30 1 1 400.0 400.0 +70 1 1 150.0 150.0 +NULL 1 1 400.0 400.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary), sum(salary) filter (where id + dept_id > 500) from emp group by dept_id +-- !query schema +struct 500)):double> +-- !query output +10 2 2 400.0 NULL +100 2 2 800.0 800.0 +20 1 1 300.0 NULL +30 1 1 400.0 NULL +70 1 1 150.0 150.0 +NULL 1 1 400.0 NULL + + +-- !query +select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query schema +struct 200)):double> +-- !query output +10 2 2 400.0 NULL +100 2 2 NULL 800.0 +20 1 1 300.0 300.0 +30 1 1 NULL 400.0 +70 1 1 150.0 150.0 +NULL 1 1 NULL 400.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct hiredate), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id + dept_id > 500) from emp group by dept_id +-- !query schema +struct 500)):double> +-- !query output +10 2 2 400.0 NULL +100 2 2 NULL 800.0 +20 1 1 300.0 NULL +30 1 1 NULL NULL +70 1 1 150.0 150.0 +NULL 1 1 NULL NULL + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double> +-- !query output +10 0 400.0 +100 2 800.0 +20 1 300.0 +30 1 400.0 +70 1 150.0 +NULL 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id +-- !query schema +struct 500)):bigint,sum(salary):double> +-- !query output +10 0 400.0 +100 2 800.0 +20 0 300.0 +30 0 400.0 +70 1 150.0 +NULL 0 400.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double> +-- !query output +10 2 0 400.0 +100 2 2 800.0 +20 1 1 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary) from emp group by dept_id +-- !query schema +struct 500)):bigint,sum(salary):double> +-- !query output +10 2 0 400.0 +100 2 2 800.0 +20 1 0 300.0 +30 1 0 400.0 +70 1 1 150.0 +NULL 1 0 400.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id > 200), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double,sum(salary) FILTER (WHERE (id > 200)):double> +-- !query output +10 2 0 400.0 NULL +100 2 2 800.0 800.0 +20 1 1 300.0 300.0 +30 1 1 400.0 400.0 +70 1 1 150.0 150.0 +NULL 1 1 400.0 400.0 + + +-- !query +select dept_id, count(distinct emp_name), count(distinct emp_name) filter (where id + dept_id > 500), sum(salary), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query schema +struct 500)):bigint,sum(salary):double,sum(salary) FILTER (WHERE (id > 200)):double> +-- !query output +10 2 0 400.0 NULL +100 2 2 800.0 800.0 +20 1 0 300.0 300.0 +30 1 0 400.0 400.0 +70 1 1 150.0 150.0 +NULL 1 0 400.0 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate):bigint,sum(salary):double> +-- !query output +10 0 2 400.0 +100 2 2 800.0 +20 1 1 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary):double> +-- !query output +10 0 1 400.0 +100 2 1 800.0 +20 1 0 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 0 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary) FILTER (WHERE (salary < 400.0)):double> +-- !query output +10 0 1 400.0 +100 2 1 NULL +20 1 0 300.0 +30 1 1 NULL +70 1 1 150.0 +NULL 1 0 NULL + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D), sum(salary) filter (where id > 200) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary) FILTER (WHERE (salary < 400.0)):double,sum(salary) FILTER (WHERE (id > 200)):double> +-- !query output +10 0 1 400.0 NULL +100 2 1 NULL 800.0 +20 1 0 300.0 300.0 +30 1 1 NULL 400.0 +70 1 1 150.0 150.0 +NULL 1 0 NULL 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT emp_name):bigint,sum(salary):double> +-- !query output +10 0 2 400.0 +100 2 2 800.0 +20 1 1 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name) filter (where id > 200), count(distinct emp_name) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT emp_name) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary):double> +-- !query output +10 0 1 400.0 +100 2 1 800.0 +20 1 0 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 0 400.0 + + +-- !query +select dept_id, sum(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate):bigint,sum(salary):double> +-- !query output +10 NULL 2 400.0 +100 1500 2 800.0 +20 320 1 300.0 +30 430 1 400.0 +70 870 1 150.0 +NULL NULL 1 400.0 + + +-- !query +select dept_id, sum(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary):double> +-- !query output +10 NULL 1 400.0 +100 1500 1 800.0 +20 320 0 300.0 +30 430 1 400.0 +70 870 1 150.0 +NULL NULL 0 400.0 + + +-- !query +select dept_id, avg(distinct (id + dept_id)) filter (where id > 200), count(distinct hiredate) filter (where hiredate > date "2003-01-01"), sum(salary) filter (where salary < 400.00D) from emp group by dept_id +-- !query schema +struct 200)):double,count(DISTINCT hiredate) FILTER (WHERE (hiredate > DATE '2003-01-01')):bigint,sum(salary) FILTER (WHERE (salary < 400.0)):double> +-- !query output +10 NULL 1 400.0 +100 750.0 1 NULL +20 320.0 0 300.0 +30 430.0 1 NULL +70 870.0 1 150.0 +NULL NULL 0 NULL + + +-- !query +select dept_id, count(distinct emp_name, hiredate) filter (where id > 200), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double> +-- !query output +10 0 400.0 +100 2 800.0 +20 1 300.0 +30 1 400.0 +70 1 150.0 +NULL 1 400.0 + + +-- !query +select dept_id, count(distinct emp_name, hiredate) filter (where id > 0), sum(salary) from emp group by dept_id +-- !query schema +struct 0)):bigint,sum(salary):double> +-- !query output +10 2 400.0 +100 2 800.0 +20 1 300.0 +30 1 400.0 +70 1 150.0 +NULL 1 400.0 + + +-- !query +select dept_id, count(distinct 1), count(distinct 1) filter (where id > 200), sum(salary) from emp group by dept_id +-- !query schema +struct 200)):bigint,sum(salary):double> +-- !query output +10 1 0 400.0 +100 1 1 800.0 +20 1 1 300.0 +30 1 1 400.0 +70 1 1 150.0 +NULL 1 1 400.0 + + +-- !query +SELECT 'foo', APPROX_COUNT_DISTINCT(a) FILTER (WHERE b >= 0) FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct= 0)):bigint> +-- !query output + + + +-- !query +SELECT 'foo', MAX(STRUCT(a)) FILTER (WHERE b >= 1) FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct= 1)):struct> +-- !query output + + + +-- !query +SELECT a + b, COUNT(b) FILTER (WHERE b >= 2) FROM testData GROUP BY a + b +-- !query schema +struct<(a + b):int,count(b) FILTER (WHERE (b >= 2)):bigint> +-- !query output +2 0 +3 1 +4 1 +5 1 +NULL 0 + + +-- !query +SELECT a + 2, COUNT(b) FILTER (WHERE b IN (1, 2)) FROM testData GROUP BY a + 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT a + 1 + 1, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY a + 1 +-- !query schema +struct<((a + 1) + 1):int,count(b) FILTER (WHERE (b > 0)):bigint> +-- !query output +3 2 +4 2 +5 2 +NULL 1 + + +-- !query +SELECT COUNT(DISTINCT b) FILTER (WHERE b > 0), COUNT(DISTINCT b, c) FILTER (WHERE b > 0 AND c > 2) +FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a +-- !query schema +struct 0)):bigint,count(DISTINCT b, c) FILTER (WHERE ((b > 0) AND (c > 2))):bigint> +-- !query output +1 1 + + +-- !query +SELECT a AS k, COUNT(b) FILTER (WHERE b > 0) FROM testData GROUP BY k +-- !query schema +struct 0)):bigint> +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT emp.dept_id, + avg(salary), + avg(salary) FILTER (WHERE id > (SELECT 200)) +FROM emp +GROUP BY dept_id +-- !query schema +struct scalarsubquery())):double> +-- !query output +10 133.33333333333334 NULL +100 400.0 400.0 +20 300.0 300.0 +30 400.0 400.0 +70 150.0 150.0 +NULL 400.0 400.0 + + +-- !query +SELECT emp.dept_id, + avg(salary), + avg(salary) FILTER (WHERE emp.dept_id = (SELECT dept_id FROM dept LIMIT 1)) +FROM emp +GROUP BY dept_id +-- !query schema +struct +-- !query output +10 133.33333333333334 133.33333333333334 +100 400.0 NULL +20 300.0 NULL +30 400.0 NULL +70 150.0 NULL +NULL 400.0 NULL + + +-- !query +SELECT emp.dept_id, + avg(salary), + avg(salary) FILTER (WHERE EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id)) +FROM emp +GROUP BY dept_id +-- !query schema +struct +-- !query output +10 133.33333333333334 133.33333333333334 +100 400.0 NULL +20 300.0 300.0 +30 400.0 400.0 +70 150.0 150.0 +NULL 400.0 NULL + + +-- !query +SELECT emp.dept_id, + Sum(salary), + Sum(salary) FILTER (WHERE NOT EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id)) +FROM emp +GROUP BY dept_id +-- !query schema +struct +-- !query output +10 400.0 NULL +100 800.0 800.0 +20 300.0 NULL +30 400.0 NULL +70 150.0 NULL +NULL 400.0 400.0 + + +-- !query +SELECT emp.dept_id, + avg(salary), + avg(salary) FILTER (WHERE emp.dept_id IN (SELECT DISTINCT dept_id + FROM dept)) +FROM emp +GROUP BY dept_id +-- !query schema +struct +-- !query output +10 133.33333333333334 133.33333333333334 +100 400.0 NULL +20 300.0 300.0 +30 400.0 400.0 +70 150.0 150.0 +NULL 400.0 NULL + + +-- !query +SELECT emp.dept_id, + Sum(salary), + Sum(salary) FILTER (WHERE emp.dept_id NOT IN (SELECT DISTINCT dept_id + FROM dept)) +FROM emp +GROUP BY dept_id +-- !query schema +struct +-- !query output +10 400.0 NULL +100 800.0 800.0 +20 300.0 NULL +30 400.0 NULL +70 150.0 NULL +NULL 400.0 NULL + + +-- !query +SELECT t1.b FROM (SELECT COUNT(b) FILTER (WHERE a >= 2) AS b FROM testData) t1 +-- !query schema +struct +-- !query output +4 + + +-- !query +SELECT count(num1) FILTER (WHERE rand(int(num2)) > 1) FROM FilterExpressionTestData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_AGGREGATE_FILTER.NON_DETERMINISTIC", + "sqlState" : "42903", + "messageParameters" : { + "filterExpr" : "\"(rand(num2) > 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "count(num1) FILTER (WHERE rand(int(num2)) > 1)" + } ] +} + + +-- !query +SELECT sum(num1) FILTER (WHERE str) FROM FilterExpressionTestData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_AGGREGATE_FILTER.NOT_BOOLEAN", + "sqlState" : "42903", + "messageParameters" : { + "filterExpr" : "\"str\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "sum(num1) FILTER (WHERE str)" + } ] +} + + +-- !query +SELECT sum(num1) FILTER (WHERE max(num2) > 1) FROM FilterExpressionTestData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_AGGREGATE_FILTER.CONTAINS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "\"max(num2)\"", + "filterExpr" : "\"(max(num2) > 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "sum(num1) FILTER (WHERE max(num2) > 1)" + } ] +} + + +-- !query +SELECT sum(num1) FILTER (WHERE nth_value(num2, 2) OVER(ORDER BY num2) > 1) FROM FilterExpressionTestData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_AGGREGATE_FILTER.CONTAINS_WINDOW_FUNCTION", + "sqlState" : "42903", + "messageParameters" : { + "filterExpr" : "\"(nth_value(num2, 2) OVER (ORDER BY num2 ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) > 1)\"", + "windowExpr" : "\"nth_value(num2, 2) OVER (ORDER BY num2 ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "sum(num1) FILTER (WHERE nth_value(num2, 2) OVER(ORDER BY num2) > 1)" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by-ordinal.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by-ordinal.sql.out new file mode 100644 index 000000000000..0f29c27268c8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by-ordinal.sql.out @@ -0,0 +1,524 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +select a, sum(b) from data group by 1 +-- !query schema +struct +-- !query output +1 3 +2 3 +3 3 + + +-- !query +select 1, 2, sum(b) from data group by 1, 2 +-- !query schema +struct<1:int,2:int,sum(b):bigint> +-- !query output +1 2 9 + + +-- !query +select a, 1, sum(b) from data group by a, 1 +-- !query schema +struct +-- !query output +1 1 3 +2 1 3 +3 1 3 + + +-- !query +select a, 1, sum(b) from data group by 1, 2 +-- !query schema +struct +-- !query output +1 1 3 +2 1 3 +3 1 3 + + +-- !query +select a, b + 2, count(2) from data group by a, 2 +-- !query schema +struct +-- !query output +1 3 1 +1 4 1 +2 3 1 +2 4 1 +3 3 1 +3 4 1 + + +-- !query +select a as aa, b + 2 as bb, count(2) from data group by 1, 2 +-- !query schema +struct +-- !query output +1 3 1 +1 4 1 +2 3 1 +2 4 1 +3 3 1 +3 4 1 + + +-- !query +select sum(b) from data group by 1 + 0 +-- !query schema +struct +-- !query output +9 + + +-- !query +select a, b from data group by -1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 33, + "fragment" : "-1" + } ] +} + + +-- !query +select a, b from data group by 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "0", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 32, + "fragment" : "0" + } ] +} + + +-- !query +select a, b from data group by 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "3", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 32, + "fragment" : "3" + } ] +} + + +-- !query +select a, b, sum(b) from data group by 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "sum(data.b) AS `sum(b)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 40, + "stopIndex" : 40, + "fragment" : "3" + } ] +} + + +-- !query +select a, b, sum(b) + 2 from data group by 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "(sum(data.b) + CAST(2 AS BIGINT)) AS `(sum(b) + 2)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 44, + "fragment" : "3" + } ] +} + + +-- !query +select a, rand(0), sum(b) +from +(select /*+ REPARTITION(1) */ a, b from data) group by a, 2 +-- !query schema +struct +-- !query output +1 0.5234194256885571 2 +1 0.7604953758285915 1 +2 0.0953472826424725 1 +2 0.3163249920547614 2 +3 0.2710259815484829 2 +3 0.7141011170991605 1 + + +-- !query +select * from data group by a, b, 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "STAR_GROUP_BY_POS", + "sqlState" : "0A000" +} + + +-- !query +select a, count(a) from (select 1 as a) tmp group by 1 order by 1 +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select count(a), a from (select 1 as a) tmp group by 2 having a > 0 +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select a, a AS k, count(b) from data group by k, 1 +-- !query schema +struct +-- !query output +1 1 2 +2 2 2 +3 3 2 + + +-- !query +select a, b, count(1) from data group by cube(1, 2) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by cube(1, b) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by 1, 2 with cube +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by rollup(1, 2) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by rollup(1, b) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by 1, 2 with rollup +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL NULL 6 + + +-- !query +select a, b, count(1) from data group by grouping sets((1), (2), (1, 2)) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 + + +-- !query +select a, b, count(1) from data group by grouping sets((1), (b), (a, 2)) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 + + +-- !query +select a, b, count(1) from data group by a, 2 grouping sets((1), (b), (a, 2)) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 + + +-- !query +select a, b, count(1) from data group by a, -1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 46, + "fragment" : "-1" + } ] +} + + +-- !query +select a, b, count(1) from data group by a, 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "count(1) AS `count(1)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 45, + "fragment" : "3" + } ] +} + + +-- !query +select a, b, count(1) from data group by cube(-1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 47, + "stopIndex" : 48, + "fragment" : "-1" + } ] +} + + +-- !query +select a, b, count(1) from data group by cube(1, 3) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "count(1) AS `count(1)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 50, + "stopIndex" : 50, + "fragment" : "3" + } ] +} + + +-- !query +set spark.sql.groupByOrdinal=false +-- !query schema +struct +-- !query output +spark.sql.groupByOrdinal false + + +-- !query +select sum(b) from data group by -1 +-- !query schema +struct +-- !query output +9 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by.sql.out new file mode 100644 index 000000000000..8608dbef843f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/group-by.sql.out @@ -0,0 +1,1172 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT a, COUNT(b) FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 32, + "fragment" : "SELECT a, COUNT(b) FROM testData" + } ] +} + + +-- !query +SELECT COUNT(a), COUNT(b) FROM testData +-- !query schema +struct +-- !query output +7 7 + + +-- !query +SELECT a, COUNT(b) FROM testData GROUP BY a +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT a, COUNT(b) FROM testData GROUP BY b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a +-- !query schema +struct +-- !query output +0 1 +2 2 +2 2 +3 2 + + +-- !query +SELECT 'foo', COUNT(a) FROM testData GROUP BY 1 +-- !query schema +struct +-- !query output +foo 7 + + +-- !query +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct> +-- !query output + + + +-- !query +SELECT a + b, COUNT(b) FROM testData GROUP BY a + b +-- !query schema +struct<(a + b):int,count(b):bigint> +-- !query output +2 1 +3 2 +4 2 +5 1 +NULL 1 + + +-- !query +SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1 +-- !query schema +struct<((a + 1) + 1):int,count(b):bigint> +-- !query output +3 2 +4 2 +5 2 +NULL 1 + + +-- !query +SELECT count(1) FROM testData GROUP BY struct(a + 0.1 AS aa) +-- !query schema +struct +-- !query output +2 +2 +2 +3 + + +-- !query +SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) +FROM testData +-- !query schema +struct +-- !query output +-0.2723801058145729 -1.5069204152249134 1 3 2.142857142857143 0.8095238095238094 0.8997354108424372 15 7 + + +-- !query +SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a +-- !query schema +struct +-- !query output +1 1 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"k\"", + "expressionAnyValue" : "\"any_value(k)\"" + } +} + + +-- !query +set spark.sql.groupByAliases=false +-- !query schema +struct +-- !query output +spark.sql.groupByAliases false + + +-- !query +SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a +-- !query schema +struct +-- !query output + + + +-- !query +SELECT COUNT(1) FROM testData WHERE false +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT 1 from ( + SELECT 1 AS z, + MIN(a.x) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z +-- !query schema +struct<1:int> +-- !query output + + + +-- !query +SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) +-- !query schema +struct +-- !query output +1.0 1.0 3 + + +-- !query +SELECT 1 FROM range(10) HAVING true +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT 1 FROM range(10) HAVING MAX(id) > 0 +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT id FROM range(10) HAVING id > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 38, + "fragment" : "SELECT id FROM range(10) HAVING id > 0" + } ] +} + + +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=true +-- !query schema +struct +-- !query output +spark.sql.legacy.parser.havingWithoutGroupByAsWhere true + + +-- !query +SELECT 1 FROM range(10) HAVING true +-- !query schema +struct<1:int> +-- !query output +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + + +-- !query +SELECT 1 FROM range(10) HAVING MAX(id) > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(max(id) > 0)\"", + "expressionList" : "max(id)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 42, + "fragment" : "SELECT 1 FROM range(10) HAVING MAX(id) > 0" + } ] +} + + +-- !query +SELECT id FROM range(10) HAVING id > 0 +-- !query schema +struct +-- !query output +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=false +-- !query schema +struct +-- !query output +spark.sql.legacy.parser.havingWithoutGroupByAsWhere false + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0 +-- !query schema +struct +-- !query output +NULL NULL NULL NULL NULL + + +-- !query +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4 +-- !query schema +struct +-- !query output +NULL NULL NULL NULL NULL + + +-- !query +SELECT every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5 +-- !query schema +struct +-- !query output +false true true false true + + +-- !query +SELECT k, every(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k +-- !query schema +struct +-- !query output +1 false true true false true +2 true true true true true +3 false false false false false +4 NULL NULL NULL NULL NULL +5 false true true false true + + +-- !query +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false +-- !query schema +struct +-- !query output +1 false +3 false +5 false + + +-- !query +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL +-- !query schema +struct +-- !query output +4 NULL + + +-- !query +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY k +-- !query schema +struct +-- !query output +2 true + + +-- !query +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY k +-- !query schema +struct +-- !query output + + + +-- !query +SELECT every(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"every(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "every(1)" + } ] +} + + +-- !query +SELECT some(1S) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "first", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"some(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "some(1S)" + } ] +} + + +-- !query +SELECT any(1L) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "first", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"any(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 14, + "fragment" : "any(1L)" + } ] +} + + +-- !query +SELECT every("true") +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT bool_and(1.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "first", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"bool_and(1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "bool_and(1.0)" + } ] +} + + +-- !query +SELECT bool_or(1.0D) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "first", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"bool_or(1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "bool_or(1.0D)" + } ] +} + + +-- !query +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true false +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true false + + +-- !query +SELECT k, v, some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT k, v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT k, v, bool_and(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true false +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true false + + +-- !query +SELECT k, v, bool_or(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT count(*) FROM test_agg HAVING count(*) > 1L +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true +-- !query schema +struct +-- !query output +1 true +2 true +5 true + + +-- !query +SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT count(*) FROM test_agg WHERE count(*) > 1L +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(count(1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 49, + "fragment" : "SELECT count(*) FROM test_agg WHERE count(*) > 1L" + } ] +} + + +-- !query +SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"((count(1) + 1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L" + } ] +} + + +-- !query +SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(((k = 1) OR (k = 2)) OR (((count(1) + 1) > 1) OR (max(k) > 1)))\"", + "expressionList" : "count(1), max(test_agg.k)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 86, + "fragment" : "SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1" + } ] +} + + +-- !query +SELECT AVG(DISTINCT decimal_col), SUM(DISTINCT decimal_col) FROM VALUES (CAST(1 AS DECIMAL(9, 0))) t(decimal_col) +-- !query schema +struct +-- !query output +1.0000 1 + + +-- !query +SELECT not(a IS NULL), count(*) AS c +FROM testData +GROUP BY a IS NULL +-- !query schema +struct<(NOT (a IS NULL)):boolean,c:bigint> +-- !query output +false 2 +true 7 + + +-- !query +SELECT if(not(a IS NULL), rand(0), 1), count(*) AS c +FROM testData +GROUP BY a IS NULL +-- !query schema +struct<(IF((NOT (a IS NULL)), rand(0), 1)):double,c:bigint> +-- !query output +0.7604953758285915 7 +1.0 2 + + +-- !query +SELECT + histogram_numeric(col, 2) as histogram_2, + histogram_numeric(col, 3) as histogram_3, + histogram_numeric(col, 5) as histogram_5, + histogram_numeric(col, 10) as histogram_10 +FROM VALUES + (1), (2), (3), (4), (5), (6), (7), (8), (9), (10), + (11), (12), (13), (14), (15), (16), (17), (18), (19), (20), + (21), (22), (23), (24), (25), (26), (27), (28), (29), (30), + (31), (32), (33), (34), (35), (3), (37), (38), (39), (40), + (41), (42), (43), (44), (45), (46), (47), (48), (49), (50) AS tab(col) +-- !query schema +struct>,histogram_3:array>,histogram_5:array>,histogram_10:array>> +-- !query output +[{"x":12,"y":26.0},{"x":38,"y":24.0}] [{"x":9,"y":20.0},{"x":25,"y":11.0},{"x":40,"y":19.0}] [{"x":5,"y":11.0},{"x":14,"y":8.0},{"x":22,"y":7.0},{"x":30,"y":10.0},{"x":43,"y":14.0}] [{"x":3,"y":6.0},{"x":8,"y":6.0},{"x":13,"y":4.0},{"x":17,"y":3.0},{"x":20,"y":4.0},{"x":25,"y":6.0},{"x":31,"y":7.0},{"x":39,"y":5.0},{"x":43,"y":4.0},{"x":48,"y":5.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1), (2), (3) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1L), (2L), (3L) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1F), (2F), (3F) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1.0,"y":1.0},{"x":2.0,"y":1.0},{"x":3.0,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1D), (2D), (3D) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1.0,"y":1.0},{"x":2.0,"y":1.0},{"x":3.0,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (1S), (2S), (3S) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BYTE)), (CAST(2 AS BYTE)), (CAST(3 AS BYTE)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS TINYINT)), (CAST(2 AS TINYINT)), (CAST(3 AS TINYINT)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS SMALLINT)), (CAST(2 AS SMALLINT)), (CAST(3 AS SMALLINT)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS BIGINT)), (CAST(2 AS BIGINT)), (CAST(3 AS BIGINT)) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1,"y":1.0},{"x":2,"y":1.0},{"x":3,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES + (CAST(1 AS DECIMAL(4, 2))), (CAST(2 AS DECIMAL(4, 2))), (CAST(3 AS DECIMAL(4, 2))) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":1.00,"y":1.0},{"x":2.00,"y":1.0},{"x":3.00,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (TIMESTAMP '2017-03-01 00:00:00'), + (TIMESTAMP '2017-04-01 00:00:00'), (TIMESTAMP '2017-05-01 00:00:00') AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":2017-03-01 00:00:00,"y":1.0},{"x":2017-04-01 00:00:00,"y":1.0},{"x":2017-05-01 00:00:00,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '100-00' YEAR TO MONTH), + (INTERVAL '110-00' YEAR TO MONTH), (INTERVAL '120-00' YEAR TO MONTH) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":100-0,"y":1.0},{"x":110-0,"y":1.0},{"x":120-0,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) FROM VALUES (INTERVAL '12 20:4:0' DAY TO SECOND), + (INTERVAL '12 21:4:0' DAY TO SECOND), (INTERVAL '12 22:4:0' DAY TO SECOND) AS tab(col) +-- !query schema +struct>> +-- !query output +[{"x":12 20:04:00.000000000,"y":1.0},{"x":12 21:04:00.000000000,"y":1.0},{"x":12 22:04:00.000000000,"y":1.0}] + + +-- !query +SELECT histogram_numeric(col, 3) +FROM VALUES (NULL), (NULL), (NULL) AS tab(col) +-- !query schema +struct>> +-- !query output +NULL + + +-- !query +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)), (CAST(NULL AS DOUBLE)) AS tab(col) +-- !query schema +struct>> +-- !query output +NULL + + +-- !query +SELECT histogram_numeric(col, 3) +FROM VALUES (CAST(NULL AS INT)), (CAST(NULL AS INT)), (CAST(NULL AS INT)) AS tab(col) +-- !query schema +struct>> +-- !query output +NULL + + +-- !query +SELECT + collect_list(col), + array_agg(col) +FROM VALUES + (1), (2), (1) AS tab(col) +-- !query schema +struct,collect_list(col):array> +-- !query output +[1,2,1] [1,2,1] + + +-- !query +SELECT + a, + collect_list(b), + array_agg(b) +FROM VALUES + (1,4),(2,3),(1,4),(2,4) AS v(a,b) +GROUP BY a +-- !query schema +struct,collect_list(b):array> +-- !query output +1 [4,4] [4,4] +2 [3,4] [3,4] + + +-- !query +SELECT c * 2 AS d +FROM ( + SELECT if(b > 1, 1, b) AS c + FROM ( + SELECT if(a < 0, 0, a) AS b + FROM VALUES (-1), (1), (2) AS t1(a) + ) t2 + GROUP BY b + ) t3 +GROUP BY c +-- !query schema +struct +-- !query output +0 +2 + + +-- !query +SELECT col1, count(*) AS cnt +FROM VALUES + (0.0), + (-0.0), + (double('NaN')), + (double('NaN')), + (double('Infinity')), + (double('Infinity')), + (-double('Infinity')), + (-double('Infinity')) +GROUP BY col1 +ORDER BY col1 +-- !query schema +struct +-- !query output +-Infinity 2 +0.0 2 +Infinity 2 +NaN 2 + + +-- !query +SELECT count(*) +FROM VALUES (ARRAY(MAP(1, 2, 2, 3), MAP(1, 3))), (ARRAY(MAP(2, 3), MAP(1, 3))), (ARRAY(MAP(2, 3, 1, 2), MAP(1, 3))) as t(a) +GROUP BY a +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT count(*) +FROM VALUES (named_struct('b', map(1, 2, 2, 3))), (named_struct('b', map(1, 3))), (named_struct('b', map(2, 3, 1, 2))) as t(a) +GROUP BY a +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT count(*) +FROM VALUES (named_struct('b', map(1, 2, 2, 3))), (named_struct('b', map(1, 3))), (named_struct('b', map(2, 3, 1, 2))) as t(a) +GROUP BY a.b +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT count(*) +FROM VALUES (named_struct('b', array(map(1, 2, 2, 3), map(1, 3)))), (named_struct('b', array(map(2, 3), map(1, 3)))), (named_struct('b', array(map(2, 3, 1, 2), map(1, 3)))) as t(a) +GROUP BY a +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT count(*) +FROM VALUES (ARRAY(named_struct('b', map(1, 2, 2, 3)), named_struct('b', map(1, 3)))), (ARRAY(named_struct('b', map(2, 3)), named_struct('b', map(1, 3)))), (ARRAY(named_struct('b', map(2, 3, 1, 2)), named_struct('b', map(1, 3)))) as t(a) +group BY a +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT count(*) +FROM VALUES (map(array(1, 2), 2, array(2, 3), 3)), (map(array(1, 3), 3)), (map(array(2, 3), 3, array(1, 2), 2)) as t(a) +group BY a +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT count(*) +FROM VALUES (map(array(1, 2, 3), 3)), (map(array(3, 2, 1), 3)) as t(a) +group BY a +-- !query schema +struct +-- !query output +1 +1 + + +-- !query +SELECT count(*) +FROM VALUES (ARRAY(map(array(1, 2), 2, array(2, 3), 3))), (ARRAY(MAP(ARRAY(1, 3), 3))), (ARRAY(map(array(2, 3), 3, array(1, 2), 2))) as t(a) +group BY a +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT count(*) +FROM VALUES (map(named_struct('b', 1), 2, named_struct('b', 2), 3)), (map(named_struct('b', 1), 3)), (map(named_struct('b', 2), 3, named_struct('b', 1), 2)) as t(a) +group BY a +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT count(*) +FROM VALUES (named_struct('b', map(named_struct('c', 1), 2, named_struct('c', 2), 3))), (named_struct('b', map(named_struct('c', 1), 3))), (named_struct('b', map(named_struct('c', 2), 3, named_struct('c', 1), 2))) as t(a) +group BY a +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT count(*) +FROM VALUES (named_struct('b', map(named_struct('c', 1), 2, named_struct('c', 2), 3))), (named_struct('b', map(named_struct('c', 1), 3))), (named_struct('b', map(named_struct('c', 2), 3, named_struct('c', 1), 2))) as t(a) +group BY a.b +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT count(*) +FROM VALUES (Map(1, Map(1,2), 2, Map(2, 3, 1, 2))), (Map(2, Map(1, 2, 2,3), 1, Map(1, 2))), (Map(1, Map(1,2), 2, Map(2, 4))) as t(a) +GROUP BY a +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT count(*) +FROM VALUES (Map(1, Array(Map(1,2)), 2, Array(Map(2, 3, 1, 2)))), (Map(2, Array(Map(1, 2, 2,3)), 1, Array(Map(1, 2)))), (Map(1, Array(Map(1,2)), 2, Array(Map(2, 4)))) as t(a) +GROUP BY a +-- !query schema +struct +-- !query output +1 +2 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/grouping_set.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/grouping_set.sql.out new file mode 100644 index 000000000000..5849acd094fb --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/grouping_set.sql.out @@ -0,0 +1,244 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW grouping AS SELECT * FROM VALUES + ("1", "2", "3", 1), + ("4", "5", "6", 1), + ("7", "8", "9", 1) + as grouping(a, b, c, d) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT a, b, c, count(d) FROM grouping GROUP BY a, b, c GROUPING SETS (()) +-- !query schema +struct +-- !query output +NULL NULL NULL 3 + + +-- !query +SELECT a, b, c, count(d) FROM grouping GROUP BY a, b, c GROUPING SETS ((a)) +-- !query schema +struct +-- !query output +1 NULL NULL 1 +4 NULL NULL 1 +7 NULL NULL 1 + + +-- !query +SELECT a, b, c, count(d) FROM grouping GROUP BY a, b, c GROUPING SETS ((c)) +-- !query schema +struct +-- !query output +NULL NULL 3 1 +NULL NULL 6 1 +NULL NULL 9 1 + + +-- !query +SELECT c1, sum(c2) FROM (VALUES ('x', 10, 0), ('y', 20, 0)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1) +-- !query schema +struct +-- !query output +x 10 +y 20 + + +-- !query +SELECT c1, sum(c2), grouping(c1) FROM (VALUES ('x', 10, 0), ('y', 20, 0)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1) +-- !query schema +struct +-- !query output +x 10 0 +y 20 0 + + +-- !query +SELECT c1, c2, Sum(c3), grouping__id +FROM (VALUES ('x', 'a', 10), ('y', 'b', 20) ) AS t (c1, c2, c3) +GROUP BY GROUPING SETS ( ( c1 ), ( c2 ) ) +HAVING GROUPING__ID > 1 +-- !query schema +struct +-- !query output +NULL a 10 2 +NULL b 20 2 + + +-- !query +SELECT grouping(c1) FROM (VALUES ('x', 'a', 10), ('y', 'b', 20)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1,c2) +-- !query schema +struct +-- !query output +0 +0 +1 +1 + + +-- !query +SELECT -c1 AS c1 FROM (values (1,2), (3,2)) t(c1, c2) GROUP BY GROUPING SETS ((c1), (c1, c2)) +-- !query schema +struct +-- !query output +-1 +-1 +-3 +-3 + + +-- !query +SELECT a + b, b, sum(c) FROM (VALUES (1,1,1),(2,2,2)) AS t(a,b,c) GROUP BY GROUPING SETS ( (a + b), (b)) +-- !query schema +struct<(a + b):int,b:int,sum(c):bigint> +-- !query output +2 NULL 1 +4 NULL 2 +NULL 1 1 +NULL 2 2 + + +-- !query +SELECT a + b, b, sum(c) FROM (VALUES (1,1,1),(2,2,2)) AS t(a,b,c) GROUP BY GROUPING SETS ( (a + b), (b + a), (b)) +-- !query schema +struct<(a + b):int,b:int,sum(c):bigint> +-- !query output +2 NULL 1 +2 NULL 1 +4 NULL 2 +4 NULL 2 +NULL 1 1 +NULL 2 2 + + +-- !query +SELECT c1 AS col1, c2 AS col2 +FROM (VALUES (1, 2), (3, 2)) t(c1, c2) +GROUP BY GROUPING SETS ( ( c1 ), ( c1, c2 ) ) +HAVING col2 IS NOT NULL +ORDER BY -col1 +-- !query schema +struct +-- !query output +3 2 +1 2 + + +-- !query +SELECT a, b, c, count(d) FROM grouping GROUP BY WITH ROLLUP +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'ROLLUP'", + "hint" : ": extra input 'ROLLUP'" + } +} + + +-- !query +SELECT a, b, c, count(d) FROM grouping GROUP BY WITH CUBE +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'CUBE'", + "hint" : ": extra input 'CUBE'" + } +} + + +-- !query +SELECT c1 FROM (values (1,2), (3,2)) t(c1, c2) GROUP BY GROUPING SETS (()) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"c1\"", + "expressionAnyValue" : "\"any_value(c1)\"" + } +} + + +-- !query +SELECT k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)) +-- !query schema +struct +-- !query output +1 1 1.0 +1 1 1.0 +1 NULL 1.0 +2 2 2.0 +2 2 2.0 +2 NULL 2.0 + + +-- !query +SELECT grouping__id, k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)) +-- !query schema +struct +-- !query output +0 1 1 1.0 +0 1 1 1.0 +0 2 2 2.0 +0 2 2 2.0 +1 1 NULL 1.0 +1 2 NULL 2.0 + + +-- !query +SELECT grouping(k1), k1, k2, avg(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)) +-- !query schema +struct +-- !query output +0 1 1 1.0 +0 1 1 1.0 +0 1 NULL 1.0 +0 2 2 2.0 +0 2 2 2.0 +0 2 NULL 2.0 + + +-- !query +SELECT grouping_id(k1, k2), avg(v) from (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY k1, k2 GROUPING SETS ((k2, k1), k1) +-- !query schema +struct +-- !query output +0 1.0 +0 2.0 +1 1.0 +1 2.0 + + +-- !query +SELECT CASE WHEN a IS NULL THEN count(b) WHEN b IS NULL THEN count(c) END +FROM grouping +GROUP BY GROUPING SETS (a, b, c) +-- !query schema +struct +-- !query output +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/having.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/having.sql.out new file mode 100644 index 000000000000..c9d588642636 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/having.sql.out @@ -0,0 +1,154 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view hav as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", 5) + as hav(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2 +-- !query schema +struct +-- !query output +one 6 +three 3 + + +-- !query +SELECT count(k) FROM hav GROUP BY v + 1 HAVING v + 1 = 2 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT count(k) FROM hav GROUP BY v HAVING v = array(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"ARRAY\"", + "sqlExpr" : "\"(v = array(1))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 55, + "fragment" : "v = array(1)" + } ] +} + + +-- !query +SELECT MIN(t.v) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(COUNT(1) > 0) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT a + b FROM VALUES (1L, 2), (3L, 4) AS T(a, b) GROUP BY a + b HAVING a + b > 1 +-- !query schema +struct<(a + b):bigint> +-- !query output +3 +7 + + +-- !query +SELECT SUM(a) AS b, CAST('2020-01-01' AS DATE) AS fake FROM VALUES (1, 10), (2, 20) AS T(a, b) GROUP BY b HAVING b > 10 +-- !query schema +struct +-- !query output +2 2020-01-01 + + +-- !query +SELECT SUM(a) AS b FROM VALUES (1, 10), (2, 20) AS T(a, b) GROUP BY GROUPING SETS ((b), (a, b)) HAVING b > 10 +-- !query schema +struct +-- !query output +2 +2 + + +-- !query +SELECT SUM(a) AS b FROM VALUES (1, 10), (2, 20) AS T(a, b) GROUP BY CUBE(a, b) HAVING b > 10 +-- !query schema +struct +-- !query output +2 +2 + + +-- !query +SELECT SUM(a) AS b FROM VALUES (1, 10), (2, 20) AS T(a, b) GROUP BY ROLLUP(a, b) HAVING b > 10 +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT c1 FROM VALUES (1, 2) as t(c1, c2) GROUP BY GROUPING SETS(t.c1) HAVING t.c1 = 1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT c1 FROM VALUES (1, 2) as t(c1, c2) GROUP BY CUBE(t.c1) HAVING t.c1 = 1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT c1 FROM VALUES (1, 2) as t(c1, c2) GROUP BY ROLLUP(t.c1) HAVING t.c1 = 1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT c1 FROM VALUES (1, 2) as t(c1, c2) GROUP BY t.c1 HAVING t.c1 = 1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2 ORDER BY sum(v) +-- !query schema +struct +-- !query output +three 3 +one 6 + + +-- !query +SELECT k, sum(v) FROM hav GROUP BY k HAVING sum(v) > 2 ORDER BY avg(v) +-- !query schema +struct +-- !query output +one 6 +three 3 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/higher-order-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/higher-order-functions.sql.out new file mode 100644 index 000000000000..7bfc35a61e09 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/higher-order-functions.sql.out @@ -0,0 +1,350 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create or replace temporary view nested as values + (1, array(32, 97), array(array(12, 99), array(123, 42), array(1))), + (2, array(77, -76), array(array(6, 96, 65), array(-1, -2))), + (3, array(12), array(array(17))) + as t(x, ys, zs) +-- !query schema +struct<> +-- !query output + + + +-- !query +select upper(x -> x) as v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_LAMBDA_FUNCTION_CALL.NON_HIGHER_ORDER_FUNCTION", + "sqlState" : "42K0D", + "messageParameters" : { + "class" : "org.apache.spark.sql.catalyst.expressions.Upper" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "upper(x -> x)" + } ] +} + + +-- !query +select ceil(x -> x) as v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_LAMBDA_FUNCTION_CALL.NON_HIGHER_ORDER_FUNCTION", + "sqlState" : "42K0D", + "messageParameters" : { + "class" : "org.apache.spark.sql.catalyst.expressions.CeilExpressionBuilder$" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "ceil(x -> x)" + } ] +} + + +-- !query +select transform(zs, z -> z) as v from nested +-- !query schema +struct>> +-- !query output +[[12,99],[123,42],[1]] +[[17]] +[[6,96,65],[-1,-2]] + + +-- !query +select transform(ys, y -> y * y) as v from nested +-- !query schema +struct> +-- !query output +[1024,9409] +[144] +[5929,5776] + + +-- !query +select transform(ys, (y, i) -> y + i) as v from nested +-- !query schema +struct> +-- !query output +[12] +[32,98] +[77,-75] + + +-- !query +select transform(zs, z -> concat(ys, z)) as v from nested +-- !query schema +struct>> +-- !query output +[[12,17]] +[[32,97,12,99],[32,97,123,42],[32,97,1]] +[[77,-76,6,96,65],[77,-76,-1,-2]] + + +-- !query +select transform(ys, 0) as v from nested +-- !query schema +struct> +-- !query output +[0,0] +[0,0] +[0] + + +-- !query +select transform(cast(null as array), x -> x + 1) as v +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select filter(ys, y -> y > 30) as v from nested +-- !query schema +struct> +-- !query output +[32,97] +[77] +[] + + +-- !query +select filter(cast(null as array), y -> true) as v +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select transform(zs, z -> filter(z, zz -> zz > 50)) as v from nested +-- !query schema +struct>> +-- !query output +[[96,65],[]] +[[99],[123],[]] +[[]] + + +-- !query +select aggregate(ys, 0, (y, a) -> y + a + x) as v from nested +-- !query schema +struct +-- !query output +131 +15 +5 + + +-- !query +select aggregate(ys, (0 as sum, 0 as n), (acc, x) -> (acc.sum + x, acc.n + 1), acc -> acc.sum / acc.n) as v from nested +-- !query schema +struct +-- !query output +0.5 +12.0 +64.5 + + +-- !query +select transform(zs, z -> aggregate(z, 1, (acc, val) -> acc * val * size(z))) as v from nested +-- !query schema +struct> +-- !query output +[1010880,8] +[17] +[4752,20664,1] + + +-- !query +select aggregate(cast(null as array), 0, (a, y) -> a + y + 1, a -> a + 2) as v +-- !query schema +struct +-- !query output +NULL + + +-- !query +select reduce(ys, 0, (y, a) -> y + a + x) as v from nested +-- !query schema +struct +-- !query output +131 +15 +5 + + +-- !query +select reduce(ys, (0 as sum, 0 as n), (acc, x) -> (acc.sum + x, acc.n + 1), acc -> acc.sum / acc.n) as v from nested +-- !query schema +struct +-- !query output +0.5 +12.0 +64.5 + + +-- !query +select transform(zs, z -> reduce(z, 1, (acc, val) -> acc * val * size(z))) as v from nested +-- !query schema +struct> +-- !query output +[1010880,8] +[17] +[4752,20664,1] + + +-- !query +select reduce(cast(null as array), 0, (a, y) -> a + y + 1, a -> a + 2) as v +-- !query schema +struct +-- !query output +NULL + + +-- !query +select exists(ys, y -> y > 30) as v from nested +-- !query schema +struct +-- !query output +false +true +true + + +-- !query +select exists(cast(null as array), y -> y > 30) as v +-- !query schema +struct +-- !query output +NULL + + +-- !query +select zip_with(ys, zs, (a, b) -> a + size(b)) as v from nested +-- !query schema +struct> +-- !query output +[13] +[34,99,null] +[80,-74] + + +-- !query +select zip_with(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) -> concat(x, y)) as v +-- !query schema +struct> +-- !query output +["ad","be","cf"] + + +-- !query +select zip_with(array('a'), array('d', null, 'f'), (x, y) -> coalesce(x, y)) as v +-- !query schema +struct> +-- !query output +["a",null,"f"] + + +-- !query +create or replace temporary view nested as values + (1, map(1, 1, 2, 2, 3, 3)), + (2, map(4, 4, 5, 5, 6, 6)) + as t(x, ys) +-- !query schema +struct<> +-- !query output + + + +-- !query +select transform_keys(ys, (k, v) -> k) as v from nested +-- !query schema +struct> +-- !query output +{1:1,2:2,3:3} +{4:4,5:5,6:6} + + +-- !query +select transform_keys(ys, (k, v) -> k + 1) as v from nested +-- !query schema +struct> +-- !query output +{2:1,3:2,4:3} +{5:4,6:5,7:6} + + +-- !query +select transform_keys(ys, (k, v) -> k + v) as v from nested +-- !query schema +struct> +-- !query output +{10:5,12:6,8:4} +{2:1,4:2,6:3} + + +-- !query +select transform_values(ys, (k, v) -> v) as v from nested +-- !query schema +struct> +-- !query output +{1:1,2:2,3:3} +{4:4,5:5,6:6} + + +-- !query +select transform_values(ys, (k, v) -> v + 1) as v from nested +-- !query schema +struct> +-- !query output +{1:2,2:3,3:4} +{4:5,5:6,6:7} + + +-- !query +select transform_values(ys, (k, v) -> k + v) as v from nested +-- !query schema +struct> +-- !query output +{1:2,2:4,3:6} +{4:8,5:10,6:12} + + +-- !query +select transform(ys, all -> all * all) as v from values (array(32, 97)) as t(ys) +-- !query schema +struct> +-- !query output +[1024,9409] + + +-- !query +select transform(ys, (all, i) -> all + i) as v from values (array(32, 97)) as t(ys) +-- !query schema +struct> +-- !query output +[32,98] + + +-- !query +select aggregate(split('abcdefgh',''), array(array('')), (acc, x) -> array(array(x))) +-- !query schema +struct>> +-- !query output +[["h"]] diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/hll.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/hll.sql.out new file mode 100644 index 000000000000..ea9dc8e8af17 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/hll.sql.out @@ -0,0 +1,269 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +DROP TABLE IF EXISTS t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t1 USING JSON AS VALUES (0), (1), (2), (2), (2), (3), (4) as tab(col) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT hll_sketch_estimate(hll_sketch_agg(col)) AS result FROM t1 +-- !query schema +struct +-- !query output +5 + + +-- !query +SELECT hll_sketch_estimate(hll_sketch_agg(col, 12)) +FROM VALUES (50), (60), (60), (60), (75), (100) tab(col) +-- !query schema +struct +-- !query output +4 + + +-- !query +SELECT hll_sketch_estimate(hll_sketch_agg(col)) +FROM VALUES ('abc'), ('def'), ('abc'), ('ghi'), ('abc') tab(col) +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT hll_sketch_estimate( + hll_union( + hll_sketch_agg(col1), + hll_sketch_agg(col2))) + FROM VALUES + (1, 4), + (1, 4), + (2, 5), + (2, 5), + (3, 6) AS tab(col1, col2) +-- !query schema +struct +-- !query output +6 + + +-- !query +SELECT hll_sketch_estimate(hll_union_agg(sketch, true)) + FROM (SELECT hll_sketch_agg(col) as sketch + FROM VALUES (1) AS tab(col) + UNION ALL + SELECT hll_sketch_agg(col, 20) as sketch + FROM VALUES (1) AS tab(col)) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT hll_sketch_agg(col) +FROM VALUES (ARRAY(1, 2)), (ARRAY(3, 4)) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"col\"", + "inputType" : "\"ARRAY\"", + "paramIndex" : "first", + "requiredType" : "(\"INT\" or \"BIGINT\" or \"STRING\" or \"BINARY\")", + "sqlExpr" : "\"hll_sketch_agg(col, 12)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "hll_sketch_agg(col)" + } ] +} + + +-- !query +SELECT hll_sketch_agg(col, 2) +FROM VALUES (50), (60), (60) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "HLL_INVALID_LG_K", + "sqlState" : "22546", + "messageParameters" : { + "function" : "`hll_sketch_agg`", + "max" : "21", + "min" : "4", + "value" : "2" + } +} + + +-- !query +SELECT hll_sketch_agg(col, 40) +FROM VALUES (50), (60), (60) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "HLL_INVALID_LG_K", + "sqlState" : "22546", + "messageParameters" : { + "function" : "`hll_sketch_agg`", + "max" : "21", + "min" : "4", + "value" : "40" + } +} + + +-- !query +SELECT hll_union( + hll_sketch_agg(col1, 12), + hll_sketch_agg(col2, 13)) + FROM VALUES + (1, 4), + (1, 4), + (2, 5), + (2, 5), + (3, 6) AS tab(col1, col2) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "HLL_UNION_DIFFERENT_LG_K", + "sqlState" : "22000", + "messageParameters" : { + "function" : "`hll_union`", + "left" : "12", + "right" : "13" + } +} + + +-- !query +SELECT hll_union_agg(sketch, false) +FROM (SELECT hll_sketch_agg(col, 12) as sketch + FROM VALUES (1) AS tab(col) + UNION ALL + SELECT hll_sketch_agg(col, 20) as sketch + FROM VALUES (1) AS tab(col)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "HLL_UNION_DIFFERENT_LG_K", + "sqlState" : "22000", + "messageParameters" : { + "function" : "`hll_union_agg`", + "left" : "12", + "right" : "20" + } +} + + +-- !query +SELECT hll_union(1, 2) + FROM VALUES + (1, 4), + (1, 4), + (2, 5), + (2, 5), + (3, 6) AS tab(col1, col2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "\"BINARY\"", + "sqlExpr" : "\"hll_union(1, 2, false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "hll_union(1, 2)" + } ] +} + + +-- !query +SELECT hll_sketch_estimate(CAST ('abc' AS BINARY)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "HLL_INVALID_INPUT_SKETCH_BUFFER", + "sqlState" : "22546", + "messageParameters" : { + "function" : "`hll_sketch_estimate`" + } +} + + +-- !query +SELECT hll_union(CAST ('abc' AS BINARY), CAST ('def' AS BINARY)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "HLL_INVALID_INPUT_SKETCH_BUFFER", + "sqlState" : "22546", + "messageParameters" : { + "function" : "`hll_union`" + } +} + + +-- !query +SELECT hll_union_agg(buffer, false) +FROM (SELECT CAST('abc' AS BINARY) AS buffer) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "HLL_INVALID_INPUT_SKETCH_BUFFER", + "sqlState" : "22546", + "messageParameters" : { + "function" : "`hll_union_agg`" + } +} + + +-- !query +DROP TABLE IF EXISTS t1 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/identifier-clause.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/identifier-clause.sql.out new file mode 100644 index 000000000000..596745b4ba5d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/identifier-clause.sql.out @@ -0,0 +1,1405 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SET hivevar:colname = 'c' +-- !query schema +struct +-- !query output +hivevar:colname 'c' + + +-- !query +SELECT IDENTIFIER(${colname} || '_1') FROM VALUES(1) AS T(c_1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('c1') FROM VALUES(1) AS T(c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('t.c1') FROM VALUES(1) AS T(c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('`t`.c1') FROM VALUES(1) AS T(c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('`c 1`') FROM VALUES(1) AS T(`c 1`) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('``') FROM VALUES(1) AS T(``) +-- !query schema +struct<:int> +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('c' || '1') FROM VALUES(1) AS T(c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +CREATE SCHEMA IF NOT EXISTS s +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE s.tab(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +USE SCHEMA s +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO IDENTIFIER('ta' || 'b') VALUES(1) +-- !query schema +struct<> +-- !query output + + + +-- !query +DELETE FROM IDENTIFIER('ta' || 'b') WHERE 1=0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "DELETE", + "tableName" : "`spark_catalog`.`s`.`tab`" + } +} + + +-- !query +UPDATE IDENTIFIER('ta' || 'b') SET c1 = 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUnsupportedOperationException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2096", + "messageParameters" : { + "ddl" : "UPDATE TABLE" + } +} + + +-- !query +MERGE INTO IDENTIFIER('ta' || 'b') AS t USING IDENTIFIER('ta' || 'b') AS s ON s.c1 = t.c1 + WHEN MATCHED THEN UPDATE SET c1 = 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUnsupportedOperationException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2096", + "messageParameters" : { + "ddl" : "MERGE INTO TABLE" + } +} + + +-- !query +SELECT * FROM IDENTIFIER('tab') +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM IDENTIFIER('s.tab') +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM IDENTIFIER('`s`.`tab`') +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM IDENTIFIER('t' || 'a' || 'b') +-- !query schema +struct +-- !query output +1 + + +-- !query +USE SCHEMA default +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE s.tab +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP SCHEMA s +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT IDENTIFIER('COAL' || 'ESCE')(NULL, 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT IDENTIFIER('abs')(c1) FROM VALUES(-1) AS T(c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM IDENTIFIER('ra' || 'nge')(0, 1) +-- !query schema +struct +-- !query output +0 + + +-- !query +CREATE TABLE IDENTIFIER('tab')(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE SCHEMA identifier_clauses +-- !query schema +struct<> +-- !query output + + + +-- !query +USE identifier_clauses +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE IDENTIFIER('ta' || 'b')(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS IDENTIFIER('identifier_clauses.' || 'tab') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +REPLACE TABLE IDENTIFIER('identifier_clauses.' || 'tab')(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION", + "sqlState" : "0A000", + "messageParameters" : { + "operation" : "REPLACE TABLE", + "tableName" : "`spark_catalog`.`identifier_clauses`.`tab`" + } +} + + +-- !query +CACHE TABLE IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +UNCACHE TABLE IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +USE default +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP SCHEMA identifier_clauses +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tab(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO tab VALUES (1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT c1 FROM tab +-- !query schema +struct +-- !query output +1 + + +-- !query +DESCRIBE IDENTIFIER('ta' || 'b') +-- !query schema +struct +-- !query output +c1 int + + +-- !query +ANALYZE TABLE IDENTIFIER('ta' || 'b') COMPUTE STATISTICS +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER TABLE IDENTIFIER('ta' || 'b') ADD COLUMN c2 INT +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW TBLPROPERTIES IDENTIFIER('ta' || 'b') +-- !query schema +struct +-- !query output + + + +-- !query +SHOW COLUMNS FROM IDENTIFIER('ta' || 'b') +-- !query schema +struct +-- !query output +c1 +c2 + + +-- !query +COMMENT ON TABLE IDENTIFIER('ta' || 'b') IS 'hello' +-- !query schema +struct<> +-- !query output + + + +-- !query +REFRESH TABLE IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +REPAIR TABLE IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_PARTITIONED_TABLE", + "sqlState" : "42809", + "messageParameters" : { + "operation" : "MSCK REPAIR TABLE", + "tableIdentWithDB" : "`spark_catalog`.`default`.`tab`" + } +} + + +-- !query +TRUNCATE TABLE IDENTIFIER('ta' || 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS tab +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW IDENTIFIER('v')(c1) AS VALUES(1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 + + +-- !query +ALTER VIEW IDENTIFIER('v') AS VALUES(2) +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IDENTIFIER('v') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW IDENTIFIER('v')(c1) AS VALUES(1) +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IDENTIFIER('v') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE SCHEMA IDENTIFIER('id' || 'ent') +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER SCHEMA IDENTIFIER('id' || 'ent') SET PROPERTIES (somekey = 'somevalue') +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER SCHEMA IDENTIFIER('id' || 'ent') SET LOCATION 'someloc' +-- !query schema +struct<> +-- !query output + + + +-- !query +COMMENT ON SCHEMA IDENTIFIER('id' || 'ent') IS 'some comment' +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE SCHEMA IDENTIFIER('id' || 'ent') +-- !query schema +struct +-- !query output +Catalog Name spark_catalog +Comment some comment +Location [not included in comparison]/{warehouse_dir}/someloc +Namespace Name ident +Owner [not included in comparison] + + +-- !query +SHOW TABLES IN IDENTIFIER('id' || 'ent') +-- !query schema +struct +-- !query output + + + +-- !query +SHOW TABLE EXTENDED IN IDENTIFIER('id' || 'ent') LIKE 'hello' +-- !query schema +struct +-- !query output + + + +-- !query +USE IDENTIFIER('id' || 'ent') +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW CURRENT SCHEMA +-- !query schema +struct +-- !query output +spark_catalog ident + + +-- !query +USE SCHEMA IDENTIFIER('id' || 'ent') +-- !query schema +struct<> +-- !query output + + + +-- !query +USE SCHEMA default +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP SCHEMA IDENTIFIER('id' || 'ent') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE SCHEMA ident +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') +-- !query schema +struct +-- !query output +Class: test.org.apache.spark.sql.MyDoubleAvg +Function: spark_catalog.ident.mydoubleavg +Usage: N/A. + + +-- !query +REFRESH FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IDENTIFIER('ident.' || 'myDoubleAvg') +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP SCHEMA ident +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY FUNCTION IDENTIFIER('my' || 'DoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY FUNCTION IDENTIFIER('my' || 'DoubleAvg') +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE var = 'sometable' +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE IDENTIFIER(var)(c1 INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VAR var = 'c1' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT IDENTIFIER(var) FROM VALUES(1) AS T(c1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SET VAR var = 'some' +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IDENTIFIER(var || 'table') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT IDENTIFIER('c 1') FROM VALUES(1) AS T(`c 1`) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'1'", + "hint" : ": extra input '1'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "IDENTIFIER('c 1')" + } ] +} + + +-- !query +SELECT IDENTIFIER('') FROM VALUES(1) AS T(``) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_EMPTY_STATEMENT", + "sqlState" : "42617", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "IDENTIFIER('')" + } ] +} + + +-- !query +VALUES(IDENTIFIER(CAST(NULL AS STRING))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NULL", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "CAST(NULL AS STRING)", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 38, + "fragment" : "CAST(NULL AS STRING)" + } ] +} + + +-- !query +VALUES(IDENTIFIER(1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.WRONG_TYPE", + "sqlState" : "42601", + "messageParameters" : { + "dataType" : "int", + "expr" : "1", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 19, + "fragment" : "1" + } ] +} + + +-- !query +VALUES(IDENTIFIER(SUBSTR('HELLO', 1, RAND() + 1))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.NOT_CONSTANT", + "sqlState" : "42601", + "messageParameters" : { + "expr" : "substr('HELLO', 1, CAST((rand() + CAST(1 AS DOUBLE)) AS INT))", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 48, + "fragment" : "SUBSTR('HELLO', 1, RAND() + 1)" + } ] +} + + +-- !query +SELECT `IDENTIFIER`('abs')(c1) FROM VALUES(-1) AS T(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`IDENTIFIER`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "`IDENTIFIER`('abs')" + } ] +} + + +-- !query +CREATE TABLE IDENTIFIER(1)(c1 INT) USING csv +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.WRONG_TYPE", + "sqlState" : "42601", + "messageParameters" : { + "dataType" : "int", + "expr" : "1", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 25, + "fragment" : "1" + } ] +} + + +-- !query +CREATE TABLE IDENTIFIER('a.b.c')(c1 INT) USING csv +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +CREATE VIEW IDENTIFIER('a.b.c')(c1) AS VALUES(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +DROP TABLE IDENTIFIER('a.b.c') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +DROP VIEW IDENTIFIER('a.b.c') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +COMMENT ON TABLE IDENTIFIER('a.b.c.d') IS 'hello' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRES_SINGLE_PART_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespace" : "`a`.`b`.`c`", + "sessionCatalog" : "spark_catalog" + } +} + + +-- !query +VALUES(IDENTIFIER(1)()) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_CONSTANT_STRING.WRONG_TYPE", + "sqlState" : "42601", + "messageParameters" : { + "dataType" : "int", + "expr" : "1", + "name" : "IDENTIFIER" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 19, + "fragment" : "1" + } ] +} + + +-- !query +VALUES(IDENTIFIER('a.b.c.d')()) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "IDENTIFIER_TOO_MANY_NAME_PARTS", + "sqlState" : "42601", + "messageParameters" : { + "identifier" : "`a`.`b`.`c`.`d`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "IDENTIFIER('a.b.c.d')()" + } ] +} + + +-- !query +CREATE TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.CREATE_TEMP_FUNC_WITH_DATABASE", + "sqlState" : "42000", + "messageParameters" : { + "database" : "`default`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 108, + "fragment" : "CREATE TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg') AS 'test.org.apache.spark.sql.MyDoubleAvg'" + } ] +} + + +-- !query +DROP TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.MULTI_PART_NAME", + "sqlState" : "42000", + "messageParameters" : { + "name" : "`default`.`myDoubleAvg`", + "statement" : "DROP TEMPORARY FUNCTION" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "DROP TEMPORARY FUNCTION IDENTIFIER('default.my' || 'DoubleAvg')" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW IDENTIFIER('default.v')(c1) AS VALUES(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "TEMP_VIEW_NAME_TOO_MANY_NAME_PARTS", + "sqlState" : "428EK", + "messageParameters" : { + "actualName" : "`default`.`v`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "CREATE TEMPORARY VIEW IDENTIFIER('default.v')(c1) AS VALUES(1)" + } ] +} + + +-- !query +create temporary view identifier('v1') as (select my_col from (values (1), (2), (1) as (my_col)) group by 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +cache table identifier('t1') as (select my_col from (values (1), (2), (1) as (my_col)) group by 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table identifier('t2') using csv as (select my_col from (values (1), (2), (1) as (my_col)) group by 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into identifier('t2') select my_col from (values (3) as (my_col)) group by 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop view v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE agg = 'max' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE col = 'c1' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE tab = 'T' +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), + T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) +SELECT IDENTIFIER(agg)(IDENTIFIER(col)) FROM IDENTIFIER(tab) +-- !query schema +struct +-- !query output +c + + +-- !query +WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), + T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) +SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('T') +-- !query schema +struct +-- !query output +c + + +-- !query +WITH ABC(c1, c2) AS (VALUES(1, 2), (2, 3)) +SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('A' || 'BC') +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT row_number() OVER IDENTIFIER('x.win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''x.win''", + "hint" : "" + } +} + + +-- !query +SELECT T1.c1 FROM VALUES(1) AS T1(c1) JOIN VALUES(1) AS T2(c1) USING (IDENTIFIER('c1')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT IDENTIFIER('t').c1 FROM VALUES(1) AS T(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t`", + "proposal" : "`c1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "IDENTIFIER('t')" + } ] +} + + +-- !query +SELECT map('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''a''", + "hint" : "" + } +} + + +-- !query +SELECT named_struct('a', 1).IDENTIFIER('a') FROM VALUES(1) AS T(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''a''", + "hint" : "" + } +} + + +-- !query +SELECT * FROM s.IDENTIFIER('tab') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.INVALID_TABLE_VALUED_FUNC_NAME", + "sqlState" : "42000", + "messageParameters" : { + "funcName" : "`s`.`IDENTIFIER`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 33, + "fragment" : "s.IDENTIFIER('tab')" + } ] +} + + +-- !query +SELECT * FROM IDENTIFIER('s').IDENTIFIER('tab') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'.'", + "hint" : "" + } +} + + +-- !query +SELECT * FROM IDENTIFIER('s').tab +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'.'", + "hint" : "" + } +} + + +-- !query +SELECT row_number() OVER IDENTIFIER('win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''win''", + "hint" : "" + } +} + + +-- !query +SELECT row_number() OVER win FROM VALUES(1) AS T(c1) WINDOW IDENTIFIER('win') AS (ORDER BY c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'WINDOW'", + "hint" : "" + } +} + + +-- !query +WITH identifier('v')(identifier('c1')) AS (VALUES(1)) (SELECT c1 FROM v) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "''v''", + "hint" : "" + } +} + + +-- !query +INSERT INTO tab(IDENTIFIER('c1')) VALUES(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : ": missing ')'" + } +} + + +-- !query +CREATE OR REPLACE VIEW v(IDENTIFIER('c1')) AS VALUES(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +CREATE TABLE tab(IDENTIFIER('c1') INT) USING CSV +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/ilike-all.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/ilike-all.sql.out new file mode 100644 index 000000000000..96dfe0f97c3c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/ilike-all.sql.out @@ -0,0 +1,144 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW ilike_all_table AS SELECT * FROM (VALUES + ('gOOgle', '%oo%'), + ('facebook', '%OO%'), + ('liNkedin', '%In')) + as t1(company, pat) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT company FROM ilike_all_table WHERE company ILIKE ALL ('%oO%', '%Go%') +-- !query schema +struct +-- !query output +gOOgle + + +-- !query +SELECT company FROM ilike_all_table WHERE company ILIKE ALL ('microsoft', '%yoo%') +-- !query schema +struct +-- !query output + + + +-- !query +SELECT + company, + CASE + WHEN company ILIKE ALL ('%oo%', '%GO%') THEN 'Y' + ELSE 'N' + END AS is_available, + CASE + WHEN company ILIKE ALL ('%OO%', 'go%') OR company ILIKE ALL ('%IN', 'ms%') THEN 'Y' + ELSE 'N' + END AS mix +FROM ilike_all_table +-- !query schema +struct +-- !query output +facebook N N +gOOgle Y Y +liNkedin N N + + +-- !query +SELECT company FROM ilike_all_table WHERE company ILIKE ALL ('%oo%', pat) +-- !query schema +struct +-- !query output +facebook +gOOgle + + +-- !query +SELECT company FROM ilike_all_table WHERE company NOT ILIKE ALL ('%oo%', '%In', 'Fa%') +-- !query schema +struct +-- !query output + + + +-- !query +SELECT company FROM ilike_all_table WHERE company NOT ILIKE ALL ('microsoft', '%yoo%') +-- !query schema +struct +-- !query output +facebook +gOOgle +liNkedin + + +-- !query +SELECT company FROM ilike_all_table WHERE company NOT ILIKE ALL ('%oo%', 'fA%') +-- !query schema +struct +-- !query output +liNkedin + + +-- !query +SELECT company FROM ilike_all_table WHERE NOT company ILIKE ALL ('%oO%', 'fa%') +-- !query schema +struct +-- !query output +gOOgle +liNkedin + + +-- !query +SELECT company FROM ilike_all_table WHERE company ILIKE ALL ('%OO%', NULL) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT company FROM ilike_all_table WHERE company NOT ILIKE ALL ('%Oo%', NULL) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT company FROM ilike_all_table WHERE company ILIKE ALL (NULL, NULL) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT company FROM ilike_all_table WHERE company NOT ILIKE ALL (NULL, NULL) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT company FROM ilike_any_table WHERE company ILIKE ALL () +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Expected something between '(' and ')'." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 62, + "fragment" : "ILIKE ALL ()" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/ilike-any.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/ilike-any.sql.out new file mode 100644 index 000000000000..c8bd38d32462 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/ilike-any.sql.out @@ -0,0 +1,150 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW ilike_any_table AS SELECT * FROM (VALUES + ('Google', '%Oo%'), + ('FaceBook', '%oO%'), + ('linkedIn', '%IN')) + as t1(company, pat) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT company FROM ilike_any_table WHERE company ILIKE ANY ('%oo%', '%IN', 'fA%') +-- !query schema +struct +-- !query output +FaceBook +Google +linkedIn + + +-- !query +SELECT company FROM ilike_any_table WHERE company ILIKE ANY ('microsoft', '%yoo%') +-- !query schema +struct +-- !query output + + + +-- !query +select + company, + CASE + WHEN company ILIKE ANY ('%oO%', '%IN', 'Fa%') THEN 'Y' + ELSE 'N' + END AS is_available, + CASE + WHEN company ILIKE ANY ('%OO%', 'fa%') OR company ILIKE ANY ('%in', 'MS%') THEN 'Y' + ELSE 'N' + END AS mix +FROM ilike_any_table +-- !query schema +struct +-- !query output +FaceBook Y Y +Google Y Y +linkedIn Y Y + + +-- !query +SELECT company FROM ilike_any_table WHERE company ILIKE ANY ('%zZ%', pat) +-- !query schema +struct +-- !query output +FaceBook +Google +linkedIn + + +-- !query +SELECT company FROM ilike_any_table WHERE company NOT ILIKE ANY ('%oO%', '%iN', 'fa%') +-- !query schema +struct +-- !query output +FaceBook +Google +linkedIn + + +-- !query +SELECT company FROM ilike_any_table WHERE company NOT ILIKE ANY ('microsoft', '%yOo%') +-- !query schema +struct +-- !query output +FaceBook +Google +linkedIn + + +-- !query +SELECT company FROM ilike_any_table WHERE company NOT ILIKE ANY ('%oo%', 'Fa%') +-- !query schema +struct +-- !query output +Google +linkedIn + + +-- !query +SELECT company FROM ilike_any_table WHERE NOT company ILIKE ANY ('%OO%', 'fa%') +-- !query schema +struct +-- !query output +linkedIn + + +-- !query +SELECT company FROM ilike_any_table WHERE company ILIKE ANY ('%oO%', NULL) +-- !query schema +struct +-- !query output +FaceBook +Google + + +-- !query +SELECT company FROM ilike_any_table WHERE company NOT ILIKE ANY ('%oo%', NULL) +-- !query schema +struct +-- !query output +linkedIn + + +-- !query +SELECT company FROM ilike_any_table WHERE company ILIKE ANY (NULL, NULL) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT company FROM ilike_any_table WHERE company NOT ILIKE ANY (NULL, NULL) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT company FROM ilike_any_table WHERE company ILIKE ANY () +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Expected something between '(' and ')'." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 62, + "fragment" : "ILIKE ANY ()" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/ilike.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/ilike.sql.out new file mode 100644 index 000000000000..d618d3ee10ab --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/ilike.sql.out @@ -0,0 +1,208 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select null ilike 'a' +-- !query schema +struct +-- !query output +NULL + + +-- !query +select 'a' ilike null +-- !query schema +struct +-- !query output +NULL + + +-- !query +select null ilike null +-- !query schema +struct +-- !query output +NULL + + +-- !query +select 'a' ilike 'a' +-- !query schema +struct +-- !query output +true + + +-- !query +select 'a' ilike 'b' +-- !query schema +struct +-- !query output +false + + +-- !query +select 'A' ilike 'a' +-- !query schema +struct +-- !query output +true + + +-- !query +select 'a' ilike 'A' +-- !query schema +struct +-- !query output +true + + +-- !query +select 'abdef' ilike 'aBdef' +-- !query schema +struct +-- !query output +true + + +-- !query +select 'a_%b' ilike 'a\\__b' +-- !query schema +struct +-- !query output +true + + +-- !query +select 'addb' ilike 'A_%b' +-- !query schema +struct +-- !query output +true + + +-- !query +select 'abC' ilike 'a%' +-- !query schema +struct +-- !query output +true + + +-- !query +select 'a\nb' ilike 'a_B' +-- !query schema +struct +-- !query output +true + + +-- !query +select '' ilike '' +-- !query schema +struct +-- !query output +true + + +-- !query +select 'A' ilike '' +-- !query schema +struct +-- !query output +false + + +-- !query +select '' ilike 'a' +-- !query schema +struct +-- !query output +false + + +-- !query +select ilike('\__', '\\\__') +-- !query schema +struct +-- !query output +true + + +-- !query +select ilike('\\\__', '%\\%\%') +-- !query schema +struct +-- !query output +false + + +-- !query +select 'a\u20ACA' ilike '_\u20AC_' +-- !query schema +struct +-- !query output +true + + +-- !query +select 'A€a' ilike '_€_' +-- !query schema +struct +-- !query output +true + + +-- !query +select 'a€AA' ilike '_\u20AC_a' +-- !query schema +struct +-- !query output +true + + +-- !query +select 'a\u20ACaz' ilike '_€_Z' +-- !query schema +struct +-- !query output +true + + +-- !query +select 'ЀЁЂѺΏỀ' ilike 'ѐёђѻώề' +-- !query schema +struct +-- !query output +true + + +-- !query +select 'Addb' ilike 'a%#%b' escape '#' +-- !query schema +struct +-- !query output +false + + +-- !query +select 'a_%b' ilike 'a%#%B' escape '#' +-- !query schema +struct +-- !query output +true + + +-- !query +select 'Addb' ilike 'A%$%b' escape '$' +-- !query schema +struct +-- !query output +false + + +-- !query +select 'a_%b' ilike 'a%+%B' escape '+' +-- !query schema +struct +-- !query output +true diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/inline-table.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/inline-table.sql.out new file mode 100644 index 000000000000..0a2c7b0f55ed --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/inline-table.sql.out @@ -0,0 +1,289 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select * from values ("one", 1) +-- !query schema +struct +-- !query output +one 1 + + +-- !query +select * from values ("one", 1) as data +-- !query schema +struct +-- !query output +one 1 + + +-- !query +select * from values ("one", 1) as data(a, b) +-- !query schema +struct +-- !query output +one 1 + + +-- !query +select * from values 1, 2, 3 as data(a) +-- !query schema +struct +-- !query output +1 +2 +3 + + +-- !query +select * from values ("one", 1), ("two", 2), ("three", null) as data(a, b) +-- !query schema +struct +-- !query output +one 1 +three NULL +two 2 + + +-- !query +select * from values ("one", null), ("two", null) as data(a, b) +-- !query schema +struct +-- !query output +one NULL +two NULL + + +-- !query +select * from values ("one", 1), ("two", 2L) as data(a, b) +-- !query schema +struct +-- !query output +one 1 +two 2 + + +-- !query +select * from values ("one", 1 + 0), ("two", 1 + 3L) as data(a, b) +-- !query schema +struct +-- !query output +one 1 +two 4 + + +-- !query +select * from values ("one", 1 as one) as data(a, b) +-- !query schema +struct +-- !query output +one 1 + + +-- !query +select a from values ("one", current_timestamp) as data(a, b) +-- !query schema +struct +-- !query output +one + + +-- !query +select * from values ("one", array(0, 1)), ("two", array(2, 3)) as data(a, b) +-- !query schema +struct> +-- !query output +one [0,1] +two [2,3] + + +-- !query +select * from values ("one", 2.0), ("two", 3.0D) as data(a, b) +-- !query schema +struct +-- !query output +one 2.0 +two 3.0 + + +-- !query +select * from values ("one", rand(5)), ("two", 3.0D) as data(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_INLINE_TABLE.CANNOT_EVALUATE_EXPRESSION_IN_INLINE_TABLE", + "sqlState" : "42000", + "messageParameters" : { + "expr" : "\"rand(5)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 36, + "fragment" : "rand(5)" + } ] +} + + +-- !query +select * from values ("one", 2.0), ("two") as data(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INLINE_TABLE.NUM_COLUMNS_MISMATCH", + "sqlState" : "42000", + "messageParameters" : { + "actualNumCols" : "1", + "expectedNumCols" : "2", + "rowIndex" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 56, + "fragment" : "values (\"one\", 2.0), (\"two\") as data(a, b)" + } ] +} + + +-- !query +select * from values ("one", array(0, 1)), ("two", struct(1, 2)) as data(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_INLINE_TABLE.INCOMPATIBLE_TYPES_IN_INLINE_TABLE", + "sqlState" : "42000", + "messageParameters" : { + "colName" : "`b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 78, + "fragment" : "values (\"one\", array(0, 1)), (\"two\", struct(1, 2)) as data(a, b)" + } ] +} + + +-- !query +select * from values ("one"), ("two") as data(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INLINE_TABLE.NUM_COLUMNS_MISMATCH", + "sqlState" : "42000", + "messageParameters" : { + "actualNumCols" : "1", + "expectedNumCols" : "2", + "rowIndex" : "0" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 51, + "fragment" : "values (\"one\"), (\"two\") as data(a, b)" + } ] +} + + +-- !query +select * from values ("one", random_not_exist_func(1)), ("two", 2) as data(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`random_not_exist_func`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 53, + "fragment" : "random_not_exist_func(1)" + } ] +} + + +-- !query +select * from values ("one", count(1)), ("two", 2) as data(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_INLINE_TABLE.CANNOT_EVALUATE_EXPRESSION_IN_INLINE_TABLE", + "sqlState" : "42000", + "messageParameters" : { + "expr" : "\"count(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 37, + "fragment" : "count(1)" + } ] +} + + +-- !query +select * from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991-12-06 01:00:00.0'), timestamp('1991-12-06 12:00:00.0'))) as data(a, b) +-- !query schema +struct> +-- !query output +1991-12-06 00:00:00 [1991-12-06 01:00:00,1991-12-06 12:00:00] + + +-- !query +select * from values (try_add(5, 0)) +-- !query schema +struct +-- !query output +5 + + +-- !query +select * from values (try_divide(5, 0)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select * from values (10 + try_divide(5, 0)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select count(distinct ct) from values now(), now(), now() as data(ct) +-- !query schema +struct +-- !query output +1 + + +-- !query +select count(distinct ct) from values current_timestamp(), current_timestamp() as data(ct) +-- !query schema +struct +-- !query output +1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/inner-join.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/inner-join.sql.out new file mode 100644 index 000000000000..e7074c11d36f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/inner-join.sql.out @@ -0,0 +1,64 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW ta AS +SELECT a, 'a' AS tag FROM t1 +UNION ALL +SELECT a, 'b' AS tag FROM t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW tb AS +SELECT a, 'a' AS tag FROM t3 +UNION ALL +SELECT a, 'b' AS tag FROM t4 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag +-- !query schema +struct +-- !query output +1 a +1 a +1 b +1 b diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/intersect-all.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/intersect-all.sql.out new file mode 100644 index 000000000000..44f95f225abe --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/intersect-all.sql.out @@ -0,0 +1,378 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (1, 3), + (2, 3), + (null, null), + (null, null) + AS tab1(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (2, 3), + (3, 4), + (null, null), + (null, null) + AS tab2(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab2 +-- !query schema +struct +-- !query output +1 2 +1 2 +2 3 +NULL NULL +NULL NULL + + +-- !query +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab1 WHERE k = 1 +-- !query schema +struct +-- !query output +1 2 +1 2 +1 3 +1 3 + + +-- !query +SELECT * FROM tab1 WHERE k > 2 +INTERSECT ALL +SELECT * FROM tab2 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab2 WHERE k > 3 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM tab1 +INTERSECT ALL +SELECT CAST(1 AS BIGINT), CAST(2 AS BIGINT) +-- !query schema +struct +-- !query output +1 2 + + +-- !query +SELECT * FROM tab1 +INTERSECT ALL +SELECT array(1), 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"ARRAY\"", + "dataType2" : "\"INT\"", + "hint" : "", + "operator" : "INTERSECT ALL", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 51, + "fragment" : "SELECT * FROM tab1\nINTERSECT ALL\nSELECT array(1), 2" + } ] +} + + +-- !query +SELECT k FROM tab1 +INTERSECT ALL +SELECT k, v FROM tab2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NUM_COLUMNS_MISMATCH", + "sqlState" : "42826", + "messageParameters" : { + "firstNumColumns" : "1", + "invalidNumColumns" : "2", + "invalidOrdinalNum" : "second", + "operator" : "INTERSECT ALL" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT k FROM tab1\nINTERSECT ALL\nSELECT k, v FROM tab2" + } ] +} + + +-- !query +SELECT * FROM tab2 +INTERSECT ALL +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab2 +-- !query schema +struct +-- !query output +1 2 +1 2 +2 3 +NULL NULL +NULL NULL + + +-- !query +SELECT * FROM tab1 +EXCEPT +SELECT * FROM tab2 +UNION ALL +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab2 +-- !query schema +struct +-- !query output +1 2 +1 2 +1 3 +2 3 +NULL NULL +NULL NULL + + +-- !query +SELECT * FROM tab1 +EXCEPT +SELECT * FROM tab2 +EXCEPT +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab2 +-- !query schema +struct +-- !query output +1 3 + + +-- !query +( + ( + ( + SELECT * FROM tab1 + EXCEPT + SELECT * FROM tab2 + ) + EXCEPT + SELECT * FROM tab1 + ) + INTERSECT ALL + SELECT * FROM tab2 +) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM (SELECT tab1.k, + tab2.v + FROM tab1 + JOIN tab2 + ON tab1.k = tab2.k) +INTERSECT ALL +SELECT * +FROM (SELECT tab1.k, + tab2.v + FROM tab1 + JOIN tab2 + ON tab1.k = tab2.k) +-- !query schema +struct +-- !query output +1 2 +1 2 +1 2 +1 2 +1 2 +1 2 +1 2 +1 2 +2 3 + + +-- !query +SELECT * +FROM (SELECT tab1.k, + tab2.v + FROM tab1 + JOIN tab2 + ON tab1.k = tab2.k) +INTERSECT ALL +SELECT * +FROM (SELECT tab2.v AS k, + tab1.k AS v + FROM tab1 + JOIN tab2 + ON tab1.k = tab2.k) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT v FROM tab1 GROUP BY v +INTERSECT ALL +SELECT k FROM tab2 GROUP BY k +-- !query schema +struct +-- !query output +2 +3 +NULL + + +-- !query +SET spark.sql.legacy.setopsPrecedence.enabled= true +-- !query schema +struct +-- !query output +spark.sql.legacy.setopsPrecedence.enabled true + + +-- !query +SELECT * FROM tab1 +EXCEPT +SELECT * FROM tab2 +UNION ALL +SELECT * FROM tab1 +INTERSECT ALL +SELECT * FROM tab2 +-- !query schema +struct +-- !query output +1 2 +1 2 +2 3 +NULL NULL +NULL NULL + + +-- !query +SELECT * FROM tab1 +EXCEPT +SELECT * FROM tab2 +UNION ALL +SELECT * FROM tab1 +INTERSECT +SELECT * FROM tab2 +-- !query schema +struct +-- !query output +1 2 +2 3 +NULL NULL + + +-- !query +SET spark.sql.legacy.setopsPrecedence.enabled = false +-- !query schema +struct +-- !query output +spark.sql.legacy.setopsPrecedence.enabled false + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW tab3 AS VALUES (decimal(1)), (decimal(2)) tbl3(v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t.v FROM ( + SELECT v FROM tab3 + INTERSECT + SELECT v + v AS v FROM tab3 +) t +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT SUM(t.v) FROM ( + SELECT v FROM tab3 + INTERSECT + SELECT v + v AS v FROM tab3 +) t +-- !query schema +struct +-- !query output +2 + + +-- !query +DROP VIEW IF EXISTS tab1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS tab2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS tab3 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/interval.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/interval.sql.out new file mode 100644 index 000000000000..c32ecc19448a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/interval.sql.out @@ -0,0 +1,3899 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15') +-- !query schema +struct<((TIMESTAMP '2019-10-15 10:11:12.001002' - DATE '2019-10-15') * 3):interval day to second> +-- !query output +1 06:33:36.003006000 + + +-- !query +select interval 4 month 2 weeks 3 microseconds * 1.5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 4 month 2 weeks 3 microseconds" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "interval 4 month 2 weeks 3 microseconds" + } ] +} + + +-- !query +select interval 2 years 4 months +-- !query schema +struct +-- !query output +2-4 + + +-- !query +select interval 2 weeks 3 microseconds * 1.5 +-- !query schema +struct<(INTERVAL '14 00:00:00.000003' DAY TO SECOND * 1.5):interval day to second> +-- !query output +21 00:00:00.000005000 + + +-- !query +select (timestamp'2019-10-15' - timestamp'2019-10-14') / 1.5 +-- !query schema +struct<((TIMESTAMP '2019-10-15 00:00:00' - TIMESTAMP '2019-10-14 00:00:00') / 1.5):interval day to second> +-- !query output +0 16:00:00.000000000 + + +-- !query +select interval 2147483647 month * 2 +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +integer overflow + + +-- !query +select interval 2147483647 month / 0.5 +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Overflow + + +-- !query +select interval 2147483647 day * 2 +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select interval 2147483647 day / 0.5 +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select interval 2 second * '2' +-- !query schema +struct<(INTERVAL '02' SECOND * 2):interval day to second> +-- !query output +0 00:00:04.000000000 + + +-- !query +select interval 2 second / '2' +-- !query schema +struct<(INTERVAL '02' SECOND / 2):interval day to second> +-- !query output +0 00:00:01.000000000 + + +-- !query +select interval 2 year * '2' +-- !query schema +struct<(INTERVAL '2' YEAR * 2):interval year to month> +-- !query output +4-0 + + +-- !query +select interval 2 year / '2' +-- !query schema +struct<(INTERVAL '2' YEAR / 2):interval year to month> +-- !query output +1-0 + + +-- !query +select interval 2 second * 'a' +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval 2 second * 'a'" + } ] +} + + +-- !query +select interval 2 second / 'a' +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval 2 second / 'a'" + } ] +} + + +-- !query +select interval 2 year * 'a' +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "interval 2 year * 'a'" + } ] +} + + +-- !query +select interval 2 year / 'a' +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "interval 2 year / 'a'" + } ] +} + + +-- !query +select '2' * interval 2 second +-- !query schema +struct<(INTERVAL '02' SECOND * 2):interval day to second> +-- !query output +0 00:00:04.000000000 + + +-- !query +select '2' * interval 2 year +-- !query schema +struct<(INTERVAL '2' YEAR * 2):interval year to month> +-- !query output +4-0 + + +-- !query +select 'a' * interval 2 second +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'a' * interval 2 second" + } ] +} + + +-- !query +select 'a' * interval 2 year +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "'a' * interval 2 year" + } ] +} + + +-- !query +select '2' / interval 2 second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL SECOND\"", + "sqlExpr" : "\"(2 / INTERVAL '02' SECOND)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'2' / interval 2 second" + } ] +} + + +-- !query +select '2' / interval 2 year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "'2' / interval 2 year" + } ] +} + + +-- !query +select interval '2 seconds' / 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "interval '2 seconds' / 0" + } ] +} + + +-- !query +select interval '2 seconds' / null +-- !query schema +struct<(INTERVAL '02' SECOND / NULL):interval day to second> +-- !query output +NULL + + +-- !query +select interval '2 seconds' * null +-- !query schema +struct<(INTERVAL '02' SECOND * NULL):interval day to second> +-- !query output +NULL + + +-- !query +select null * interval '2 seconds' +-- !query schema +struct<(INTERVAL '02' SECOND * NULL):interval day to second> +-- !query output +NULL + + +-- !query +select interval '2' year / 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "interval '2' year / 0" + } ] +} + + +-- !query +select interval '2' year / null +-- !query schema +struct<(INTERVAL '2' YEAR / NULL):interval year to month> +-- !query output +NULL + + +-- !query +select interval '2' year * null +-- !query schema +struct<(INTERVAL '2' YEAR * NULL):interval year to month> +-- !query output +NULL + + +-- !query +select null * interval '2' year +-- !query schema +struct<(INTERVAL '2' YEAR * NULL):interval year to month> +-- !query output +NULL + + +-- !query +select 2 / interval '2' year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "2 / interval '2' year" + } ] +} + + +-- !query +select 2 / interval '2' hour +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"INTERVAL HOUR\"", + "sqlExpr" : "\"(2 / INTERVAL '02' HOUR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "2 / interval '2' hour" + } ] +} + + +-- !query +select null / interval '2' year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(NULL / INTERVAL '2' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "null / interval '2' year" + } ] +} + + +-- !query +select null / interval '2' hour +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"INTERVAL HOUR\"", + "sqlExpr" : "\"(NULL / INTERVAL '02' HOUR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "null / interval '2' hour" + } ] +} + + +-- !query +select -interval '-1 month 1 day -1 second' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '-1 month 1 day -1 second'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 43, + "fragment" : "interval '-1 month 1 day -1 second'" + } ] +} + + +-- !query +select -interval '-1 year 1 month' +-- !query schema +struct<(- INTERVAL '-0-11' YEAR TO MONTH):interval year to month> +-- !query output +0-11 + + +-- !query +select -interval '-1 day 1 hour -1 minute 1 second' +-- !query schema +struct<(- INTERVAL '-0 23:00:59' DAY TO SECOND):interval day to second> +-- !query output +0 23:00:59.000000000 + + +-- !query +select -interval -1 month 1 day -1 second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval -1 month 1 day -1 second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 41, + "fragment" : "interval -1 month 1 day -1 second" + } ] +} + + +-- !query +select -interval -1 year 1 month +-- !query schema +struct<(- INTERVAL '-0-11' YEAR TO MONTH):interval year to month> +-- !query output +0-11 + + +-- !query +select -interval -1 day 1 hour -1 minute 1 second +-- !query schema +struct<(- INTERVAL '-0 23:00:59' DAY TO SECOND):interval day to second> +-- !query output +0 23:00:59.000000000 + + +-- !query +select +interval '-1 month 1 day -1 second' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '-1 month 1 day -1 second'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 43, + "fragment" : "interval '-1 month 1 day -1 second'" + } ] +} + + +-- !query +select +interval '-1 year 1 month' +-- !query schema +struct<(+ INTERVAL '-0-11' YEAR TO MONTH):interval year to month> +-- !query output +-0-11 + + +-- !query +select +interval '-1 day 1 hour -1 minute 1 second' +-- !query schema +struct<(+ INTERVAL '-0 23:00:59' DAY TO SECOND):interval day to second> +-- !query output +-0 23:00:59.000000000 + + +-- !query +select +interval -1 month 1 day -1 second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval -1 month 1 day -1 second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 41, + "fragment" : "interval -1 month 1 day -1 second" + } ] +} + + +-- !query +select +interval -1 year 1 month +-- !query schema +struct<(+ INTERVAL '-0-11' YEAR TO MONTH):interval year to month> +-- !query output +-0-11 + + +-- !query +select +interval -1 day 1 hour -1 minute 1 second +-- !query schema +struct<(+ INTERVAL '-0 23:00:59' DAY TO SECOND):interval day to second> +-- !query output +-0 23:00:59.000000000 + + +-- !query +select interval -'1-1' year to month +-- !query schema +struct +-- !query output +-1-1 + + +-- !query +select interval -'-1-1' year to month +-- !query schema +struct +-- !query output +1-1 + + +-- !query +select interval +'-1-1' year to month +-- !query schema +struct +-- !query output +-1-1 + + +-- !query +select interval - '1 2:3:4.001' day to second +-- !query schema +struct +-- !query output +-1 02:03:04.001000000 + + +-- !query +select interval +'1 2:3:4.001' day to second +-- !query schema +struct +-- !query output +1 02:03:04.001000000 + + +-- !query +select interval -'-1 2:3:4.001' day to second +-- !query schema +struct +-- !query output +1 02:03:04.001000000 + + +-- !query +select interval -'1' year +-- !query schema +struct +-- !query output +-1-0 + + +-- !query +select interval -'-1' year +-- !query schema +struct +-- !query output +1-0 + + +-- !query +select interval -'11' month +-- !query schema +struct +-- !query output +-0-11 + + +-- !query +select interval -'-11' month +-- !query schema +struct +-- !query output +0-11 + + +-- !query +select interval -'1' day +-- !query schema +struct +-- !query output +-1 00:00:00.000000000 + + +-- !query +select interval -'-1' day +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select interval -'23' hour +-- !query schema +struct +-- !query output +-0 23:00:00.000000000 + + +-- !query +select interval -'-23' hour +-- !query schema +struct +-- !query output +0 23:00:00.000000000 + + +-- !query +select interval -'59' minute +-- !query schema +struct +-- !query output +-0 00:59:00.000000000 + + +-- !query +select interval -'-59' minute +-- !query schema +struct +-- !query output +0 00:59:00.000000000 + + +-- !query +select interval -'59' second +-- !query schema +struct +-- !query output +-0 00:00:59.000000000 + + +-- !query +select interval -'-59' second +-- !query schema +struct +-- !query output +0 00:00:59.000000000 + + +-- !query +select make_interval(1) +-- !query schema +struct +-- !query output +1 years + + +-- !query +select make_interval(1, 2) +-- !query schema +struct +-- !query output +1 years 2 months + + +-- !query +select make_interval(1, 2, 3) +-- !query schema +struct +-- !query output +1 years 2 months 21 days + + +-- !query +select make_interval(1, 2, 3, 4) +-- !query schema +struct +-- !query output +1 years 2 months 25 days + + +-- !query +select make_interval(1, 2, 3, 4, 5) +-- !query schema +struct +-- !query output +1 years 2 months 25 days 5 hours + + +-- !query +select make_interval(1, 2, 3, 4, 5, 6) +-- !query schema +struct +-- !query output +1 years 2 months 25 days 5 hours 6 minutes + + +-- !query +select make_interval(1, 2, 3, 4, 5, 6, 7.008009) +-- !query schema +struct +-- !query output +1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds + + +-- !query +select make_interval(1, 2, 3, 4, 0, 0, 123456789012.123456) +-- !query schema +struct +-- !query output +1 years 2 months 25 days 34293552 hours 30 minutes 12.123456 seconds + + +-- !query +select make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "18", + "scale" : "6", + "value" : "1234567890123456789" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789)" + } ] +} + + +-- !query +select make_dt_interval(1) +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select make_dt_interval(1, 2) +-- !query schema +struct +-- !query output +1 02:00:00.000000000 + + +-- !query +select make_dt_interval(1, 2, 3) +-- !query schema +struct +-- !query output +1 02:03:00.000000000 + + +-- !query +select make_dt_interval(1, 2, 3, 4.005006) +-- !query schema +struct +-- !query output +1 02:03:04.005006000 + + +-- !query +select make_dt_interval(1, 0, 0, 123456789012.123456) +-- !query schema +struct +-- !query output +1428899 00:30:12.123456000 + + +-- !query +select make_dt_interval(2147483647) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "make_dt_interval(2147483647)" + } ] +} + + +-- !query +select make_ym_interval(1) +-- !query schema +struct +-- !query output +1-0 + + +-- !query +select make_ym_interval(1, 2) +-- !query schema +struct +-- !query output +1-2 + + +-- !query +select make_ym_interval(0, 1) +-- !query schema +struct +-- !query output +0-1 + + +-- !query +select make_ym_interval(178956970, 7) +-- !query schema +struct +-- !query output +178956970-7 + + +-- !query +select make_ym_interval(178956970, 8) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "make_ym_interval(178956970, 8)" + } ] +} + + +-- !query +select make_ym_interval(-178956970, -8) +-- !query schema +struct +-- !query output +-178956970-8 + + +-- !query +select make_ym_interval(-178956970, -9) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "make_ym_interval(-178956970, -9)" + } ] +} + + +-- !query +select cast('1 second' as interval) +-- !query schema +struct +-- !query output +1 seconds + + +-- !query +select cast('+1 second' as interval) +-- !query schema +struct +-- !query output +1 seconds + + +-- !query +select cast('-1 second' as interval) +-- !query schema +struct +-- !query output +-1 seconds + + +-- !query +select cast('+ 1 second' as interval) +-- !query schema +struct +-- !query output +1 seconds + + +-- !query +select cast('- 1 second' as interval) +-- !query schema +struct +-- !query output +-1 seconds + + +-- !query +select cast('- -1 second' as interval) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('- +1 second' as interval) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select interval 13.123456789 seconds, interval -13.123456789 second +-- !query schema +struct +-- !query output +0 00:00:13.123456000 -0 00:00:13.123456000 + + +-- !query +select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond" + } ] +} + + +-- !query +select interval 1 year 2 month +-- !query schema +struct +-- !query output +1-2 + + +-- !query +select interval 4 day 5 hour 6 minute 7 seconds +-- !query schema +struct +-- !query output +4 05:06:07.000000000 + + +-- !query +select interval 3 week 8 millisecond 9 microsecond +-- !query schema +struct +-- !query output +21 00:00:00.008009000 + + +-- !query +select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 92, + "fragment" : "interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second" + } ] +} + + +-- !query +select interval '30' year '25' month +-- !query schema +struct +-- !query output +32-1 + + +-- !query +select interval '-100' day '40' hour '80' minute '299.889987299' second +-- !query schema +struct +-- !query output +-98 06:35:00.110013000 + + +-- !query +select interval '0-0' year to month +-- !query schema +struct +-- !query output +0-0 + + +-- !query +select interval '0 0:0:0' day to second +-- !query schema +struct +-- !query output +0 00:00:00.000000000 + + +-- !query +select interval '0 0:0:0.1' day to second +-- !query schema +struct +-- !query output +0 00:00:00.100000000 + + +-- !query +select interval '10-9' year to month +-- !query schema +struct +-- !query output +10-9 + + +-- !query +select interval '20 15' day to hour +-- !query schema +struct +-- !query output +20 15:00:00.000000000 + + +-- !query +select interval '20 15:40' day to minute +-- !query schema +struct +-- !query output +20 15:40:00.000000000 + + +-- !query +select interval '20 15:40:32.99899999' day to second +-- !query schema +struct +-- !query output +20 15:40:32.998999000 + + +-- !query +select interval '15:40' hour to minute +-- !query schema +struct +-- !query output +0 15:40:00.000000000 + + +-- !query +select interval '15:40:32.99899999' hour to second +-- !query schema +struct +-- !query output +0 15:40:32.998999000 + + +-- !query +select interval '40:32.99899999' minute to second +-- !query schema +struct +-- !query output +0 00:40:32.998999000 + + +-- !query +select interval '40:32' minute to second +-- !query schema +struct +-- !query output +0 00:40:32.000000000 + + +-- !query +select interval 30 day day +-- !query schema +struct +-- !query output +30 00:00:00.000000000 + + +-- !query +select interval 30 days days +-- !query schema +struct +-- !query output +30 00:00:00.000000000 + + +-- !query +select interval '20 15:40:32.99899999' day to hour +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "20 15:40:32.99899999", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR`", + "typeName" : "interval day to hour" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 50, + "fragment" : "'20 15:40:32.99899999' day to hour" + } ] +} + + +-- !query +select interval '20 15:40:32.99899999' day to minute +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "20 15:40:32.99899999", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]d h:m`, `INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE`", + "typeName" : "interval day to minute" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 52, + "fragment" : "'20 15:40:32.99899999' day to minute" + } ] +} + + +-- !query +select interval '15:40:32.99899999' hour to minute +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "15:40:32.99899999", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE`", + "typeName" : "interval hour to minute" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 50, + "fragment" : "'15:40:32.99899999' hour to minute" + } ] +} + + +-- !query +select interval '15:40.99899999' hour to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "15:40.99899999", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND`", + "typeName" : "interval hour to second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 47, + "fragment" : "'15:40.99899999' hour to second" + } ] +} + + +-- !query +select interval '15:40' hour to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "15:40", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND`", + "typeName" : "interval hour to second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 38, + "fragment" : "'15:40' hour to second" + } ] +} + + +-- !query +select interval '20 40:32.99899999' minute to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "20 40:32.99899999", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND`", + "typeName" : "interval minute to second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 52, + "fragment" : "'20 40:32.99899999' minute to second" + } ] +} + + +-- !query +select interval 10 nanoseconds +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.INVALID_UNIT", + "sqlState" : "22006", + "messageParameters" : { + "input" : " 10 nanoseconds", + "unit" : "nanoseconds" + } +} + + +-- !query +select map(1, interval 1 day, 2, interval 3 week) +-- !query schema +struct> +-- !query output +{1:1 00:00:00.000000000,2:21 00:00:00.000000000} + + +-- !query +select map(1, interval 1 day, 2, interval 2 day) +-- !query schema +struct> +-- !query output +{1:1 00:00:00.000000000,2:2 00:00:00.000000000} + + +-- !query +select map(1, interval 1 year, 2, interval 2 month) +-- !query schema +struct> +-- !query output +{1:1-0,2:0-2} + + +-- !query +select map(1, interval 1 month, 2, interval 2 month) +-- !query schema +struct> +-- !query output +{1:0-1,2:0-2} + + +-- !query +select map(1, interval 1 week, 2, interval 2 day) +-- !query schema +struct> +-- !query output +{1:7 00:00:00.000000000,2:2 00:00:00.000000000} + + +-- !query +select map(1, interval 2 millisecond, 3, interval 3 microsecond) +-- !query schema +struct> +-- !query output +{1:0 00:00:00.002000000,3:0 00:00:00.000003000} + + +-- !query +select interval 'interval 3 year 1 month' +-- !query schema +struct +-- !query output +3-1 + + +-- !query +select interval '3 year 1 month' +-- !query schema +struct +-- !query output +3-1 + + +-- !query +SELECT interval 'interval 2 weeks 2 days 1 hour 3 minutes 2 seconds 100 millisecond 200 microseconds' +-- !query schema +struct +-- !query output +16 01:03:02.100200000 + + +-- !query +SELECT interval '2 weeks 2 days 1 hour 3 minutes 2 seconds 100 millisecond 200 microseconds' +-- !query schema +struct +-- !query output +16 01:03:02.100200000 + + +-- !query +select interval 1 fake_unit +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'fake_unit'", + "hint" : "" + } +} + + +-- !query +select interval 1 year to month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0027", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 31, + "fragment" : "1 year to month" + } ] +} + + +-- !query +select interval '1' year to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNSUPPORTED_FROM_TO_EXPRESSION", + "sqlState" : "22006", + "messageParameters" : { + "from" : "year", + "input" : "1", + "to" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 34, + "fragment" : "'1' year to second" + } ] +} + + +-- !query +select interval '10-9' year to month '2-1' year to month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "interval '10-9' year to month '2-1' year to month" + } ] +} + + +-- !query +select interval '10-9' year to month '12:11:10' hour to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "interval '10-9' year to month '12:11:10' hour to second" + } ] +} + + +-- !query +select interval '1 15:11' day to minute '12:11:10' hour to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 65, + "fragment" : "interval '1 15:11' day to minute '12:11:10' hour to second" + } ] +} + + +-- !query +select interval 1 year '2-1' year to month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "interval 1 year '2-1' year to month" + } ] +} + + +-- !query +select interval 1 year '12:11:10' hour to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "interval 1 year '12:11:10' hour to second" + } ] +} + + +-- !query +select interval '10-9' year to month '1' year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "interval '10-9' year to month '1' year" + } ] +} + + +-- !query +select interval '12:11:10' hour to second '1' year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "interval '12:11:10' hour to second '1' year" + } ] +} + + +-- !query +select interval (-30) day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval (-30)" + } ] +} + + +-- !query +select interval (a + 1) day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "interval (a + 1)" + } ] +} + + +-- !query +select interval 30 day day day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'day'", + "hint" : ": extra input 'day'" + } +} + + +-- !query +select interval (-30) days +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval (-30)" + } ] +} + + +-- !query +select interval (a + 1) days +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "interval (a + 1)" + } ] +} + + +-- !query +select interval 30 days days days +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'days'", + "hint" : ": extra input 'days'" + } +} + + +-- !query +SELECT INTERVAL '178956970-7' YEAR TO MONTH +-- !query schema +struct +-- !query output +178956970-7 + + +-- !query +SELECT INTERVAL '178956970-8' YEAR TO MONTH +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.INTERVAL_PARSING", + "sqlState" : "22006", + "messageParameters" : { + "input" : "178956970-8", + "interval" : "year-month" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 43, + "fragment" : "'178956970-8' YEAR TO MONTH" + } ] +} + + +-- !query +SELECT INTERVAL '-178956970-8' YEAR TO MONTH +-- !query schema +struct +-- !query output +-178956970-8 + + +-- !query +SELECT INTERVAL -'178956970-8' YEAR TO MONTH +-- !query schema +struct +-- !query output +-178956970-8 + + +-- !query +select + interval '2-2' year to month + interval '3' month, + interval '2' year - interval '3-3' year to month, + interval '99 11:22:33.123456789' day to second + interval '10 9:8' day to minute, + interval '22:33.123456789' minute to second - interval '10' day +-- !query schema +struct<(INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' MONTH):interval year to month,(INTERVAL '2' YEAR - INTERVAL '3-3' YEAR TO MONTH):interval year to month,(INTERVAL '99 11:22:33.123456' DAY TO SECOND + INTERVAL '10 09:08' DAY TO MINUTE):interval day to second,(INTERVAL '22:33.123456' MINUTE TO SECOND - INTERVAL '10' DAY):interval day to second> +-- !query output +2-5 -1-3 109 20:30:33.123456000 -9 23:37:26.876544000 + + +-- !query +select + interval '2' year + '3-3 year to month', + interval '2' year - '3 month', + '3-2 year to month' + interval '2-2' year to month, + '3 year' - interval '2-2' year to month, + interval '99 11:22:33.123456789' day to second + '12:12 hour to second', + interval '99 11:22:33.123456789' day to second - '12 hour', + '4 day' + interval '10' day, + '4 22 day to hour' - interval '10' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3 year to month)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 48, + "fragment" : "interval '2' year + '3-3 year to month'" + } ] +} + + +-- !query +select + interval '2' year + null, + interval '2' year - null, + interval '2' hour + null, + interval '2' hour - null, + null + interval '2' year, + null - interval '2' year, + null + interval '2' hour, + null - interval '2' hour +-- !query schema +struct<(INTERVAL '2' YEAR + NULL):interval year,(INTERVAL '2' YEAR - NULL):interval year,(INTERVAL '02' HOUR + NULL):interval hour,(INTERVAL '02' HOUR - NULL):interval hour,(NULL + INTERVAL '2' YEAR):interval year,(NULL - INTERVAL '2' YEAR):interval year,(NULL + INTERVAL '02' HOUR):interval hour,(NULL - INTERVAL '02' HOUR):interval hour> +-- !query output +NULL NULL NULL NULL NULL NULL NULL NULL + + +-- !query +select interval '2' year + '3-3' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "interval '2' year + '3-3'" + } ] +} + + +-- !query +select interval '2' year - '4' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR - 4)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year - '4'" + } ] +} + + +-- !query +select '4 11:11' - interval '4 22:12' day to minute +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'4 11:11'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "'4 11:11' - interval '4 22:12' day to minute" + } ] +} + + +-- !query +select '4 12:12:12' + interval '4 22:12' day to minute +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'4 12:12:12'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "'4 12:12:12' + interval '4 22:12' day to minute" + } ] +} + + +-- !query +create temporary view interval_view as select '1' str +-- !query schema +struct<> +-- !query output + + + +-- !query +select interval '2' year + str from interval_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year + str" + } ] +} + + +-- !query +select interval '2' year - str from interval_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR - str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year - str" + } ] +} + + +-- !query +select str - interval '4 22:12' day to minute from interval_view +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "str - interval '4 22:12' day to minute" + } ] +} + + +-- !query +select str + interval '4 22:12' day to minute from interval_view +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "str + interval '4 22:12' day to minute" + } ] +} + + +-- !query +select interval '2-2' year to month + interval '3' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"", + "inputType" : "\"INTERVAL YEAR TO MONTH\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '2-2' year to month + interval '3' day" + } ] +} + + +-- !query +select interval '3' day + interval '2-2' year to month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"", + "inputType" : "\"INTERVAL YEAR TO MONTH\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '3' day + interval '2-2' year to month" + } ] +} + + +-- !query +select interval '2-2' year to month - interval '3' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"", + "inputType" : "\"INTERVAL YEAR TO MONTH\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + (- INTERVAL '3' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '2-2' year to month - interval '3' day" + } ] +} + + +-- !query +select interval '3' day - interval '2-2' year to month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"INTERVAL YEAR TO MONTH\"", + "sqlExpr" : "\"(INTERVAL '3' DAY - INTERVAL '2-2' YEAR TO MONTH)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '3' day - interval '2-2' year to month" + } ] +} + + +-- !query +select 1 - interval '2' second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"1 + (- INTERVAL '02' SECOND)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "1 - interval '2' second" + } ] +} + + +-- !query +select 1 + interval '2' month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"INTERVAL MONTH\"", + "sqlExpr" : "\"(1 + INTERVAL '2' MONTH)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "1 + interval '2' month" + } ] +} + + +-- !query +select interval '2' second + 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"1 + INTERVAL '02' SECOND\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' second + 1" + } ] +} + + +-- !query +select interval '2' month - 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL MONTH\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(INTERVAL '2' MONTH - 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "interval '2' month - 1" + } ] +} + + +-- !query +select interval '\t interval 1 day' +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select interval 'interval \t 1\tday' +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select interval 'interval\t1\tday' +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select interval '1\t' day +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select interval '1 ' day +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select interval '2-2\t' year to month +-- !query schema +struct +-- !query output +2-2 + + +-- !query +select interval '-\t2-2\t' year to month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING", + "sqlState" : "22006", + "messageParameters" : { + "input" : "-\t2-2\t", + "intervalStr" : "year-month", + "supportedFormat" : "`[+|-]y-m`, `INTERVAL [+|-]'[+|-]y-m' YEAR TO MONTH`", + "typeName" : "interval year to month" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 40, + "fragment" : "'-\\t2-2\\t' year to month" + } ] +} + + +-- !query +select interval '\n0 12:34:46.789\t' day to second +-- !query schema +struct +-- !query output +0 12:34:46.789000000 + + +-- !query +select interval '\n-\t10\t 12:34:46.789\t' day to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "\n-\t10\t 12:34:46.789\t", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]d h:m:s.n`, `INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND`", + "typeName" : "interval day to second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 56, + "fragment" : "'\\n-\\t10\\t 12:34:46.789\\t' day to second" + } ] +} + + +-- !query +select interval '中文 interval 1 day' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'中文 interval 1 day'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "interval '中文 interval 1 day'" + } ] +} + + +-- !query +select interval 'interval中文 1 day' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'interval中文 1 day'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "interval 'interval中文 1 day'" + } ] +} + + +-- !query +select interval 'interval 1中文day' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'interval 1中文day'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "interval 'interval 1中文day'" + } ] +} + + +-- !query +select -(a) from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015" +} + + +-- !query +select a - b from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITH_SUGGESTION", + "sqlState" : "22015", + "messageParameters" : { + "functionName" : "`try_subtract`" + } +} + + +-- !query +select b + interval '1 month' from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITH_SUGGESTION", + "sqlState" : "22015", + "messageParameters" : { + "functionName" : "`try_add`" + } +} + + +-- !query +select a * 1.1 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Overflow + + +-- !query +select a / 0.5 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Overflow + + +-- !query +SELECT + from_csv('1, 1 day', 'a INT, b interval'), + from_csv('1, 1', 'a INT, b interval day'), + to_csv(from_csv('1, 1 day', 'a INT, b interval')), + to_csv(from_csv('1, 1', 'a INT, b interval day')), + to_csv(named_struct('a', interval 32 hour, 'b', interval 70 minute)), + from_csv(to_csv(named_struct('a', interval 32 hour, 'b', interval 70 minute)), 'a interval hour, b interval minute') +-- !query schema +struct,from_csv(1, 1):struct,to_csv(from_csv(1, 1 day)):string,to_csv(from_csv(1, 1)):string,to_csv(named_struct(a, INTERVAL '32' HOUR, b, INTERVAL '70' MINUTE)):string,from_csv(to_csv(named_struct(a, INTERVAL '32' HOUR, b, INTERVAL '70' MINUTE))):struct> +-- !query output +{"a":1,"b":1 days} {"a":1,"b":1 00:00:00.000000000} 1,1 days 1,INTERVAL '1' DAY INTERVAL '32' HOUR,INTERVAL '70' MINUTE {"a":1 08:00:00.000000000,"b":0 01:10:00.000000000} + + +-- !query +SELECT + from_json('{"a":"1 days"}', 'a interval'), + from_csv('1, 1', 'a INT, b interval year'), + to_json(from_json('{"a":"1 days"}', 'a interval')), + to_csv(from_csv('1, 1', 'a INT, b interval year')), + to_csv(named_struct('a', interval 32 year, 'b', interval 10 month)), + from_csv(to_csv(named_struct('a', interval 32 year, 'b', interval 10 month)), 'a interval year, b interval month') +-- !query schema +struct,from_csv(1, 1):struct,to_json(from_json({"a":"1 days"})):string,to_csv(from_csv(1, 1)):string,to_csv(named_struct(a, INTERVAL '32' YEAR, b, INTERVAL '10' MONTH)):string,from_csv(to_csv(named_struct(a, INTERVAL '32' YEAR, b, INTERVAL '10' MONTH))):struct> +-- !query output +{"a":1 days} {"a":1,"b":1-0} {"a":"1 days"} 1,INTERVAL '1' YEAR INTERVAL '32' YEAR,INTERVAL '10' MONTH {"a":32-0,"b":0-10} + + +-- !query +SELECT + from_json('{"a":"1"}', 'a interval day'), + to_json(from_json('{"a":"1"}', 'a interval day')), + to_json(map('a', interval 100 day 130 minute)), + from_json(to_json(map('a', interval 100 day 130 minute)), 'a interval day to minute') +-- !query schema +struct,to_json(from_json({"a":"1"})):string,to_json(map(a, INTERVAL '100 02:10' DAY TO MINUTE)):string,from_json(to_json(map(a, INTERVAL '100 02:10' DAY TO MINUTE))):struct> +-- !query output +{"a":1 00:00:00.000000000} {"a":"INTERVAL '1' DAY"} {"a":"INTERVAL '100 02:10' DAY TO MINUTE"} {"a":100 02:10:00.000000000} + + +-- !query +SELECT + from_json('{"a":"1"}', 'a interval year'), + to_json(from_json('{"a":"1"}', 'a interval year')), + to_json(map('a', interval 32 year 10 month)), + from_json(to_json(map('a', interval 32 year 10 month)), 'a interval year to month') +-- !query schema +struct,to_json(from_json({"a":"1"})):string,to_json(map(a, INTERVAL '32-10' YEAR TO MONTH)):string,from_json(to_json(map(a, INTERVAL '32-10' YEAR TO MONTH))):struct> +-- !query output +{"a":1-0} {"a":"INTERVAL '1' YEAR"} {"a":"INTERVAL '32-10' YEAR TO MONTH"} {"a":32-10} + + +-- !query +select interval '+' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'+'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "interval '+'" + } ] +} + + +-- !query +select interval '+.' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'+.'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "interval '+.'" + } ] +} + + +-- !query +select interval '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "interval '1'" + } ] +} + + +-- !query +select interval '1.2' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1.2'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval '1.2'" + } ] +} + + +-- !query +select interval '- 2' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'- 2'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval '- 2'" + } ] +} + + +-- !query +select interval '1 day -' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1 day -'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "interval '1 day -'" + } ] +} + + +-- !query +select interval '1 day 1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1 day 1'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "interval '1 day 1'" + } ] +} + + +-- !query +select interval '1 day 2' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0026", + "messageParameters" : { + "value" : "1 day 2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 29, + "fragment" : "'1 day 2' day" + } ] +} + + +-- !query +select interval 'interval 1' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0026", + "messageParameters" : { + "value" : "interval 1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 32, + "fragment" : "'interval 1' day" + } ] +} + + +-- !query +select interval '-\t 1' day +-- !query schema +struct +-- !query output +-1 00:00:00.000000000 + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / 2 +-- !query schema +struct<(INTERVAL '-178956970-8' YEAR TO MONTH / 2):interval year to month> +-- !query output +-89478485-4 + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / 5 +-- !query schema +struct<(INTERVAL '-178956970-8' YEAR TO MONTH / 5):interval year to month> +-- !query output +-35791394-2 + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITH_SUGGESTION", + "sqlState" : "22015", + "messageParameters" : { + "functionName" : "`try_divide`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "(INTERVAL '-178956970-8' YEAR TO MONTH) / -1" + } ] +} + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1L +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITH_SUGGESTION", + "sqlState" : "22015", + "messageParameters" : { + "functionName" : "`try_divide`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "(INTERVAL '-178956970-8' YEAR TO MONTH) / -1L" + } ] +} + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1.0 +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Overflow + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1.0D +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +rounded value is out of range for input 2.147483648E9 and rounding mode HALF_UP + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / 2 +-- !query schema +struct<(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / 2):interval day to second> +-- !query output +-53375995 14:00:27.387904000 + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / 5 +-- !query schema +struct<(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / 5):interval day to second> +-- !query output +-21350398 05:36:10.955162000 + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITH_SUGGESTION", + "sqlState" : "22015", + "messageParameters" : { + "functionName" : "`try_divide`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 65, + "fragment" : "(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1" + } ] +} + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1L +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITH_SUGGESTION", + "sqlState" : "22015", + "messageParameters" : { + "functionName" : "`try_divide`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 66, + "fragment" : "(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1L" + } ] +} + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1.0 +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Overflow + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1.0D +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +rounded value is out of range for input 9.223372036854776E18 and rounding mode HALF_UP + + +-- !query +SELECT INTERVAL '106751991 04' DAY TO HOUR +-- !query schema +struct +-- !query output +106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '106751991 04:00' DAY TO MINUTE +-- !query schema +struct +-- !query output +106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '106751991 04:00:54.775807' DAY TO SECOND +-- !query schema +struct +-- !query output +106751991 04:00:54.775807000 + + +-- !query +SELECT INTERVAL '2562047788:00' HOUR TO MINUTE +-- !query schema +struct +-- !query output +106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '2562047788:00:54.775807' HOUR TO SECOND +-- !query schema +struct +-- !query output +106751991 04:00:54.775807000 + + +-- !query +SELECT INTERVAL '153722867280:54.775807' MINUTE TO SECOND +-- !query schema +struct +-- !query output +106751991 04:00:54.775807000 + + +-- !query +SELECT INTERVAL '-106751991 04' DAY TO HOUR +-- !query schema +struct +-- !query output +-106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '-106751991 04:00' DAY TO MINUTE +-- !query schema +struct +-- !query output +-106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND +-- !query schema +struct +-- !query output +-106751991 04:00:54.775808000 + + +-- !query +SELECT INTERVAL '-2562047788:00' HOUR TO MINUTE +-- !query schema +struct +-- !query output +-106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '-2562047788:00:54.775808' HOUR TO SECOND +-- !query schema +struct +-- !query output +-106751991 04:00:54.775808000 + + +-- !query +SELECT INTERVAL '-153722867280:54.775808' MINUTE TO SECOND +-- !query schema +struct +-- !query output +-106751991 04:00:54.775808000 + + +-- !query +SELECT INTERVAL '106751992 04' DAY TO HOUR +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: day 106751992 outside range [0, 106751991]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 42, + "fragment" : "'106751992 04' DAY TO HOUR" + } ] +} + + +-- !query +SELECT INTERVAL '-106751992 04' DAY TO HOUR +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: day 106751992 outside range [0, 106751991]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 43, + "fragment" : "'-106751992 04' DAY TO HOUR" + } ] +} + + +-- !query +SELECT INTERVAL '2562047789:00' HOUR TO MINUTE +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: hour 2562047789 outside range [0, 2562047788]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 46, + "fragment" : "'2562047789:00' HOUR TO MINUTE" + } ] +} + + +-- !query +SELECT INTERVAL '-2562047789:00' HOUR TO MINUTE +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: hour 2562047789 outside range [0, 2562047788]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 47, + "fragment" : "'-2562047789:00' HOUR TO MINUTE" + } ] +} + + +-- !query +SELECT INTERVAL '153722867281:54.775808' MINUTE TO SECOND +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: minute 153722867281 outside range [0, 153722867280]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 57, + "fragment" : "'153722867281:54.775808' MINUTE TO SECOND" + } ] +} + + +-- !query +SELECT INTERVAL '-153722867281:54.775808' MINUTE TO SECOND +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: minute 153722867281 outside range [0, 153722867280]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 58, + "fragment" : "'-153722867281:54.775808' MINUTE TO SECOND" + } ] +} + + +-- !query +SELECT INTERVAL '178956970' YEAR +-- !query schema +struct +-- !query output +178956970-0 + + +-- !query +SELECT INTERVAL '-178956970' YEAR +-- !query schema +struct +-- !query output +-178956970-0 + + +-- !query +SELECT INTERVAL '2147483647' MONTH +-- !query schema +struct +-- !query output +178956970-7 + + +-- !query +SELECT INTERVAL '-2147483647' MONTH +-- !query schema +struct +-- !query output +-178956970-7 + + +-- !query +SELECT INTERVAL '106751991' DAY +-- !query schema +struct +-- !query output +106751991 00:00:00.000000000 + + +-- !query +SELECT INTERVAL '-106751991' DAY +-- !query schema +struct +-- !query output +-106751991 00:00:00.000000000 + + +-- !query +SELECT INTERVAL '2562047788' HOUR +-- !query schema +struct +-- !query output +106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '-2562047788' HOUR +-- !query schema +struct +-- !query output +-106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '153722867280' MINUTE +-- !query schema +struct +-- !query output +106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '-153722867280' MINUTE +-- !query schema +struct +-- !query output +-106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '54.775807' SECOND +-- !query schema +struct +-- !query output +0 00:00:54.775807000 + + +-- !query +SELECT INTERVAL '-54.775807' SECOND +-- !query schema +struct +-- !query output +-0 00:00:54.775807000 + + +-- !query +SELECT INTERVAL '1' DAY > INTERVAL '1' HOUR +-- !query schema +struct<(INTERVAL '1' DAY > INTERVAL '01' HOUR):boolean> +-- !query output +true + + +-- !query +SELECT INTERVAL '1 02' DAY TO HOUR = INTERVAL '02:10:55' HOUR TO SECOND +-- !query schema +struct<(INTERVAL '1 02' DAY TO HOUR = INTERVAL '02:10:55' HOUR TO SECOND):boolean> +-- !query output +false + + +-- !query +SELECT INTERVAL '1' YEAR < INTERVAL '1' MONTH +-- !query schema +struct<(INTERVAL '1' YEAR < INTERVAL '1' MONTH):boolean> +-- !query output +false + + +-- !query +SELECT INTERVAL '-1-1' YEAR TO MONTH = INTERVAL '-13' MONTH +-- !query schema +struct<(INTERVAL '-1-1' YEAR TO MONTH = INTERVAL '-13' MONTH):boolean> +-- !query output +true + + +-- !query +SELECT INTERVAL 1 MONTH > INTERVAL 20 DAYS +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL MONTH\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(INTERVAL '1' MONTH > INTERVAL '20' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "INTERVAL 1 MONTH > INTERVAL 20 DAYS" + } ] +} + + +-- !query +SELECT INTERVAL '1' DAY < '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY < 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "INTERVAL '1' DAY < '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' DAY = '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY = 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "INTERVAL '1' DAY = '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' DAY > '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY > 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "INTERVAL '1' DAY > '1'" + } ] +} + + +-- !query +SELECT '1' < INTERVAL '1' DAY +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 < INTERVAL '1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' < INTERVAL '1' DAY" + } ] +} + + +-- !query +SELECT '1' = INTERVAL '1' DAY +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 = INTERVAL '1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' = INTERVAL '1' DAY" + } ] +} + + +-- !query +SELECT '1' > INTERVAL '1' DAY +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 > INTERVAL '1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' > INTERVAL '1' DAY" + } ] +} + + +-- !query +SELECT INTERVAL '1' YEAR < '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR < 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "INTERVAL '1' YEAR < '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' YEAR = '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR = 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "INTERVAL '1' YEAR = '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' YEAR > '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR > 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "INTERVAL '1' YEAR > '1'" + } ] +} + + +-- !query +SELECT '1' < INTERVAL '1' YEAR +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 < INTERVAL '1' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'1' < INTERVAL '1' YEAR" + } ] +} + + +-- !query +SELECT '1' = INTERVAL '1' YEAR +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 = INTERVAL '1' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'1' = INTERVAL '1' YEAR" + } ] +} + + +-- !query +SELECT '1' > INTERVAL '1' YEAR +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 > INTERVAL '1' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'1' > INTERVAL '1' YEAR" + } ] +} + + +-- !query +SELECT array(INTERVAL '1' YEAR, INTERVAL '1' MONTH) +-- !query schema +struct> +-- !query output +[1-0,0-1] + + +-- !query +SELECT array(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE) +-- !query schema +struct> +-- !query output +[1 00:00:00.000000000,0 01:01:00.000000000] + + +-- !query +SELECT array(INTERVAL 1 MONTH, INTERVAL 20 DAYS) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"INTERVAL MONTH\" or \"INTERVAL DAY\")", + "functionName" : "`array`", + "sqlExpr" : "\"array(INTERVAL '1' MONTH, INTERVAL '20' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "array(INTERVAL 1 MONTH, INTERVAL 20 DAYS)" + } ] +} + + +-- !query +SELECT coalesce(INTERVAL '1' YEAR, INTERVAL '1' MONTH) +-- !query schema +struct +-- !query output +1-0 + + +-- !query +SELECT coalesce(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE) +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +SELECT coalesce(INTERVAL 1 MONTH, INTERVAL 20 DAYS) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"INTERVAL MONTH\" or \"INTERVAL DAY\")", + "functionName" : "`coalesce`", + "sqlExpr" : "\"coalesce(INTERVAL '1' MONTH, INTERVAL '20' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "coalesce(INTERVAL 1 MONTH, INTERVAL 20 DAYS)" + } ] +} + + +-- !query +SELECT abs(INTERVAL '-10' YEAR) +-- !query schema +struct +-- !query output +10-0 + + +-- !query +SELECT abs(INTERVAL -'1 02:03:04.123' DAY TO SECOND) +-- !query schema +struct +-- !query output +1 02:03:04.123000000 + + +-- !query +SELECT div(INTERVAL '1-1' YEAR TO MONTH, INTERVAL '1' YEAR) +-- !query schema +struct<(INTERVAL '1-1' YEAR TO MONTH div INTERVAL '1' YEAR):bigint> +-- !query output +1 + + +-- !query +SELECT div(INTERVAL '1-1' YEAR TO MONTH, INTERVAL '-1' MONTH) +-- !query schema +struct<(INTERVAL '1-1' YEAR TO MONTH div INTERVAL '-1' MONTH):bigint> +-- !query output +-13 + + +-- !query +SELECT div(INTERVAL '1 06' DAY TO HOUR, INTERVAL '1' DAY) +-- !query schema +struct<(INTERVAL '1 06' DAY TO HOUR div INTERVAL '1' DAY):bigint> +-- !query output +1 + + +-- !query +SELECT div(INTERVAL '1 06' DAY TO HOUR, INTERVAL '-1' HOUR) +-- !query schema +struct<(INTERVAL '1 06' DAY TO HOUR div INTERVAL '-01' HOUR):bigint> +-- !query output +-30 + + +-- !query +SELECT div(INTERVAL '1' MONTH, INTERVAL '-1' DAY) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL MONTH\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(INTERVAL '1' MONTH div INTERVAL '-1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "div(INTERVAL '1' MONTH, INTERVAL '-1' DAY)" + } ] +} + + +-- !query +SELECT signum(INTERVAL '-10' YEAR) +-- !query schema +struct +-- !query output +-1.0 + + +-- !query +SELECT signum(INTERVAL '10' MONTH) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT signum(INTERVAL '0-0' YEAR TO MONTH) +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT signum(INTERVAL '-10' DAY) +-- !query schema +struct +-- !query output +-1.0 + + +-- !query +SELECT signum(INTERVAL '10' HOUR) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT signum(INTERVAL '0 0:0:0' DAY TO SECOND) +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT width_bucket(INTERVAL '0' YEAR, INTERVAL '0' YEAR, INTERVAL '10' YEAR, 10) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT width_bucket(INTERVAL '-1' YEAR, INTERVAL -'1-2' YEAR TO MONTH, INTERVAL '1-2' YEAR TO MONTH, 10) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT width_bucket(INTERVAL '0' DAY, INTERVAL '0' DAY, INTERVAL '10' DAY, 10) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT width_bucket(INTERVAL '-59' MINUTE, INTERVAL -'1 01' DAY TO HOUR, INTERVAL '1 2:3:4.001' DAY TO SECOND, 10) +-- !query schema +struct +-- !query output +5 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/join-empty-relation.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/join-empty-relation.sql.out new file mode 100644 index 000000000000..7fede20fe3c5 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/join-empty-relation.sql.out @@ -0,0 +1,191 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW empty_table as SELECT a FROM t2 WHERE false +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM t1 INNER JOIN empty_table +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM t1 CROSS JOIN empty_table +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM t1 LEFT OUTER JOIN empty_table +-- !query schema +struct +-- !query output +1 NULL + + +-- !query +SELECT * FROM t1 RIGHT OUTER JOIN empty_table +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM t1 FULL OUTER JOIN empty_table +-- !query schema +struct +-- !query output +1 NULL + + +-- !query +SELECT * FROM t1 LEFT SEMI JOIN empty_table +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM t1 LEFT ANTI JOIN empty_table +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM empty_table INNER JOIN t1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM empty_table CROSS JOIN t1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM empty_table LEFT OUTER JOIN t1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM empty_table RIGHT OUTER JOIN t1 +-- !query schema +struct +-- !query output +NULL 1 + + +-- !query +SELECT * FROM empty_table FULL OUTER JOIN t1 +-- !query schema +struct +-- !query output +NULL 1 + + +-- !query +SELECT * FROM empty_table LEFT SEMI JOIN t1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM empty_table LEFT ANTI JOIN t1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM empty_table INNER JOIN empty_table +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM empty_table CROSS JOIN empty_table +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM empty_table LEFT OUTER JOIN empty_table +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM empty_table RIGHT OUTER JOIN empty_table +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM empty_table FULL OUTER JOIN empty_table +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM empty_table LEFT SEMI JOIN empty_table +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM empty_table LEFT ANTI JOIN empty_table +-- !query schema +struct +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/join-lateral.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/join-lateral.sql.out new file mode 100644 index 000000000000..11bafb2cf63c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/join-lateral.sql.out @@ -0,0 +1,1937 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW t2(c1, c2) AS VALUES (0, 2), (0, 3) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW t3(c1, c2) AS VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4)) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW t4(c1, c2) AS VALUES (0, 1), (0, 2), (1, 1), (1, 3) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM t1, LATERAL (SELECT c1) +-- !query schema +struct +-- !query output +0 1 0 +1 2 1 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT c1 FROM t2) +-- !query schema +struct +-- !query output +0 1 0 +0 1 0 +1 2 0 +1 2 0 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT t1.c1 FROM t2) +-- !query schema +struct +-- !query output +0 1 0 +0 1 0 +1 2 1 +1 2 1 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT t1.c1 + t2.c1 FROM t2) +-- !query schema +struct +-- !query output +0 1 0 +0 1 0 +1 2 1 +1 2 1 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT *) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT * FROM t2) +-- !query schema +struct +-- !query output +0 1 0 2 +0 1 0 3 +1 2 0 2 +1 2 0 3 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT t1.*) +-- !query schema +struct +-- !query output +0 1 0 1 +1 2 1 2 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT t1.*, t2.* FROM t2) +-- !query schema +struct +-- !query output +0 1 0 1 0 2 +0 1 0 1 0 3 +1 2 1 2 0 2 +1 2 1 2 0 3 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT t1.* FROM t2 AS t1) +-- !query schema +struct +-- !query output +0 1 0 2 +0 1 0 3 +1 2 0 2 +1 2 0 3 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT t1.*, t2.* FROM t2, LATERAL (SELECT t1.*, t2.*, t3.* FROM t2 AS t3)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CANNOT_RESOLVE_STAR_EXPAND", + "sqlState" : "42704", + "messageParameters" : { + "columns" : "`c1`, `c2`", + "targetString" : "`t1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 71, + "stopIndex" : 74, + "fragment" : "t1.*" + } ] +} + + +-- !query +SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3 +-- !query schema +struct +-- !query output +0 1 1 + + +-- !query +SELECT * FROM t1 LEFT JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3 +-- !query schema +struct +-- !query output +0 1 1 +1 2 NULL + + +-- !query +SELECT * FROM t1 CROSS JOIN LATERAL (SELECT c1 + c2 AS c3) +-- !query schema +struct +-- !query output +0 1 1 +1 2 3 + + +-- !query +SELECT * FROM t1 NATURAL JOIN LATERAL (SELECT c1 + c2 AS c2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INCOMPATIBLE_JOIN_TYPES", + "sqlState" : "42613", + "messageParameters" : { + "joinType1" : "LATERAL", + "joinType2" : "NATURAL" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 18, + "stopIndex" : 60, + "fragment" : "NATURAL JOIN LATERAL (SELECT c1 + c2 AS c2)" + } ] +} + + +-- !query +SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c2) USING (c2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_FEATURE.LATERAL_JOIN_USING", + "sqlState" : "0A000", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 18, + "stopIndex" : 63, + "fragment" : "JOIN LATERAL (SELECT c1 + c2 AS c2) USING (c2)" + } ] +} + + +-- !query +SELECT * FROM LATERAL (SELECT * FROM t1) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT * FROM t2) +-- !query schema +struct +-- !query output +0 1 0 2 +0 1 0 3 +1 2 0 2 +1 2 0 3 + + +-- !query +SELECT * FROM LATERAL (SELECT * FROM t1), LATERAL (SELECT * FROM t2) +-- !query schema +struct +-- !query output +0 1 0 2 +0 1 0 3 +1 2 0 2 +1 2 0 3 + + +-- !query +SELECT * FROM LATERAL (SELECT * FROM t1) JOIN LATERAL (SELECT * FROM t2) +-- !query schema +struct +-- !query output +0 1 0 2 +0 1 0 3 +1 2 0 2 +1 2 0 3 + + +-- !query +SELECT a, b FROM t1, LATERAL (SELECT c1, c2) s(a, b) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +SELECT * FROM (SELECT 1 AS c1, 2 AS c2), LATERAL (SELECT c1, c2) +-- !query schema +struct +-- !query output +1 2 1 2 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT c2 FROM t2 WHERE t1.c1 = t2.c1) +-- !query schema +struct +-- !query output +0 1 2 +0 1 3 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT c2 FROM t2 WHERE t1.c2 < t2.c2) +-- !query schema +struct +-- !query output +0 1 2 +0 1 3 +1 2 3 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT max(c2) AS m FROM t2 WHERE t1.c2 < t2.c2) +-- !query schema +struct +-- !query output +0 1 3 +1 2 3 + + +-- !query +SELECT * FROM t1 JOIN t2 JOIN LATERAL (SELECT t1.c2 + t2.c2) +-- !query schema +struct +-- !query output +0 1 0 2 3 +0 1 0 3 4 +1 2 0 2 4 +1 2 0 3 5 + + +-- !query +SELECT * FROM t1 JOIN LATERAL (SELECT t1.c1 AS a, t2.c1 AS b) s JOIN t2 ON s.b = t2.c1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t2`.`c1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 55, + "fragment" : "t2.c1" + } ] +} + + +-- !query +SELECT x FROM VALUES (0) t(x) JOIN LATERAL (SELECT x + rand(0) AS y) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT x FROM (SELECT SUM(c1) AS x FROM t1), LATERAL (SELECT x + rand(0) AS y) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT c1 + c2 + rand(0) AS c3) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.NON_DETERMINISTIC_LATERAL_SUBQUERIES", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "LateralJoin lateral-subquery#x [c1#x && c2#x], Inner\n: +- SubqueryAlias __auto_generated_subquery_name\n: +- Project [(cast((outer(c1#x) + outer(c2#x)) as double) + rand(number)) AS c3#x]\n: +- OneRowRelation\n+- SubqueryAlias spark_catalog.default.t1\n +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])\n +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n +- LocalRelation [col1#x, col2#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 58, + "fragment" : "FROM t1, LATERAL (SELECT c1 + c2 + rand(0) AS c3)" + } ] +} + + +-- !query +SELECT * FROM t1, LATERAL (SELECT rand(0) FROM t2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.NON_DETERMINISTIC_LATERAL_SUBQUERIES", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "LateralJoin lateral-subquery#x [], Inner\n: +- SubqueryAlias __auto_generated_subquery_name\n: +- Project [rand(number) AS rand(number)#x]\n: +- SubqueryAlias spark_catalog.default.t2\n: +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])\n: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n: +- LocalRelation [col1#x, col2#x]\n+- SubqueryAlias spark_catalog.default.t1\n +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])\n +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n +- LocalRelation [col1#x, col2#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 50, + "fragment" : "FROM t1, LATERAL (SELECT rand(0) FROM t2)" + } ] +} + + +-- !query +SELECT * FROM t1 JOIN LATERAL (SELECT * FROM t2) s ON t1.c1 + rand(0) = s.c1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.LATERAL_JOIN_CONDITION_NON_DETERMINISTIC", + "sqlState" : "0A000", + "messageParameters" : { + "condition" : "((CAST(spark_catalog.default.t1.c1 AS DOUBLE) + rand(0)) = CAST(s.c1 AS DOUBLE))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 55, + "stopIndex" : 76, + "fragment" : "t1.c1 + rand(0) = s.c1" + } ] +} + + +-- !query +SELECT * FROM t1, +LATERAL (SELECT c1 + c2 AS a), +LATERAL (SELECT c1 - c2 AS b), +LATERAL (SELECT a * b AS c) +-- !query schema +struct +-- !query output +0 1 1 -1 -1 +1 2 3 -1 -3 + + +-- !query +SELECT * FROM t1 +LEFT OUTER JOIN LATERAL (SELECT c2 FROM t2 WHERE t1.c1 = t2.c1) s +LEFT OUTER JOIN t1 t3 ON s.c2 = t3.c2 +-- !query schema +struct +-- !query output +0 1 2 1 2 +0 1 3 NULL NULL +1 2 NULL NULL NULL + + +-- !query +SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT c1)) +-- !query schema +struct +-- !query output +0 1 0 2 0 +0 1 0 3 0 +1 2 0 2 0 +1 2 0 3 0 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1 + 1 AS c1), LATERAL (SELECT c1)) +-- !query schema +struct +-- !query output +0 1 1 1 +1 2 2 2 + + +-- !query +SELECT * FROM t1, LATERAL ( + SELECT * FROM (SELECT c1, MIN(c2) m FROM t2 WHERE t1.c1 = t2.c1 GROUP BY c1) s, + LATERAL (SELECT m WHERE m > c1) +) +-- !query schema +struct +-- !query output +0 1 0 2 2 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.c1 + t2.c1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t1`.`c1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 62, + "stopIndex" : 66, + "fragment" : "t1.c1" + } ] +} + + +-- !query +SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1), LATERAL (SELECT c2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`c2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 71, + "stopIndex" : 72, + "fragment" : "c2" + } ] +} + + +-- !query +SELECT * FROM t1, LATERAL (SELECT c2, (SELECT MIN(c2) FROM t2)) +-- !query schema +struct +-- !query output +0 1 1 2 +1 2 2 2 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT (SELECT SUM(c2) FROM t2 WHERE c1 = a) FROM (SELECT c1 AS a)) +-- !query schema +struct +-- !query output +0 1 5 +1 2 NULL + + +-- !query +SELECT * FROM t1, LATERAL (SELECT c1, (SELECT SUM(c2) FROM t2 WHERE c1 = t1.c1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t1`.`c1`", + "proposal" : "`t2`.`c1`, `t2`.`c2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 74, + "stopIndex" : 78, + "fragment" : "t1.c1" + } ] +} + + +-- !query +SELECT * FROM t1 WHERE c1 = (SELECT MIN(a) FROM t2, LATERAL (SELECT c1 AS a)) +-- !query schema +struct +-- !query output +0 1 + + +-- !query +SELECT * FROM t1 WHERE c1 = (SELECT MIN(a) FROM t2, LATERAL (SELECT c1 AS a) WHERE c1 = t1.c1) +-- !query schema +struct +-- !query output +0 1 + + +-- !query +SELECT * FROM t1 JOIN lateral (SELECT * FROM t2 JOIN t4 ON t2.c1 = t4.c1 AND t2.c1 = t1.c1) +-- !query schema +struct +-- !query output +0 1 0 2 0 1 +0 1 0 2 0 2 +0 1 0 3 0 1 +0 1 0 3 0 2 + + +-- !query +SELECT * FROM t1 JOIN lateral (SELECT * FROM t2 JOIN t4 ON t2.c1 != t4.c1 AND t2.c1 != t1.c1) +-- !query schema +struct +-- !query output +1 2 0 2 1 1 +1 2 0 2 1 3 +1 2 0 3 1 1 +1 2 0 3 1 3 + + +-- !query +SELECT * FROM t1 LEFT JOIN lateral (SELECT * FROM t4 LEFT JOIN t2 ON t2.c1 = t4.c1 AND t2.c1 = t1.c1) +-- !query schema +struct +-- !query output +0 1 0 1 0 2 +0 1 0 1 0 3 +0 1 0 2 0 2 +0 1 0 2 0 3 +0 1 1 1 NULL NULL +0 1 1 3 NULL NULL +1 2 0 1 NULL NULL +1 2 0 2 NULL NULL +1 2 1 1 NULL NULL +1 2 1 3 NULL NULL + + +-- !query +SELECT * FROM t1, LATERAL (SELECT COUNT(*) cnt FROM t2 WHERE c1 = t1.c1) +-- !query schema +struct +-- !query output +0 1 2 +1 2 0 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT COUNT(*) cnt, SUM(c2) sum FROM t2 WHERE c1 = t1.c1) +-- !query schema +struct +-- !query output +0 1 2 5 +1 2 0 NULL + + +-- !query +SELECT * FROM t1, LATERAL (SELECT SUM(c2) IS NULL FROM t2 WHERE t1.c1 = t2.c1) +-- !query schema +struct +-- !query output +0 1 false +1 2 true + + +-- !query +SELECT * FROM t1, LATERAL (SELECT COUNT(*) + CASE WHEN sum(c2) IS NULL THEN 0 ELSE sum(c2) END FROM t2 WHERE t1.c1 = t2.c1) +-- !query schema +struct +-- !query output +0 1 7 +1 2 0 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT c1, COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1 GROUP BY c1) +-- !query schema +struct +-- !query output +0 1 0 2 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT c2, COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1 GROUP BY c2) +-- !query schema +struct +-- !query output +0 1 2 1 +0 1 3 1 + + +-- !query +SELECT * FROM t1 JOIN LATERAL (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) +-- !query schema +struct +-- !query output +0 1 2 +1 2 0 + + +-- !query +SELECT * FROM t1 LEFT JOIN LATERAL (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) +-- !query schema +struct +-- !query output +0 1 2 +1 2 0 + + +-- !query +SELECT * FROM t1 CROSS JOIN LATERAL (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) +-- !query schema +struct +-- !query output +0 1 2 +1 2 0 + + +-- !query +SELECT * FROM t1 LEFT JOIN LATERAL (SELECT c1, COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1 GROUP BY c1) +-- !query schema +struct +-- !query output +0 1 0 2 +1 2 NULL NULL + + +-- !query +SELECT * FROM t1 CROSS JOIN LATERAL (SELECT c1, COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1 GROUP BY c1) +-- !query schema +struct +-- !query output +0 1 0 2 + + +-- !query +SELECT * FROM t1 JOIN LATERAL (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) ON cnt + 1 = c1 +-- !query schema +struct +-- !query output +1 2 0 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT COUNT(*) cnt FROM t1 t2 WHERE t1.c1 = t2.c1) +-- !query schema +struct +-- !query output +0 1 1 +1 2 1 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT COUNT(*) cnt FROM t1 t2 WHERE t1.c1 = t2.c1 HAVING cnt > 0) +-- !query schema +struct +-- !query output +0 1 1 +1 2 1 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT SUM(cnt) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1)) +-- !query schema +struct +-- !query output +0 1 2 +1 2 0 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT COUNT(cnt) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1)) +-- !query schema +struct +-- !query output +0 1 1 +1 2 1 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT SUM(cnt) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1 GROUP BY c1)) +-- !query schema +struct +-- !query output +0 1 2 +1 2 NULL + + +-- !query +SELECT * FROM t1, LATERAL ( + SELECT COUNT(*) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) + JOIN t2 ON cnt = t2.c1 +) +-- !query schema +struct +-- !query output +0 1 0 +1 2 2 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) WHERE cnt = c1 - 1) +-- !query schema +struct +-- !query output +1 2 0 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT COUNT(*) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) WHERE cnt = c1 - 1) +-- !query schema +struct +-- !query output +0 1 0 +1 2 1 + + +-- !query +SELECT * FROM t1, LATERAL ( + SELECT COUNT(*) FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) + WHERE cnt = c1 - 1 GROUP BY cnt +) +-- !query schema +struct +-- !query output +1 2 1 + + +-- !query +SELECT * FROM t1, LATERAL ( + SELECT * FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) + JOIN t2 ON cnt = t2.c1 +) +-- !query schema +struct +-- !query output +1 2 0 0 2 +1 2 0 0 3 + + +-- !query +SELECT * FROM t1, LATERAL ( + SELECT l.cnt + r.cnt + FROM (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) l + JOIN (SELECT COUNT(*) cnt FROM t2 WHERE t1.c1 = t2.c1) r +) +-- !query schema +struct +-- !query output +0 1 4 +1 2 0 + + +-- !query +SELECT * FROM t1 LEFT JOIN LATERAL (SELECT MIN(c2) FROM t2 WHERE c1 = t1.c1 GROUP BY c1) +-- !query schema +struct +-- !query output +0 1 2 +1 2 NULL + + +-- !query +WITH cte1 AS ( + SELECT c1 FROM t1 +), cte2 AS ( + SELECT s.* FROM cte1, LATERAL (SELECT * FROM t2 WHERE c1 = cte1.c1) s +) +SELECT * FROM cte2 +-- !query schema +struct +-- !query output +0 2 +0 3 + + +-- !query +SELECT * FROM t3 JOIN LATERAL (SELECT EXPLODE(c2)) +-- !query schema +struct,col:int> +-- !query output +0 [0,1] 0 +0 [0,1] 1 +1 [2] 2 +NULL [4] 4 + + +-- !query +SELECT * FROM t3 JOIN LATERAL (SELECT EXPLODE_OUTER(c2)) +-- !query schema +struct,col:int> +-- !query output +0 [0,1] 0 +0 [0,1] 1 +1 [2] 2 +2 [] NULL +NULL [4] 4 + + +-- !query +SELECT * FROM t3 JOIN LATERAL (SELECT EXPLODE(c2)) t(c3) ON c1 = c3 +-- !query schema +struct,c3:int> +-- !query output +0 [0,1] 0 + + +-- !query +SELECT * FROM t3 LEFT JOIN LATERAL (SELECT EXPLODE(c2)) t(c3) ON c1 = c3 +-- !query schema +struct,c3:int> +-- !query output +0 [0,1] 0 +1 [2] NULL +2 [] NULL +NULL [4] NULL + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT sum(t2.c2) over (order by t2.c1) + FROM t2 + WHERE t2.c1 >= t1.c1) +-- !query schema +struct +-- !query output +0 1 5 +0 1 5 + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 = t1.c1) +-- !query schema +struct +-- !query output +0 1 1 +0 1 2 +0 1 2 +0 1 3 +1 2 1 +1 2 3 + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + UNION DISTINCT + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2) +-- !query schema +struct +-- !query output +0 1 2 +0 1 3 + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + INTERSECT ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 = t1.c1) +-- !query schema +struct +-- !query output +0 1 2 + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + INTERSECT DISTINCT + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + EXCEPT ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 = t1.c1) +-- !query schema +struct +-- !query output +0 1 3 + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + EXCEPT DISTINCT + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2) +-- !query schema +struct +-- !query output +0 1 2 +0 1 3 + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT COUNT(t2.c2) + FROM t2 + WHERE t2.c1 = t1.c1 + UNION DISTINCT + SELECT COUNT(t4.c2) + FROM t4 + WHERE t4.c1 > t1.c2) +-- !query schema +struct +-- !query output +0 1 0 +0 1 2 +1 2 0 + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c1, t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + UNION ALL + SELECT t4.c2, t4.c1 + FROM t4 + WHERE t4.c1 = t1.c1) +-- !query schema +struct +-- !query output +0 1 0 2 +0 1 0 3 +0 1 1 0 +0 1 2 0 +1 2 1 1 +1 2 3 1 + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2 + UNION ALL + SELECT t4.c2 + FROM t4) +-- !query schema +struct +-- !query output +0 1 1 +0 1 1 +0 1 2 +0 1 2 +0 1 3 +0 1 3 +1 2 1 +1 2 1 +1 2 2 +1 2 3 + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4) +-- !query schema +struct +-- !query output +0 1 1 +0 1 1 +0 1 2 +0 1 2 +0 1 3 +0 1 3 +1 2 1 +1 2 1 +1 2 2 +1 2 3 + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2 + UNION DISTINCT + SELECT t4.c2 + FROM t4) +-- !query schema +struct +-- !query output +0 1 1 +0 1 2 +0 1 3 +1 2 1 +1 2 2 +1 2 3 + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2 + INTERSECT ALL + SELECT t4.c2 + FROM t4) +-- !query schema +struct +-- !query output +0 1 2 +0 1 3 + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2 + INTERSECT DISTINCT + SELECT t4.c2 + FROM t4) +-- !query schema +struct +-- !query output +0 1 2 +0 1 3 + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2 + EXCEPT ALL + SELECT t4.c2 + FROM t4) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2 + EXCEPT DISTINCT + SELECT t4.c2 + FROM t4) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + GROUP BY t2.c2 + UNION ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2 + GROUP BY t4.c2) +-- !query schema +struct +-- !query output +0 1 2 +0 1 3 + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c1 - t1.c1 + FROM t2 + GROUP BY t2.c1 - t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2 + GROUP BY t4.c2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"(c1 - c1)\",\"(c1 - c1) AS `(c1 - outer(spark_catalog.default.t1.c1))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 69, + "stopIndex" : 90, + "fragment" : "GROUP BY t2.c1 - t1.c1" + } ] +} + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT sum(t2.c2) over (order by t2.c1) + FROM t2 + WHERE t2.c1 >= t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4) +-- !query schema +struct +-- !query output +0 1 1 +0 1 1 +0 1 2 +0 1 3 +0 1 5 +0 1 5 +1 2 1 +1 2 1 +1 2 2 +1 2 3 + + +-- !query +SELECT * FROM t1 JOIN LATERAL (SELECT * FROM t2 WHERE t2.c1 = t1.c1) +UNION ALL +SELECT * FROM t1 JOIN t4 +-- !query schema +struct +-- !query output +0 1 0 1 +0 1 0 2 +0 1 0 2 +0 1 0 3 +0 1 1 1 +0 1 1 3 +1 2 0 1 +1 2 0 2 +1 2 1 1 +1 2 1 3 + + +-- !query +SELECT * FROM t1 JOIN LATERAL + (SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 = t1.c1) +UNION ALL +SELECT * FROM t2 JOIN LATERAL + (SELECT t1.c2 + FROM t1 + WHERE t2.c1 <= t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 < t2.c1) +-- !query schema +struct +-- !query output +0 1 1 +0 1 2 +0 1 2 +0 1 3 +0 2 1 +0 2 2 +0 3 1 +0 3 2 +1 2 1 +1 2 3 + + +-- !query +SELECT * FROM t1 JOIN LATERAL + ((SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + EXCEPT DISTINCT + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2) + UNION DISTINCT + (SELECT t4.c1 + FROM t4 + WHERE t4.c1 <= t1.c2 + INTERSECT ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 <> t1.c1) +) +-- !query schema +struct +-- !query output +0 1 1 +0 1 2 +0 1 3 +1 2 1 + + +-- !query +SELECT * FROM t1 JOIN LATERAL + ((SELECT t2.c2 + FROM t2 + WHERE t2.c1 = t1.c1 + UNION ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 > t1.c2) + INTERSECT DISTINCT + (SELECT t4.c1 + FROM t4 + WHERE t4.c1 <= t1.c2 + EXCEPT ALL + SELECT t4.c2 + FROM t4 + WHERE t4.c1 <> t1.c1) +) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM t1 JOIN LATERAL (SELECT sum(c1) FROM + (SELECT * + FROM t2 + WHERE t2.c1 <= t1.c1) lhs + LEFT SEMI JOIN + (SELECT * + FROM t4) rhs + ON lhs.c1 <=> rhs.c1 and lhs.c2 <=> rhs.c2 +) +-- !query schema +struct +-- !query output +0 1 0 +1 2 0 + + +-- !query +SELECT * FROM t1 JOIN LATERAL (SELECT sum(c1) FROM + (SELECT * + FROM t2 + WHERE t2.c1 <= t1.c1) lhs + LEFT SEMI JOIN + (SELECT * + FROM t4 + WHERE t4.c1 > t1.c2) rhs + ON lhs.c1 <=> rhs.c1 and lhs.c2 <=> rhs.c2 +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (c1#x > outer(c2#x))\n+- SubqueryAlias spark_catalog.default.t4\n +- View (`spark_catalog`.`default`.`t4`, [c1#x, c2#x])\n +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n +- LocalRelation [col1#x, col2#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 125, + "stopIndex" : 166, + "fragment" : "SELECT *\n FROM t4\n WHERE t4.c1 > t1.c2" + } ] +} + + +-- !query +SELECT * FROM LATERAL EXPLODE(ARRAY(1, 2)) +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT * FROM t1, LATERAL RANGE(3) +-- !query schema +struct +-- !query output +0 1 0 +0 1 1 +0 1 2 +1 2 0 +1 2 1 +1 2 2 + + +-- !query +SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3) +-- !query schema +struct +-- !query output +0 1 0 +0 1 1 +1 2 1 +1 2 2 + + +-- !query +SELECT * FROM t3, LATERAL EXPLODE(c2) t2(v) +-- !query schema +struct,v:int> +-- !query output +0 [0,1] 0 +0 [0,1] 1 +1 [2] 2 +NULL [4] 4 + + +-- !query +SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v) +-- !query schema +struct,v:int> +-- !query output +0 [0,1] 0 +0 [0,1] 1 +1 [2] 2 +2 [] NULL +NULL [4] 4 + + +-- !query +SELECT * FROM EXPLODE(ARRAY(1, 2)) t(v), LATERAL (SELECT v + 1) +-- !query schema +struct +-- !query output +1 2 +2 3 + + +-- !query +SELECT * FROM t1 JOIN LATERAL EXPLODE(ARRAY(c1, c2)) t(c3) ON t1.c1 = c3 +-- !query schema +struct +-- !query output +0 1 0 +1 2 1 + + +-- !query +SELECT * FROM t3 JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3 +-- !query schema +struct,c3:int> +-- !query output +0 [0,1] 0 + + +-- !query +SELECT * FROM t3 LEFT JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3 +-- !query schema +struct,c3:int> +-- !query output +0 [0,1] 0 +1 [2] NULL +2 [] NULL +NULL [4] NULL + + +-- !query +SELECT * FROM t1, LATERAL (SELECT * FROM EXPLODE(ARRAY(c1, c2))) +-- !query schema +struct +-- !query output +0 1 0 +0 1 1 +1 2 1 +1 2 2 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT t1.c1 + c3 FROM EXPLODE(ARRAY(c1, c2)) t(c3)) +-- !query schema +struct +-- !query output +0 1 0 +0 1 1 +1 2 2 +1 2 3 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT t1.c1 + c3 FROM EXPLODE(ARRAY(c1, c2)) t(c3) WHERE t1.c2 > 1) +-- !query schema +struct +-- !query output +1 2 2 +1 2 3 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT * FROM EXPLODE(ARRAY(c1, c2)) l(x) JOIN EXPLODE(ARRAY(c2, c1)) r(y) ON x = y) +-- !query schema +struct +-- !query output +0 1 0 0 +0 1 1 1 +1 2 1 1 +1 2 2 2 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW array_struct(id, arr) AS VALUES + (1, ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))), + (2, ARRAY()), + (3, ARRAY(STRUCT(3, 'c'))) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM t1, LATERAL INLINE(ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))) +-- !query schema +struct +-- !query output +0 1 1 a +0 1 2 b +1 2 1 a +1 2 2 b + + +-- !query +SELECT c1, t.* FROM t1, LATERAL INLINE(ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))) t(x, y) +-- !query schema +struct +-- !query output +0 1 a +0 2 b +1 1 a +1 2 b + + +-- !query +SELECT * FROM array_struct JOIN LATERAL INLINE(arr) +-- !query schema +struct>,col1:int,col2:string> +-- !query output +1 [{"col1":1,"col2":"a"},{"col1":2,"col2":"b"}] 1 a +1 [{"col1":1,"col2":"a"},{"col1":2,"col2":"b"}] 2 b +3 [{"col1":3,"col2":"c"}] 3 c + + +-- !query +SELECT * FROM array_struct LEFT JOIN LATERAL INLINE(arr) t(k, v) ON id = k +-- !query schema +struct>,k:int,v:string> +-- !query output +1 [{"col1":1,"col2":"a"},{"col1":2,"col2":"b"}] 1 a +2 [] NULL NULL +3 [{"col1":3,"col2":"c"}] 3 c + + +-- !query +SELECT * FROM array_struct JOIN LATERAL INLINE_OUTER(arr) +-- !query schema +struct>,col1:int,col2:string> +-- !query output +1 [{"col1":1,"col2":"a"},{"col1":2,"col2":"b"}] 1 a +1 [{"col1":1,"col2":"a"},{"col1":2,"col2":"b"}] 2 b +2 [] NULL NULL +3 [{"col1":3,"col2":"c"}] 3 c + + +-- !query +DROP VIEW array_struct +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM LATERAL posexplode(ARRAY(1, 2)) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +SELECT * FROM t1, LATERAL posexplode(ARRAY(c1, c2)) t2(pos, c3) +-- !query schema +struct +-- !query output +0 1 0 0 +0 1 1 1 +1 2 0 1 +1 2 1 2 + + +-- !query +SELECT * FROM t1 JOIN LATERAL posexplode(ARRAY(c1, c2)) t(pos, c3) ON t1.c1 = c3 +-- !query schema +struct +-- !query output +0 1 0 0 +1 2 0 1 + + +-- !query +SELECT * FROM t3, LATERAL posexplode(c2) t2(pos, v) +-- !query schema +struct,pos:int,v:int> +-- !query output +0 [0,1] 0 0 +0 [0,1] 1 1 +1 [2] 0 2 +NULL [4] 0 4 + + +-- !query +SELECT * FROM t3 JOIN LATERAL posexplode(c2) t(pos, c3) ON t3.c1 = c3 +-- !query schema +struct,pos:int,c3:int> +-- !query output +0 [0,1] 0 0 + + +-- !query +SELECT * FROM t3, LATERAL posexplode_outer(c2) t2(pos, v) +-- !query schema +struct,pos:int,v:int> +-- !query output +0 [0,1] 0 0 +0 [0,1] 1 1 +1 [2] 0 2 +2 [] NULL NULL +NULL [4] 0 4 + + +-- !query +SELECT * FROM t3 LEFT JOIN LATERAL posexplode(c2) t(pos, c3) ON t3.c1 = c3 +-- !query schema +struct,pos:int,c3:int> +-- !query output +0 [0,1] 0 0 +1 [2] NULL NULL +2 [] NULL NULL +NULL [4] NULL NULL + + +-- !query +SELECT * FROM t3 LEFT JOIN LATERAL posexplode_outer(c2) t(pos, c3) ON t3.c1 = c3 +-- !query schema +struct,pos:int,c3:int> +-- !query output +0 [0,1] 0 0 +1 [2] NULL NULL +2 [] NULL NULL +NULL [4] NULL NULL + + +-- !query +CREATE OR REPLACE TEMP VIEW json_table(key, jstring) AS VALUES + ('1', '{"f1": "1", "f2": "2", "f3": 3, "f5": 5.23}'), + ('2', '{"f1": "1", "f3": "3", "f2": 2, "f4": 4.01}'), + ('3', '{"f1": 3, "f4": "4", "f3": "3", "f2": 2, "f5": 5.01}'), + ('4', cast(null as string)), + ('5', '{"f1": null, "f5": ""}'), + ('6', '[invalid JSON string]') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.key, t2.* FROM json_table t1, LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2 +-- !query schema +struct +-- !query output +1 1 2 3 NULL 5.23 +2 1 2 3 4.01 NULL +3 3 2 3 4 5.01 +4 NULL NULL NULL NULL NULL +5 NULL NULL NULL NULL +6 NULL NULL NULL NULL NULL + + +-- !query +SELECT t1.key, t2.* FROM json_table t1, LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2 WHERE t2.c0 IS NOT NULL +-- !query schema +struct +-- !query output +1 1 2 3 NULL 5.23 +2 1 2 3 4.01 NULL +3 3 2 3 4 5.01 + + +-- !query +SELECT t1.key, t2.* FROM json_table t1 + JOIN LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2(f1, f2, f3, f4, f5) + ON t1.key = t2.f1 +-- !query schema +struct +-- !query output +1 1 2 3 NULL 5.23 +3 3 2 3 4 5.01 + + +-- !query +SELECT t1.key, t2.* FROM json_table t1 + LEFT JOIN LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2(f1, f2, f3, f4, f5) + ON t1.key = t2.f1 +-- !query schema +struct +-- !query output +1 1 2 3 NULL 5.23 +2 NULL NULL NULL NULL NULL +3 3 2 3 4 5.01 +4 NULL NULL NULL NULL NULL +5 NULL NULL NULL NULL NULL +6 NULL NULL NULL NULL NULL + + +-- !query +DROP VIEW json_table +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t.* FROM t1, LATERAL stack(2, 'Key', c1, 'Value', c2) t +-- !query schema +struct +-- !query output +Key 0 +Key 1 +Value 1 +Value 2 + + +-- !query +SELECT t.* FROM t1 JOIN LATERAL stack(1, c1, c2) t(x, y) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +SELECT t.* FROM t1 JOIN t3 ON t1.c1 = t3.c1 JOIN LATERAL stack(1, t1.c2, t3.c2) t +-- !query schema +struct> +-- !query output +1 [0,1] +2 [2] + + +-- !query +SELECT t.* FROM t1, LATERAL stack(c1, c2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"outer(spark_catalog.default.t1.c1)\"", + "inputName" : "`n`", + "inputType" : "\"INT\"", + "sqlExpr" : "\"stack(outer(spark_catalog.default.t1.c1), outer(spark_catalog.default.t1.c2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 41, + "fragment" : "stack(c1, c2)" + } ] +} + + +-- !query +select * from t1 join lateral (select * from t2 where t1.c1 = t2.c1 and t1.c2 < t2.c2 limit 1) +-- !query schema +struct +-- !query output +0 1 0 2 + + +-- !query +select * from t1 join lateral (select * from t4 where t1.c1 <= t4.c1 order by t4.c2 limit 10) +-- !query schema +struct +-- !query output +0 1 0 1 +0 1 0 2 +0 1 1 1 +0 1 1 3 +1 2 1 1 +1 2 1 3 + + +-- !query +select * from t1 join lateral (select c1, min(c2) as m + from t2 where t1.c1 = t2.c1 and t1.c2 < t2.c2 + group by t2.c1 + order by m) +-- !query schema +struct +-- !query output +0 1 0 2 + + +-- !query +select * from t1 join lateral (select c1, min(c2) as m + from t4 where t1.c1 = t4.c1 + group by t4.c1 + limit 1) +-- !query schema +struct +-- !query output +0 1 0 1 +1 2 1 1 + + +-- !query +select * from t1 join lateral + ((select t4.c2 from t4 where t1.c1 <= t4.c1 order by t4.c2 limit 1) + union all + (select t4.c1 from t4 where t1.c1 = t4.c1 order by t4.c1 limit 3)) +-- !query schema +struct +-- !query output +0 1 0 +0 1 0 +0 1 1 +1 2 1 +1 2 1 +1 2 1 + + +-- !query +select * from t1 join lateral + (select * from + ((select t4.c2 as t from t4 where t1.c1 <= t4.c1) + union all + (select t4.c1 as t from t4 where t1.c1 = t4.c1)) as foo + order by foo.t limit 5) +-- !query schema +struct +-- !query output +0 1 0 +0 1 0 +0 1 1 +0 1 1 +0 1 2 +1 2 1 +1 2 1 +1 2 1 +1 2 3 + + +-- !query +select 1 +from t1 as t_outer +left join + lateral( + select b1,b2 + from + ( + select + t2.c1 as b1, + 1 as b2 + from t2 + union + select t_outer.c1 as b1, + null as b2 + ) as t_inner + where (t_inner.b1 < t_outer.c2 or t_inner.b1 is null) + and t_inner.b1 = t_outer.c1 + order by t_inner.b1,t_inner.b2 desc limit 1 + ) as lateral_table +-- !query schema +struct<1:int> +-- !query output +1 +1 + + +-- !query +DROP VIEW t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW t3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW t4 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/json-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/json-functions.sql.out new file mode 100644 index 000000000000..06b8ed88ff88 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/json-functions.sql.out @@ -0,0 +1,807 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select to_json(named_struct('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +{"a":1,"b":2} + + +-- !query +select to_json(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy')) +-- !query schema +struct +-- !query output +{"time":"26/08/2015"} + + +-- !query +select to_json(array(named_struct('a', 1, 'b', 2))) +-- !query schema +struct +-- !query output +[{"a":1,"b":2}] + + +-- !query +select to_json(map(named_struct('a', 1, 'b', 2), named_struct('a', 1, 'b', 2))) +-- !query schema +struct +-- !query output +{"[1,2]":{"a":1,"b":2}} + + +-- !query +select to_json(map('a', named_struct('a', 1, 'b', 2))) +-- !query schema +struct +-- !query output +{"a":{"a":1,"b":2}} + + +-- !query +select to_json(map('a', 1)) +-- !query schema +struct +-- !query output +{"a":1} + + +-- !query +select to_json(array(map('a',1))) +-- !query schema +struct +-- !query output +[{"a":1}] + + +-- !query +select to_json(array(map('a',1), map('b',2))) +-- !query schema +struct +-- !query output +[{"a":1},{"b":2}] + + +-- !query +select to_json(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_MAP_FUNCTION", + "sqlState" : "42K06", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "to_json(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE'))" + } ] +} + + +-- !query +select to_json(named_struct('a', 1, 'b', 2), map('mode', 1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_STRING_TYPE", + "sqlState" : "42K06", + "messageParameters" : { + "mapType" : "\"MAP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 60, + "fragment" : "to_json(named_struct('a', 1, 'b', 2), map('mode', 1))" + } ] +} + + +-- !query +select to_json() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[1, 2]", + "functionName" : "`to_json`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "to_json()" + } ] +} + + +-- !query +select from_json('{"a":1}', 'a INT') +-- !query schema +struct> +-- !query output +{"a":1} + + +-- !query +select from_json('{"time":"26/08/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')) +-- !query schema +struct> +-- !query output +{"time":2015-08-26 00:00:00} + + +-- !query +select from_json('{"a":1}', 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_SCHEMA.NON_STRING_LITERAL", + "sqlState" : "42K07", + "messageParameters" : { + "inputSchema" : "\"1\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "from_json('{\"a\":1}', 1)" + } ] +} + + +-- !query +select from_json('{"a":1}', 'a InvalidType') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'InvalidType'", + "hint" : ": extra input 'InvalidType'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "from_json('{\"a\":1}', 'a InvalidType')" + } ] +} + + +-- !query +select from_json('{"a":1}', 'a INT', named_struct('mode', 'PERMISSIVE')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_MAP_FUNCTION", + "sqlState" : "42K06", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "from_json('{\"a\":1}', 'a INT', named_struct('mode', 'PERMISSIVE'))" + } ] +} + + +-- !query +select from_json('{"a":1}', 'a INT', map('mode', 1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_STRING_TYPE", + "sqlState" : "42K06", + "messageParameters" : { + "mapType" : "\"MAP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "from_json('{\"a\":1}', 'a INT', map('mode', 1))" + } ] +} + + +-- !query +select from_json() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3]", + "functionName" : "`from_json`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "from_json()" + } ] +} + + +-- !query +SELECT json_tuple('{"a" : 1, "b" : 2}', CAST(NULL AS STRING), 'b', CAST(NULL AS STRING), 'a') +-- !query schema +struct +-- !query output +NULL 2 NULL 1 + + +-- !query +CREATE TEMPORARY VIEW jsonTable(jsonField, a) AS SELECT * FROM VALUES ('{"a": 1, "b": 2}', 'a') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT json_tuple(jsonField, 'b', CAST(NULL AS STRING), a) FROM jsonTable +-- !query schema +struct +-- !query output +2 NULL 1 + + +-- !query +SELECT json_tuple('{"a":"1"}', if(c1 < 1, null, 'a')) FROM ( SELECT rand() AS c1 ) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT json_tuple('{"a":"1"}', if(c1 < 1, null, 'a'), if(c2 < 1, null, 'a')) FROM ( SELECT 0 AS c1, rand() AS c2 ) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +DROP VIEW IF EXISTS jsonTable +-- !query schema +struct<> +-- !query output + + + +-- !query +select from_json('{"a":1, "b":2}', 'map') +-- !query schema +struct> +-- !query output +{"a":1,"b":2} + + +-- !query +select from_json('{"a":1, "b":"2"}', 'struct') +-- !query schema +struct> +-- !query output +{"a":1,"b":"2"} + + +-- !query +select schema_of_json('{"c1":0, "c2":[1]}') +-- !query schema +struct +-- !query output +STRUCT> + + +-- !query +select from_json('{"c1":[1, 2, 3]}', schema_of_json('{"c1":[0]}')) +-- !query schema +struct>> +-- !query output +{"c1":[1,2,3]} + + +-- !query +select from_json('[1, 2, 3]', 'array') +-- !query schema +struct> +-- !query output +[1,2,3] + + +-- !query +select from_json('[1, "2", 3]', 'array') +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select from_json('[1, 2, null]', 'array') +-- !query schema +struct> +-- !query output +[1,2,null] + + +-- !query +select from_json('[{"a": 1}, {"a":2}]', 'array>') +-- !query schema +struct>> +-- !query output +[{"a":1},{"a":2}] + + +-- !query +select from_json('{"a": 1}', 'array>') +-- !query schema +struct>> +-- !query output +[{"a":1}] + + +-- !query +select from_json('[null, {"a":2}]', 'array>') +-- !query schema +struct>> +-- !query output +[null,{"a":2}] + + +-- !query +select from_json('[{"a": 1}, {"b":2}]', 'array>') +-- !query schema +struct>> +-- !query output +[{"a":1},{"b":2}] + + +-- !query +select from_json('[{"a": 1}, 2]', 'array>') +-- !query schema +struct>> +-- !query output +NULL + + +-- !query +select from_json('{"d": "2012-12-15", "t": "2012-12-15 15:15:15"}', 'd date, t timestamp') +-- !query schema +struct> +-- !query output +{"d":2012-12-15,"t":2012-12-15 15:15:15} + + +-- !query +select from_json( + '{"d": "12/15 2012", "t": "12/15 2012 15:15:15"}', + 'd date, t timestamp', + map('dateFormat', 'MM/dd yyyy', 'timestampFormat', 'MM/dd yyyy HH:mm:ss')) +-- !query schema +struct> +-- !query output +{"d":2012-12-15,"t":2012-12-15 15:15:15} + + +-- !query +select from_json( + '{"d": "02-29"}', + 'd date', + map('dateFormat', 'MM-dd')) +-- !query schema +struct> +-- !query output +{"d":null} + + +-- !query +select from_json( + '{"t": "02-29"}', + 't timestamp', + map('timestampFormat', 'MM-dd')) +-- !query schema +struct> +-- !query output +{"t":null} + + +-- !query +select to_json(array('1', '2', '3')) +-- !query schema +struct +-- !query output +["1","2","3"] + + +-- !query +select to_json(array(array(1, 2, 3), array(4))) +-- !query schema +struct +-- !query output +[[1,2,3],[4]] + + +-- !query +select schema_of_json('{"c1":1}', map('primitivesAsString', 'true')) +-- !query schema +struct +-- !query output +STRUCT + + +-- !query +select schema_of_json('{"c1":01, "c2":0.1}', map('allowNumericLeadingZeros', 'true', 'prefersDecimal', 'true')) +-- !query schema +struct +-- !query output +STRUCT + + +-- !query +select schema_of_json(null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_NULL", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "json", + "sqlExpr" : "\"schema_of_json(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "schema_of_json(null)" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW jsonTable(jsonField, a) AS SELECT * FROM VALUES ('{"a": 1, "b": 2}', 'a') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT schema_of_json(jsonField) FROM jsonTable +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"jsonField\"", + "inputName" : "`json`", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"schema_of_json(jsonField)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "schema_of_json(jsonField)" + } ] +} + + +-- !query +select json_array_length(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select json_array_length(2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"json_array_length(2)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "json_array_length(2)" + } ] +} + + +-- !query +select json_array_length() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`json_array_length`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "json_array_length()" + } ] +} + + +-- !query +select json_array_length('') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select json_array_length('[]') +-- !query schema +struct +-- !query output +0 + + +-- !query +select json_array_length('[1,2,3]') +-- !query schema +struct +-- !query output +3 + + +-- !query +select json_array_length('[[1,2],[5,6,7]]') +-- !query schema +struct +-- !query output +2 + + +-- !query +select json_array_length('[{"a":123},{"b":"hello"}]') +-- !query schema +struct +-- !query output +2 + + +-- !query +select json_array_length('[1,2,3,[33,44],{"key":[2,3,4]}]') +-- !query schema +struct +-- !query output +5 + + +-- !query +select json_array_length('{"key":"not a json array"}') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select json_array_length('[1,2,3,4,5') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select json_object_keys() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`json_object_keys`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "json_object_keys()" + } ] +} + + +-- !query +select json_object_keys(null) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select json_object_keys(200) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"200\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"json_object_keys(200)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "json_object_keys(200)" + } ] +} + + +-- !query +select json_object_keys('') +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select json_object_keys('{}') +-- !query schema +struct> +-- !query output +[] + + +-- !query +select json_object_keys('{"key": 1}') +-- !query schema +struct> +-- !query output +["key"] + + +-- !query +select json_object_keys('{"key": "value", "key2": 2}') +-- !query schema +struct> +-- !query output +["key","key2"] + + +-- !query +select json_object_keys('{"arrayKey": [1, 2, 3]}') +-- !query schema +struct> +-- !query output +["arrayKey"] + + +-- !query +select json_object_keys('{"key":[1,2,3,{"key":"value"},[1,2,3]]}') +-- !query schema +struct> +-- !query output +["key"] + + +-- !query +select json_object_keys('{"f1":"abc","f2":{"f3":"a", "f4":"b"}}') +-- !query schema +struct> +-- !query output +["f1","f2"] + + +-- !query +select json_object_keys('{"k1": [1, 2, {"key": 5}], "k2": {"key2": [1, 2]}}') +-- !query schema +struct> +-- !query output +["k1","k2"] + + +-- !query +select json_object_keys('{[1,2]}') +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select json_object_keys('{"key": 45, "random_string"}') +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select json_object_keys('[1, 2, 3]') +-- !query schema +struct> +-- !query output +NULL + + +-- !query +DROP VIEW IF EXISTS jsonTable +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/keywords-enforced.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/keywords-enforced.sql.out new file mode 100644 index 000000000000..dbb4a9a14b52 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/keywords-enforced.sql.out @@ -0,0 +1,467 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT * from SQL_KEYWORDS() +-- !query schema +struct +-- !query output +ADD false +AFTER false +AGGREGATE false +ALL true +ALTER false +ALWAYS false +ANALYZE false +AND true +ANTI false +ANY true +ANY_VALUE false +ARCHIVE false +ARRAY false +AS true +ASC false +AT false +ATOMIC false +AUTHORIZATION true +BEGIN false +BETWEEN false +BIGINT false +BINARY false +BINDING false +BOOLEAN false +BOTH true +BUCKET false +BUCKETS false +BY false +BYTE false +CACHE false +CALL true +CALLED false +CASCADE false +CASE true +CAST true +CATALOG false +CATALOGS false +CHANGE false +CHAR false +CHARACTER false +CHECK true +CLEAR false +CLUSTER false +CLUSTERED false +CODEGEN false +COLLATE true +COLLATION true +COLLECTION false +COLUMN true +COLUMNS false +COMMENT false +COMMIT false +COMPACT false +COMPACTIONS false +COMPENSATION false +COMPUTE false +CONCATENATE false +CONDITION false +CONSTRAINT true +CONTAINS false +CONTINUE false +COST false +CREATE true +CROSS true +CUBE false +CURRENT false +CURRENT_DATE true +CURRENT_TIME true +CURRENT_TIMESTAMP true +CURRENT_USER true +DATA false +DATABASE false +DATABASES false +DATE false +DATEADD false +DATEDIFF false +DATE_ADD false +DATE_DIFF false +DAY false +DAYOFYEAR false +DAYS false +DBPROPERTIES false +DEC false +DECIMAL false +DECLARE false +DEFAULT false +DEFINED false +DEFINER false +DELETE false +DELIMITED false +DESC false +DESCRIBE false +DETERMINISTIC false +DFS false +DIRECTORIES false +DIRECTORY false +DISTINCT true +DISTRIBUTE false +DIV false +DO false +DOUBLE false +DROP false +ELSE true +ELSEIF false +END true +ESCAPE true +ESCAPED false +EVOLUTION false +EXCEPT true +EXCHANGE false +EXCLUDE false +EXECUTE true +EXISTS false +EXIT false +EXPLAIN false +EXPORT false +EXTEND false +EXTENDED false +EXTERNAL false +EXTRACT false +FALSE true +FETCH true +FIELDS false +FILEFORMAT false +FILTER true +FIRST false +FLOAT false +FOLLOWING false +FOR true +FOREIGN true +FORMAT false +FORMATTED false +FOUND false +FROM true +FULL true +FUNCTION false +FUNCTIONS false +GENERATED false +GLOBAL false +GRANT true +GROUP true +GROUPING false +HANDLER false +HAVING true +HOUR false +HOURS false +IDENTIFIER false +IDENTITY false +IF false +IGNORE false +ILIKE false +IMMEDIATE false +IMPORT false +IN true +INCLUDE false +INCREMENT false +INDEX false +INDEXES false +INNER true +INPATH false +INPUT false +INPUTFORMAT false +INSERT false +INT false +INTEGER false +INTERSECT true +INTERVAL false +INTO true +INVOKER false +IS true +ITEMS false +ITERATE false +JOIN true +JSON false +KEYS false +LANGUAGE false +LAST false +LATERAL true +LAZY false +LEADING true +LEAVE false +LEFT true +LIKE false +LIMIT false +LINES false +LIST false +LOAD false +LOCAL false +LOCATION false +LOCK false +LOCKS false +LOGICAL false +LONG false +LOOP false +MACRO false +MAP false +MATCHED false +MERGE false +MICROSECOND false +MICROSECONDS false +MILLISECOND false +MILLISECONDS false +MINUS false +MINUTE false +MINUTES false +MODIFIES false +MONTH false +MONTHS false +MSCK false +NAME false +NAMESPACE false +NAMESPACES false +NANOSECOND false +NANOSECONDS false +NATURAL true +NO false +NONE false +NOT true +NULL true +NULLS false +NUMERIC false +OF false +OFFSET true +ON true +ONLY true +OPTION false +OPTIONS false +OR true +ORDER true +OUT false +OUTER true +OUTPUTFORMAT false +OVER false +OVERLAPS true +OVERLAY false +OVERWRITE false +PARTITION false +PARTITIONED false +PARTITIONS false +PERCENT false +PIVOT false +PLACING false +POSITION false +PRECEDING false +PRIMARY true +PRINCIPALS false +PROPERTIES false +PURGE false +QUARTER false +QUERY false +RANGE false +READS false +REAL false +RECORDREADER false +RECORDWRITER false +RECOVER false +RECURSIVE true +REDUCE false +REFERENCES true +REFRESH false +RENAME false +REPAIR false +REPEAT false +REPEATABLE false +REPLACE false +RESET false +RESPECT false +RESTRICT false +RETURN false +RETURNS false +REVOKE false +RIGHT true +ROLE false +ROLES false +ROLLBACK false +ROLLUP false +ROW false +ROWS false +SCHEMA false +SCHEMAS false +SECOND false +SECONDS false +SECURITY false +SELECT true +SEMI false +SEPARATED false +SERDE false +SERDEPROPERTIES false +SESSION_USER true +SET false +SETS false +SHORT false +SHOW false +SINGLE false +SKEWED false +SMALLINT false +SOME true +SORT false +SORTED false +SOURCE false +SPECIFIC false +SQL true +SQLEXCEPTION false +SQLSTATE false +START false +STATISTICS false +STORED false +STRATIFY false +STRING false +STRUCT false +SUBSTR false +SUBSTRING false +SYNC false +SYSTEM_TIME false +SYSTEM_VERSION false +TABLE true +TABLES false +TABLESAMPLE false +TARGET false +TBLPROPERTIES false +TERMINATED false +THEN true +TIME true +TIMEDIFF false +TIMESTAMP false +TIMESTAMPADD false +TIMESTAMPDIFF false +TIMESTAMP_LTZ false +TIMESTAMP_NTZ false +TINYINT false +TO true +TOUCH false +TRAILING true +TRANSACTION false +TRANSACTIONS false +TRANSFORM false +TRIM false +TRUE false +TRUNCATE false +TRY_CAST false +TYPE false +UNARCHIVE false +UNBOUNDED false +UNCACHE false +UNION true +UNIQUE true +UNKNOWN true +UNLOCK false +UNPIVOT false +UNSET false +UNTIL false +UPDATE false +USE false +USER true +USING true +VALUE false +VALUES false +VAR false +VARCHAR false +VARIABLE false +VARIANT false +VERSION false +VIEW false +VIEWS false +VOID false +WEEK false +WEEKS false +WHEN true +WHERE true +WHILE false +WINDOW false +WITH true +WITHIN true +X false +YEAR false +YEARS false +ZONE false + + +-- !query +SELECT keyword from SQL_KEYWORDS() WHERE reserved +-- !query schema +struct +-- !query output +ALL +AND +ANY +AS +AUTHORIZATION +BOTH +CALL +CASE +CAST +CHECK +COLLATE +COLLATION +COLUMN +CONSTRAINT +CREATE +CROSS +CURRENT_DATE +CURRENT_TIME +CURRENT_TIMESTAMP +CURRENT_USER +DISTINCT +ELSE +END +ESCAPE +EXCEPT +EXECUTE +FALSE +FETCH +FILTER +FOR +FOREIGN +FROM +FULL +GRANT +GROUP +HAVING +IN +INNER +INTERSECT +INTO +IS +JOIN +LATERAL +LEADING +LEFT +NATURAL +NOT +NULL +OFFSET +ON +ONLY +OR +ORDER +OUTER +OVERLAPS +PRIMARY +RECURSIVE +REFERENCES +RIGHT +SELECT +SESSION_USER +SOME +SQL +TABLE +THEN +TIME +TO +TRAILING +UNION +UNIQUE +UNKNOWN +USER +USING +WHEN +WHERE +WITH +WITHIN diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/keywords.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/keywords.sql.out new file mode 100644 index 000000000000..fb4c72a88cad --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/keywords.sql.out @@ -0,0 +1,391 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT * from SQL_KEYWORDS() +-- !query schema +struct +-- !query output +ADD false +AFTER false +AGGREGATE false +ALL false +ALTER false +ALWAYS false +ANALYZE false +AND false +ANTI false +ANY false +ANY_VALUE false +ARCHIVE false +ARRAY false +AS false +ASC false +AT false +ATOMIC false +AUTHORIZATION false +BEGIN false +BETWEEN false +BIGINT false +BINARY false +BINDING false +BOOLEAN false +BOTH false +BUCKET false +BUCKETS false +BY false +BYTE false +CACHE false +CALL false +CALLED false +CASCADE false +CASE false +CAST false +CATALOG false +CATALOGS false +CHANGE false +CHAR false +CHARACTER false +CHECK false +CLEAR false +CLUSTER false +CLUSTERED false +CODEGEN false +COLLATE false +COLLATION false +COLLECTION false +COLUMN false +COLUMNS false +COMMENT false +COMMIT false +COMPACT false +COMPACTIONS false +COMPENSATION false +COMPUTE false +CONCATENATE false +CONDITION false +CONSTRAINT false +CONTAINS false +CONTINUE false +COST false +CREATE false +CROSS false +CUBE false +CURRENT false +CURRENT_DATE false +CURRENT_TIME false +CURRENT_TIMESTAMP false +CURRENT_USER false +DATA false +DATABASE false +DATABASES false +DATE false +DATEADD false +DATEDIFF false +DATE_ADD false +DATE_DIFF false +DAY false +DAYOFYEAR false +DAYS false +DBPROPERTIES false +DEC false +DECIMAL false +DECLARE false +DEFAULT false +DEFINED false +DEFINER false +DELETE false +DELIMITED false +DESC false +DESCRIBE false +DETERMINISTIC false +DFS false +DIRECTORIES false +DIRECTORY false +DISTINCT false +DISTRIBUTE false +DIV false +DO false +DOUBLE false +DROP false +ELSE false +ELSEIF false +END false +ESCAPE false +ESCAPED false +EVOLUTION false +EXCEPT false +EXCHANGE false +EXCLUDE false +EXECUTE false +EXISTS false +EXIT false +EXPLAIN false +EXPORT false +EXTEND false +EXTENDED false +EXTERNAL false +EXTRACT false +FALSE false +FETCH false +FIELDS false +FILEFORMAT false +FILTER false +FIRST false +FLOAT false +FOLLOWING false +FOR false +FOREIGN false +FORMAT false +FORMATTED false +FOUND false +FROM false +FULL false +FUNCTION false +FUNCTIONS false +GENERATED false +GLOBAL false +GRANT false +GROUP false +GROUPING false +HANDLER false +HAVING false +HOUR false +HOURS false +IDENTIFIER false +IDENTITY false +IF false +IGNORE false +ILIKE false +IMMEDIATE false +IMPORT false +IN false +INCLUDE false +INCREMENT false +INDEX false +INDEXES false +INNER false +INPATH false +INPUT false +INPUTFORMAT false +INSERT false +INT false +INTEGER false +INTERSECT false +INTERVAL false +INTO false +INVOKER false +IS false +ITEMS false +ITERATE false +JOIN false +JSON false +KEYS false +LANGUAGE false +LAST false +LATERAL false +LAZY false +LEADING false +LEAVE false +LEFT false +LIKE false +LIMIT false +LINES false +LIST false +LOAD false +LOCAL false +LOCATION false +LOCK false +LOCKS false +LOGICAL false +LONG false +LOOP false +MACRO false +MAP false +MATCHED false +MERGE false +MICROSECOND false +MICROSECONDS false +MILLISECOND false +MILLISECONDS false +MINUS false +MINUTE false +MINUTES false +MODIFIES false +MONTH false +MONTHS false +MSCK false +NAME false +NAMESPACE false +NAMESPACES false +NANOSECOND false +NANOSECONDS false +NATURAL false +NO false +NONE false +NOT false +NULL false +NULLS false +NUMERIC false +OF false +OFFSET false +ON false +ONLY false +OPTION false +OPTIONS false +OR false +ORDER false +OUT false +OUTER false +OUTPUTFORMAT false +OVER false +OVERLAPS false +OVERLAY false +OVERWRITE false +PARTITION false +PARTITIONED false +PARTITIONS false +PERCENT false +PIVOT false +PLACING false +POSITION false +PRECEDING false +PRIMARY false +PRINCIPALS false +PROPERTIES false +PURGE false +QUARTER false +QUERY false +RANGE false +READS false +REAL false +RECORDREADER false +RECORDWRITER false +RECOVER false +RECURSIVE false +REDUCE false +REFERENCES false +REFRESH false +RENAME false +REPAIR false +REPEAT false +REPEATABLE false +REPLACE false +RESET false +RESPECT false +RESTRICT false +RETURN false +RETURNS false +REVOKE false +RIGHT false +ROLE false +ROLES false +ROLLBACK false +ROLLUP false +ROW false +ROWS false +SCHEMA false +SCHEMAS false +SECOND false +SECONDS false +SECURITY false +SELECT false +SEMI false +SEPARATED false +SERDE false +SERDEPROPERTIES false +SESSION_USER false +SET false +SETS false +SHORT false +SHOW false +SINGLE false +SKEWED false +SMALLINT false +SOME false +SORT false +SORTED false +SOURCE false +SPECIFIC false +SQL false +SQLEXCEPTION false +SQLSTATE false +START false +STATISTICS false +STORED false +STRATIFY false +STRING false +STRUCT false +SUBSTR false +SUBSTRING false +SYNC false +SYSTEM_TIME false +SYSTEM_VERSION false +TABLE false +TABLES false +TABLESAMPLE false +TARGET false +TBLPROPERTIES false +TERMINATED false +THEN false +TIME false +TIMEDIFF false +TIMESTAMP false +TIMESTAMPADD false +TIMESTAMPDIFF false +TIMESTAMP_LTZ false +TIMESTAMP_NTZ false +TINYINT false +TO false +TOUCH false +TRAILING false +TRANSACTION false +TRANSACTIONS false +TRANSFORM false +TRIM false +TRUE false +TRUNCATE false +TRY_CAST false +TYPE false +UNARCHIVE false +UNBOUNDED false +UNCACHE false +UNION false +UNIQUE false +UNKNOWN false +UNLOCK false +UNPIVOT false +UNSET false +UNTIL false +UPDATE false +USE false +USER false +USING false +VALUE false +VALUES false +VAR false +VARCHAR false +VARIABLE false +VARIANT false +VERSION false +VIEW false +VIEWS false +VOID false +WEEK false +WEEKS false +WHEN false +WHERE false +WHILE false +WINDOW false +WITH false +WITHIN false +X false +YEAR false +YEARS false +ZONE false + + +-- !query +SELECT keyword from SQL_KEYWORDS() WHERE reserved +-- !query schema +struct +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/like-all.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/like-all.sql.out new file mode 100644 index 000000000000..ce3cc3a0c0f0 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/like-all.sql.out @@ -0,0 +1,144 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW like_all_table AS SELECT * FROM (VALUES + ('google', '%oo%'), + ('facebook', '%oo%'), + ('linkedin', '%in')) + as t1(company, pat) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT company FROM like_all_table WHERE company LIKE ALL ('%oo%', '%go%') +-- !query schema +struct +-- !query output +google + + +-- !query +SELECT company FROM like_all_table WHERE company LIKE ALL ('microsoft', '%yoo%') +-- !query schema +struct +-- !query output + + + +-- !query +SELECT + company, + CASE + WHEN company LIKE ALL ('%oo%', '%go%') THEN 'Y' + ELSE 'N' + END AS is_available, + CASE + WHEN company LIKE ALL ('%oo%', 'go%') OR company LIKE ALL ('%in', 'ms%') THEN 'Y' + ELSE 'N' + END AS mix +FROM like_all_table +-- !query schema +struct +-- !query output +facebook N N +google Y Y +linkedin N N + + +-- !query +SELECT company FROM like_all_table WHERE company LIKE ALL ('%oo%', pat) +-- !query schema +struct +-- !query output +facebook +google + + +-- !query +SELECT company FROM like_all_table WHERE company NOT LIKE ALL ('%oo%', '%in', 'fa%') +-- !query schema +struct +-- !query output + + + +-- !query +SELECT company FROM like_all_table WHERE company NOT LIKE ALL ('microsoft', '%yoo%') +-- !query schema +struct +-- !query output +facebook +google +linkedin + + +-- !query +SELECT company FROM like_all_table WHERE company NOT LIKE ALL ('%oo%', 'fa%') +-- !query schema +struct +-- !query output +linkedin + + +-- !query +SELECT company FROM like_all_table WHERE NOT company LIKE ALL ('%oo%', 'fa%') +-- !query schema +struct +-- !query output +google +linkedin + + +-- !query +SELECT company FROM like_all_table WHERE company LIKE ALL ('%oo%', NULL) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT company FROM like_all_table WHERE company NOT LIKE ALL ('%oo%', NULL) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT company FROM like_all_table WHERE company LIKE ALL (NULL, NULL) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT company FROM like_all_table WHERE company NOT LIKE ALL (NULL, NULL) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT company FROM like_all_table WHERE company LIKE ALL () +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Expected something between '(' and ')'." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 50, + "stopIndex" : 60, + "fragment" : "LIKE ALL ()" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/like-any.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/like-any.sql.out new file mode 100644 index 000000000000..8fcfbb607a35 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/like-any.sql.out @@ -0,0 +1,150 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW like_any_table AS SELECT * FROM (VALUES + ('google', '%oo%'), + ('facebook', '%oo%'), + ('linkedin', '%in')) + as t1(company, pat) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT company FROM like_any_table WHERE company LIKE ANY ('%oo%', '%in', 'fa%') +-- !query schema +struct +-- !query output +facebook +google +linkedin + + +-- !query +SELECT company FROM like_any_table WHERE company LIKE ANY ('microsoft', '%yoo%') +-- !query schema +struct +-- !query output + + + +-- !query +select + company, + CASE + WHEN company LIKE ANY ('%oo%', '%in', 'fa%') THEN 'Y' + ELSE 'N' + END AS is_available, + CASE + WHEN company LIKE ANY ('%oo%', 'fa%') OR company LIKE ANY ('%in', 'ms%') THEN 'Y' + ELSE 'N' + END AS mix +FROM like_any_table +-- !query schema +struct +-- !query output +facebook Y Y +google Y Y +linkedin Y Y + + +-- !query +SELECT company FROM like_any_table WHERE company LIKE ANY ('%zz%', pat) +-- !query schema +struct +-- !query output +facebook +google +linkedin + + +-- !query +SELECT company FROM like_any_table WHERE company NOT LIKE ANY ('%oo%', '%in', 'fa%') +-- !query schema +struct +-- !query output +facebook +google +linkedin + + +-- !query +SELECT company FROM like_any_table WHERE company NOT LIKE ANY ('microsoft', '%yoo%') +-- !query schema +struct +-- !query output +facebook +google +linkedin + + +-- !query +SELECT company FROM like_any_table WHERE company NOT LIKE ANY ('%oo%', 'fa%') +-- !query schema +struct +-- !query output +google +linkedin + + +-- !query +SELECT company FROM like_any_table WHERE NOT company LIKE ANY ('%oo%', 'fa%') +-- !query schema +struct +-- !query output +linkedin + + +-- !query +SELECT company FROM like_any_table WHERE company LIKE ANY ('%oo%', NULL) +-- !query schema +struct +-- !query output +facebook +google + + +-- !query +SELECT company FROM like_any_table WHERE company NOT LIKE ANY ('%oo%', NULL) +-- !query schema +struct +-- !query output +linkedin + + +-- !query +SELECT company FROM like_any_table WHERE company LIKE ANY (NULL, NULL) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT company FROM like_any_table WHERE company NOT LIKE ANY (NULL, NULL) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT company FROM like_any_table WHERE company LIKE ANY () +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Expected something between '(' and ')'." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 50, + "stopIndex" : 60, + "fragment" : "LIKE ANY ()" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/limit.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/limit.sql.out new file mode 100644 index 000000000000..750e6a6f5cbe --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/limit.sql.out @@ -0,0 +1,204 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT * FROM testdata LIMIT 2 +-- !query schema +struct +-- !query output +1 1 +2 2 + + +-- !query +SELECT * FROM arraydata LIMIT 2 +-- !query schema +struct,nestedarraycol:array>> +-- !query output +[1,2,3] [[1,2,3]] +[2,3,4] [[2,3,4]] + + +-- !query +SELECT * FROM mapdata LIMIT 2 +-- !query schema +struct> +-- !query output +{1:"a1",2:"b1",3:"c1",4:"d1",5:"e1"} +{1:"a2",2:"b2",3:"c2",4:"d2"} + + +-- !query +SELECT * FROM testdata LIMIT 2 + 1 +-- !query schema +struct +-- !query output +1 1 +2 2 +3 3 + + +-- !query +SELECT * FROM testdata LIMIT CAST(1 AS int) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT * FROM testdata LIMIT -1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_NEGATIVE", + "sqlState" : "42K0E", + "messageParameters" : { + "expr" : "\"-1\"", + "name" : "limit", + "v" : "-1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 31, + "fragment" : "-1" + } ] +} + + +-- !query +SELECT * FROM testData TABLESAMPLE (-1 ROWS) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_NEGATIVE", + "sqlState" : "42K0E", + "messageParameters" : { + "expr" : "\"-1\"", + "name" : "limit", + "v" : "-1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 37, + "stopIndex" : 38, + "fragment" : "-1" + } ] +} + + +-- !query +SELECT * FROM testdata LIMIT CAST(1 AS INT) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT * FROM testdata LIMIT CAST(NULL AS INT) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_NULL", + "sqlState" : "42K0E", + "messageParameters" : { + "expr" : "\"CAST(NULL AS INT)\"", + "name" : "limit" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 46, + "fragment" : "CAST(NULL AS INT)" + } ] +} + + +-- !query +SELECT * FROM testdata LIMIT key > 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE", + "sqlState" : "42K0E", + "messageParameters" : { + "expr" : "\"(key > 3)\"", + "name" : "limit" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 36, + "fragment" : "key > 3" + } ] +} + + +-- !query +SELECT * FROM testdata LIMIT true +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.DATA_TYPE", + "sqlState" : "42K0E", + "messageParameters" : { + "dataType" : "\"BOOLEAN\"", + "expr" : "\"true\"", + "name" : "limit" + } +} + + +-- !query +SELECT * FROM testdata LIMIT 'a' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.DATA_TYPE", + "sqlState" : "42K0E", + "messageParameters" : { + "dataType" : "\"STRING\"", + "expr" : "\"a\"", + "name" : "limit" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 32, + "fragment" : "'a'" + } ] +} + + +-- !query +SELECT * FROM (SELECT * FROM range(10) LIMIT 5) WHERE id > 3 +-- !query schema +struct +-- !query output +4 + + +-- !query +SELECT * FROM testdata WHERE key < 3 LIMIT ALL +-- !query schema +struct +-- !query output +1 1 +2 2 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/linear-regression.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/linear-regression.sql.out new file mode 100644 index 000000000000..e511ea75aae5 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/linear-regression.sql.out @@ -0,0 +1,276 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testRegression AS SELECT * FROM VALUES +(1, 10, null), (2, 10, 11), (2, 20, 22), (2, 25, null), (2, 30, 35), (2, null, 40) +AS testRegression(k, y, x) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT regr_count(y, x) FROM testRegression +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT regr_count(y, x) FROM testRegression WHERE x IS NOT NULL +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT k, count(*), regr_count(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 1 0 +2 5 3 + + +-- !query +SELECT k, count(*) FILTER (WHERE x IS NOT NULL), regr_count(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 0 0 +2 4 3 + + +-- !query +SELECT regr_r2(y, x) FROM testRegression +-- !query schema +struct +-- !query output +0.997690531177829 + + +-- !query +SELECT regr_r2(y, x) FROM testRegression WHERE x IS NOT NULL +-- !query schema +struct +-- !query output +0.997690531177829 + + +-- !query +SELECT k, corr(y, x), regr_r2(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL NULL +2 0.9988445981121533 0.997690531177829 + + +-- !query +SELECT k, corr(y, x) FILTER (WHERE x IS NOT NULL), regr_r2(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL NULL +2 0.9988445981121533 0.997690531177829 + + +-- !query +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression +-- !query schema +struct +-- !query output +22.666666666666668 20.0 + + +-- !query +SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +22.666666666666668 20.0 + + +-- !query +SELECT k, avg(x), avg(y), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL 10.0 NULL NULL +2 27.0 21.25 22.666666666666668 20.0 + + +-- !query +SELECT k, avg(x) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), avg(y) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL NULL NULL NULL +2 22.666666666666668 20.0 22.666666666666668 20.0 + + +-- !query +SELECT regr_sxx(y, x) FROM testRegression +-- !query schema +struct +-- !query output +288.6666666666667 + + +-- !query +SELECT regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +288.6666666666667 + + +-- !query +SELECT k, regr_sxx(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 288.6666666666667 + + +-- !query +SELECT k, regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 288.6666666666667 + + +-- !query +SELECT regr_sxy(y, x) FROM testRegression +-- !query schema +struct +-- !query output +240.0 + + +-- !query +SELECT regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +240.0 + + +-- !query +SELECT k, regr_sxy(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 240.0 + + +-- !query +SELECT k, regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 240.0 + + +-- !query +SELECT regr_syy(y, x) FROM testRegression +-- !query schema +struct +-- !query output +200.0 + + +-- !query +SELECT regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +200.0 + + +-- !query +SELECT k, regr_syy(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 200.0 + + +-- !query +SELECT k, regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 200.0 + + +-- !query +SELECT regr_slope(y, x) FROM testRegression +-- !query schema +struct +-- !query output +0.8314087759815242 + + +-- !query +SELECT regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +0.8314087759815242 + + +-- !query +SELECT k, regr_slope(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 0.8314087759815242 + + +-- !query +SELECT k, regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 0.8314087759815242 + + +-- !query +SELECT regr_intercept(y, x) FROM testRegression +-- !query schema +struct +-- !query output +1.1547344110854496 + + +-- !query +SELECT regr_intercept(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL +-- !query schema +struct +-- !query output +1.1547344110854496 + + +-- !query +SELECT k, regr_intercept(y, x) FROM testRegression GROUP BY k +-- !query schema +struct +-- !query output +1 NULL +2 1.1547344110854496 + + +-- !query +SELECT k, regr_intercept(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k +-- !query schema +struct +-- !query output +2 1.1547344110854496 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/listagg-collations.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/listagg-collations.sql.out new file mode 100644 index 000000000000..1f8c5822e7d8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/listagg-collations.sql.out @@ -0,0 +1,82 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT listagg(c1) WITHIN GROUP (ORDER BY c1 COLLATE utf8_binary) FROM (VALUES ('a'), ('A'), ('b'), ('B')) AS t(c1) +-- !query schema +struct +-- !query output +ABab + + +-- !query +WITH t(c1) AS (SELECT listagg(DISTINCT col1 COLLATE utf8_binary) FROM (VALUES ('a'), ('A'), ('b'), ('B'))) SELECT len(c1), regexp_count(c1, 'a'), regexp_count(c1, 'b'), regexp_count(c1, 'A'), regexp_count(c1, 'B') FROM t +-- !query schema +struct +-- !query output +4 1 1 1 1 + + +-- !query +WITH t(c1) AS (SELECT listagg(col1) WITHIN GROUP (ORDER BY col1) FROM (VALUES ('abc '), ('abc '), ('abc\n'), ('abc'), ('x'))) SELECT replace(replace(c1, ' ', ''), '\n', '$') FROM t +-- !query schema +struct +-- !query output +abcabc$abcabcx + + +-- !query +SELECT lower(listagg(c1) WITHIN GROUP (ORDER BY c1 COLLATE utf8_lcase)) FROM (VALUES ('a'), ('A'), ('b'), ('B')) AS t(c1) +-- !query schema +struct +-- !query output +aabb + + +-- !query +WITH t(c1) AS (SELECT lower(listagg(DISTINCT col1 COLLATE utf8_lcase)) FROM (VALUES ('a'), ('A'), ('b'), ('B'))) SELECT len(c1), regexp_count(c1, 'a'), regexp_count(c1, 'b') FROM t +-- !query schema +struct +-- !query output +2 1 1 + + +-- !query +SELECT lower(listagg(DISTINCT c1 COLLATE utf8_lcase) WITHIN GROUP (ORDER BY c1 COLLATE utf8_lcase)) FROM (VALUES ('a'), ('B'), ('b'), ('A')) AS t(c1) +-- !query schema +struct +-- !query output +ab + + +-- !query +WITH t(c1) AS (SELECT replace(listagg(DISTINCT col1 COLLATE unicode_rtrim) COLLATE utf8_binary, ' ', '') FROM (VALUES ('xbc '), ('xbc '), ('a'), ('xbc'))) SELECT len(c1), regexp_count(c1, 'a'), regexp_count(c1, 'xbc') FROM t +-- !query schema +struct +-- !query output +4 1 1 + + +-- !query +WITH t(c1) AS (SELECT listagg(col1) WITHIN GROUP (ORDER BY col1 COLLATE unicode_rtrim) FROM (VALUES ('abc '), ('abc\n'), ('abc'), ('x'))) SELECT replace(replace(c1, ' ', ''), '\n', '$') FROM t +-- !query schema +struct +-- !query output +abcabcabc$x + + +-- !query +SELECT listagg(DISTINCT c1 COLLATE utf8_lcase) WITHIN GROUP (ORDER BY c1 COLLATE utf8_binary) FROM (VALUES ('a'), ('b'), ('A'), ('B')) AS t(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.MISMATCH_WITH_DISTINCT_INPUT", + "sqlState" : "42K0K", + "messageParameters" : { + "funcArg" : "\"collate(c1, utf8_lcase)\"", + "funcName" : "`listagg`", + "orderingExpr" : "\"collate(c1, utf8_binary)\"" + } +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/listagg.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/listagg.sql.out new file mode 100644 index 000000000000..22ffceedb627 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/listagg.sql.out @@ -0,0 +1,376 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMP VIEW df AS +SELECT * FROM (VALUES ('a', 'b'), ('a', 'c'), ('b', 'c'), ('b', 'd'), (NULL, NULL)) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMP VIEW df2 AS +SELECT * FROM (VALUES (1, true), (2, false), (3, false)) +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH t(col) AS (SELECT listagg(col2) FROM df GROUP BY col1) SELECT len(col), regexp_count(col, 'a'), regexp_count(col, 'b'), regexp_count(col, 'c'), regexp_count(col, 'd') FROM t +-- !query schema +struct +-- !query output +2 0 0 1 1 +2 0 1 1 0 +NULL NULL NULL NULL NULL + + +-- !query +WITH t(col) AS (SELECT string_agg(col2) FROM df GROUP BY col1) SELECT len(col), regexp_count(col, 'a'), regexp_count(col, 'b'), regexp_count(col, 'c'), regexp_count(col, 'd') FROM t +-- !query schema +struct +-- !query output +2 0 0 1 1 +2 0 1 1 0 +NULL NULL NULL NULL NULL + + +-- !query +WITH t(col) AS (SELECT listagg(col2, NULL) FROM df GROUP BY col1) SELECT len(col), regexp_count(col, 'a'), regexp_count(col, 'b'), regexp_count(col, 'c'), regexp_count(col, 'd') FROM t +-- !query schema +struct +-- !query output +2 0 0 1 1 +2 0 1 1 0 +NULL NULL NULL NULL NULL + + +-- !query +SELECT listagg(col2) FROM df WHERE 1 != 1 +-- !query schema +struct +-- !query output +NULL + + +-- !query +WITH t(col) AS (SELECT listagg(col2, '|') FROM df GROUP BY col1) SELECT len(col), regexp_count(col, 'a'), regexp_count(col, 'b'), regexp_count(col, 'c'), regexp_count(col, 'd') FROM t +-- !query schema +struct +-- !query output +3 0 0 1 1 +3 0 1 1 0 +NULL NULL NULL NULL NULL + + +-- !query +WITH t(col) AS (SELECT listagg(col1) FROM df) SELECT len(col), regexp_count(col, 'a'), regexp_count(col, 'b') FROM t +-- !query schema +struct +-- !query output +4 2 2 + + +-- !query +WITH t(col) AS (SELECT listagg(DISTINCT col1) FROM df) SELECT len(col), regexp_count(col, 'a'), regexp_count(col, 'b') FROM t +-- !query schema +struct +-- !query output +2 1 1 + + +-- !query +SELECT listagg(col1) WITHIN GROUP (ORDER BY col1) FROM df +-- !query schema +struct +-- !query output +aabb + + +-- !query +SELECT listagg(col1) WITHIN GROUP (ORDER BY col1 DESC) FROM df +-- !query schema +struct +-- !query output +bbaa + + +-- !query +SELECT listagg(col1) WITHIN GROUP (ORDER BY col1 DESC) OVER (PARTITION BY col2) FROM df +-- !query schema +struct +-- !query output +NULL +a +b +ba +ba + + +-- !query +SELECT listagg(col1) WITHIN GROUP (ORDER BY col2) FROM df +-- !query schema +struct +-- !query output +aabb + + +-- !query +WITH t(col) AS (SELECT listagg(col1) WITHIN GROUP (ORDER BY col2 DESC) FROM df) SELECT (col == 'baba') || (col == 'bbaa') FROM t +-- !query schema +struct +-- !query output +truefalse + + +-- !query +WITH t(col) AS (SELECT listagg(col1, '|') WITHIN GROUP (ORDER BY col2 DESC) FROM df) SELECT (col == 'b|a|b|a') || (col == 'b|b|a|a') FROM t +-- !query schema +struct +-- !query output +truefalse + + +-- !query +SELECT listagg(col1, '|') WITHIN GROUP (ORDER BY col2 DESC) FROM df +-- !query schema +struct +-- !query output +b|a|b|a + + +-- !query +SELECT listagg(col1) WITHIN GROUP (ORDER BY col2 DESC, col1 ASC) FROM df +-- !query schema +struct +-- !query output +baba + + +-- !query +SELECT listagg(col1) WITHIN GROUP (ORDER BY col2 DESC, col1 DESC) FROM df +-- !query schema +struct +-- !query output +bbaa + + +-- !query +WITH t(col) AS (SELECT listagg(col1) FROM (VALUES (X'DEAD'), (X'BEEF'))) SELECT len(col), regexp_count(col, X'DEAD'), regexp_count(col, X'BEEF') FROM t +-- !query schema +struct +-- !query output +4 1 1 + + +-- !query +WITH t(col) AS (SELECT listagg(col1, NULL) FROM (VALUES (X'DEAD'), (X'BEEF'))) SELECT len(col), regexp_count(col, X'DEAD'), regexp_count(col, X'BEEF') FROM t +-- !query schema +struct +-- !query output +4 1 1 + + +-- !query +WITH t(col) AS (SELECT listagg(col1, X'42') FROM (VALUES (X'DEAD'), (X'BEEF'))) SELECT len(col), regexp_count(col, X'42'), regexp_count(col, X'DEAD'), regexp_count(col, X'BEEF') FROM t +-- !query schema +struct +-- !query output +5 1 1 1 + + +-- !query +WITH t(col1, col2) AS (SELECT listagg(col1), listagg(col2, ',') FROM df2) SELECT len(col1), regexp_count(col1, '1'), regexp_count(col1, '2'), regexp_count(col1, '3'), len(col2), regexp_count(col2, 'true'), regexp_count(col1, 'false') FROM t +-- !query schema +struct +-- !query output +3 1 1 1 16 1 0 + + +-- !query +SELECT listagg(c1) FROM (VALUES (ARRAY('a', 'b'))) AS t(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"c1\"", + "inputType" : "\"ARRAY\"", + "paramIndex" : "first", + "requiredType" : "(\"STRING\" or \"BINARY\")", + "sqlExpr" : "\"listagg(c1, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "listagg(c1)" + } ] +} + + +-- !query +SELECT listagg(c1, ', ') FROM (VALUES (X'DEAD'), (X'BEEF')) AS t(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"BINARY\" or \"STRING\")", + "functionName" : "`listagg`", + "sqlExpr" : "\"listagg(c1, , )\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "listagg(c1, ', ')" + } ] +} + + +-- !query +SELECT listagg(col2, col1) FROM df GROUP BY col1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"col1\"", + "inputName" : "`delimiter`", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"listagg(col2, col1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "listagg(col2, col1)" + } ] +} + + +-- !query +SELECT listagg(col1) OVER (ORDER BY col1) FROM df +-- !query schema +struct +-- !query output +NULL +aa +aa +aabb +aabb + + +-- !query +SELECT listagg(col1) WITHIN GROUP (ORDER BY col1) OVER (ORDER BY col1) FROM df +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "sqlState" : "42601", + "messageParameters" : { + "aggFunc" : "\"listagg(col1, NULL, col1 ASC NULLS FIRST)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "listagg(col1) WITHIN GROUP (ORDER BY col1) OVER (ORDER BY col1)" + } ] +} + + +-- !query +SELECT string_agg(col1) WITHIN GROUP (ORDER BY col1) OVER (ORDER BY col1) FROM df +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "sqlState" : "42601", + "messageParameters" : { + "aggFunc" : "\"listagg(col1, NULL, col1 ASC NULLS FIRST)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "string_agg(col1) WITHIN GROUP (ORDER BY col1) OVER (ORDER BY col1)" + } ] +} + + +-- !query +SELECT listagg(DISTINCT col1) OVER (ORDER BY col1) FROM df +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DISTINCT_WINDOW_FUNCTION_UNSUPPORTED", + "sqlState" : "0A000", + "messageParameters" : { + "windowExpr" : "\"listagg(DISTINCT col1, NULL) OVER (ORDER BY col1 ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "listagg(DISTINCT col1) OVER (ORDER BY col1)" + } ] +} + + +-- !query +SELECT listagg(DISTINCT col1) WITHIN GROUP (ORDER BY col2) FROM df +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.MISMATCH_WITH_DISTINCT_INPUT", + "sqlState" : "42K0K", + "messageParameters" : { + "funcArg" : "\"col1\"", + "funcName" : "`listagg`", + "orderingExpr" : "\"col2\"" + } +} + + +-- !query +SELECT listagg(DISTINCT col1) WITHIN GROUP (ORDER BY col1, col2) FROM df +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.MISMATCH_WITH_DISTINCT_INPUT", + "sqlState" : "42K0K", + "messageParameters" : { + "funcArg" : "\"col1\"", + "funcName" : "`listagg`", + "orderingExpr" : "\"col1\", \"col2\"" + } +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/literals.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/literals.sql.out new file mode 100644 index 000000000000..4e4c70cc333b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/literals.sql.out @@ -0,0 +1,787 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select null, Null, nUll +-- !query schema +struct +-- !query output +NULL NULL NULL + + +-- !query +select true, tRue, false, fALse +-- !query schema +struct +-- !query output +true true false false + + +-- !query +select 1Y +-- !query schema +struct<1:tinyint> +-- !query output +1 + + +-- !query +select 127Y, -128Y +-- !query schema +struct<127:tinyint,-128:tinyint> +-- !query output +127 -128 + + +-- !query +select 128Y +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_NUMERIC_LITERAL_RANGE", + "sqlState" : "22003", + "messageParameters" : { + "maxValue" : "127", + "minValue" : "-128", + "rawStrippedQualifier" : "128", + "typeName" : "tinyint" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 11, + "fragment" : "128Y" + } ] +} + + +-- !query +select 1S +-- !query schema +struct<1:smallint> +-- !query output +1 + + +-- !query +select 32767S, -32768S +-- !query schema +struct<32767:smallint,-32768:smallint> +-- !query output +32767 -32768 + + +-- !query +select 32768S +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_NUMERIC_LITERAL_RANGE", + "sqlState" : "22003", + "messageParameters" : { + "maxValue" : "32767", + "minValue" : "-32768", + "rawStrippedQualifier" : "32768", + "typeName" : "smallint" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : "32768S" + } ] +} + + +-- !query +select 1L, 2147483648L +-- !query schema +struct<1:bigint,2147483648:bigint> +-- !query output +1 2147483648 + + +-- !query +select 9223372036854775807L, -9223372036854775808L +-- !query schema +struct<9223372036854775807:bigint,-9223372036854775808:bigint> +-- !query output +9223372036854775807 -9223372036854775808 + + +-- !query +select 9223372036854775808L +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_NUMERIC_LITERAL_RANGE", + "sqlState" : "22003", + "messageParameters" : { + "maxValue" : "9223372036854775807", + "minValue" : "-9223372036854775808", + "rawStrippedQualifier" : "9223372036854775808", + "typeName" : "bigint" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "9223372036854775808L" + } ] +} + + +-- !query +select 1, -1 +-- !query schema +struct<1:int,-1:int> +-- !query output +1 -1 + + +-- !query +select 2147483647, -2147483648 +-- !query schema +struct<2147483647:int,-2147483648:int> +-- !query output +2147483647 -2147483648 + + +-- !query +select 9223372036854775807, -9223372036854775808 +-- !query schema +struct<9223372036854775807:bigint,-9223372036854775808:bigint> +-- !query output +9223372036854775807 -9223372036854775808 + + +-- !query +select 9223372036854775808, -9223372036854775809 +-- !query schema +struct<9223372036854775808:decimal(19,0),-9223372036854775809:decimal(19,0)> +-- !query output +9223372036854775808 -9223372036854775809 + + +-- !query +select 1234567890123456789012345678901234567890 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION", + "sqlState" : "22003", + "messageParameters" : { + "maxPrecision" : "38", + "precision" : "40" + } +} + + +-- !query +select 1234567890123456789012345678901234567890.0 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION", + "sqlState" : "22003", + "messageParameters" : { + "maxPrecision" : "38", + "precision" : "41" + } +} + + +-- !query +select 1F, 1.2F, .10f, 0.10f +-- !query schema +struct<1.0:float,1.2:float,0.1:float,0.1:float> +-- !query output +1.0 1.2 0.1 0.1 + + +-- !query +select -1F, -1.2F, -.10F, -0.10F +-- !query schema +struct<-1.0:float,-1.2:float,-0.1:float,-0.1:float> +-- !query output +-1.0 -1.2 -0.1 -0.1 + + +-- !query +select -3.4028235E39f +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_NUMERIC_LITERAL_RANGE", + "sqlState" : "22003", + "messageParameters" : { + "maxValue" : "3.4028234663852886E+38", + "minValue" : "-3.4028234663852886E+38", + "rawStrippedQualifier" : "-3.4028235E39", + "typeName" : "float" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "-3.4028235E39f" + } ] +} + + +-- !query +select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 +-- !query schema +struct<1.0:double,1.2:double,1.0E10:double,150000.0:double,0.1:double,0.1:double,10000.0:double,90.0:double,90.0:double,90.0:double,90.0:double> +-- !query output +1.0 1.2 1.0E10 150000.0 0.1 0.1 10000.0 90.0 90.0 90.0 90.0 + + +-- !query +select -1D, -1.2D, -1e10, -1.5e5, -.10D, -0.10D, -.1e5 +-- !query schema +struct<-1.0:double,-1.2:double,-1.0E10:double,-150000.0:double,-0.1:double,-0.1:double,-10000.0:double> +-- !query output +-1.0 -1.2 -1.0E10 -150000.0 -0.1 -0.1 -10000.0 + + +-- !query +select .e3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'.'", + "hint" : "" + } +} + + +-- !query +select 1E309, -1E309 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_NUMERIC_LITERAL_RANGE", + "sqlState" : "22003", + "messageParameters" : { + "maxValue" : "1.7976931348623157E+308", + "minValue" : "-1.7976931348623157E+308", + "rawStrippedQualifier" : "1E309", + "typeName" : "double" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "1E309" + } ] +} + + +-- !query +select 0.3, -0.8, .5, -.18, 0.1111, .1111 +-- !query schema +struct<0.3:decimal(1,1),-0.8:decimal(1,1),0.5:decimal(1,1),-0.18:decimal(2,2),0.1111:decimal(4,4),0.1111:decimal(4,4)> +-- !query output +0.3 -0.8 0.5 -0.18 0.1111 0.1111 + + +-- !query +select 0.3 F, 0.4 D, 0.5 BD +-- !query schema +struct +-- !query output +0.3 0.4 0.5 + + +-- !query +select 123456789012345678901234567890123456789e10d, 123456789012345678901234567890123456789.1e10d +-- !query schema +struct<1.2345678901234568E48:double,1.2345678901234568E48:double> +-- !query output +1.2345678901234568E48 1.2345678901234568E48 + + +-- !query +select "Hello Peter!", 'hello lee!' +-- !query schema +struct +-- !query output +Hello Peter! hello lee! + + +-- !query +select 'hello' 'world', 'hello' " " 'lee' +-- !query schema +struct +-- !query output +helloworld hello lee + + +-- !query +select "hello 'peter'" +-- !query schema +struct +-- !query output +hello 'peter' + + +-- !query +select 'pattern%', 'no-pattern\%', 'pattern\\%', 'pattern\\\%' +-- !query schema +struct +-- !query output +pattern% no-pattern\% pattern\% pattern\\% + + +-- !query +select '\'', '"', '\n', '\r', '\t', 'Z' +-- !query schema +struct<':string,":string, +:string, :string, :string,Z:string> +-- !query output +' " + Z + + +-- !query +select '\110\145\154\154\157\041' +-- !query schema +struct +-- !query output +Hello! + + +-- !query +select '\u0057\u006F\u0072\u006C\u0064\u0020\u003A\u0029' +-- !query schema +struct +-- !query output +World :) + + +-- !query +select dAte '2016-03-12' +-- !query schema +struct +-- !query output +2016-03-12 + + +-- !query +select date 'mar 11 2016' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'mar 11 2016'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date 'mar 11 2016'" + } ] +} + + +-- !query +select tImEstAmp '2016-03-11 20:54:00.000' +-- !query schema +struct +-- !query output +2016-03-11 20:54:00 + + +-- !query +select timestamp '2016-33-11 20:54:00.000' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2016-33-11 20:54:00.000'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "timestamp '2016-33-11 20:54:00.000'" + } ] +} + + +-- !query +select GEO '(10,-6)' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_TYPED_LITERAL", + "sqlState" : "0A000", + "messageParameters" : { + "supportedTypes" : "\"DATE\", \"TIMESTAMP_NTZ\", \"TIMESTAMP_LTZ\", \"TIMESTAMP\", \"INTERVAL\", \"X\"", + "unsupportedType" : "\"GEO\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "GEO '(10,-6)'" + } ] +} + + +-- !query +select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD +-- !query schema +struct<90912830918230182310293801923652346786:decimal(38,0),1.230E-26:decimal(29,29),123.08:decimal(5,2)> +-- !query output +90912830918230182310293801923652346786 0.00000000000000000000000001230 123.08 + + +-- !query +select 1.20E-38BD +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION", + "sqlState" : "22003", + "messageParameters" : { + "maxPrecision" : "38", + "precision" : "40" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "1.20E-38BD" + } ] +} + + +-- !query +select x'2379ACFe' +-- !query schema +struct +-- !query output +#y�� + + +-- !query +select X'XuZ' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'XuZ'", + "valueType" : "\"X\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : "X'XuZ'" + } ] +} + + +-- !query +SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 +-- !query schema +struct<3.14:decimal(3,2),-3.14:decimal(3,2),3.14E8:double,3.14E-8:double,-3.14E8:double,-3.14E-8:double,3.14E8:double,3.14E8:double,3.14E-8:double> +-- !query output +3.14 -3.14 3.14E8 3.14E-8 -3.14E8 -3.14E-8 3.14E8 3.14E8 3.14E-8 + + +-- !query +select +date '1999-01-01' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"DATE '1999-01-01'\"", + "inputType" : "\"DATE\"", + "paramIndex" : "first", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ DATE '1999-01-01')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "+date '1999-01-01'" + } ] +} + + +-- !query +select +timestamp '1999-01-01' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"TIMESTAMP '1999-01-01 00:00:00'\"", + "inputType" : "\"TIMESTAMP\"", + "paramIndex" : "first", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ TIMESTAMP '1999-01-01 00:00:00')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "+timestamp '1999-01-01'" + } ] +} + + +-- !query +select +interval '1 day' +-- !query schema +struct<(+ INTERVAL '1' DAY):interval day> +-- !query output +1 00:00:00.000000000 + + +-- !query +select +map(1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"map(1, 2)\"", + "inputType" : "\"MAP\"", + "paramIndex" : "first", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ map(1, 2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "+map(1, 2)" + } ] +} + + +-- !query +select +array(1,2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"array(1, 2)\"", + "inputType" : "\"ARRAY\"", + "paramIndex" : "first", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ array(1, 2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "+array(1,2)" + } ] +} + + +-- !query +select +named_struct('a', 1, 'b', 'spark') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"named_struct(a, 1, b, spark)\"", + "inputType" : "\"STRUCT\"", + "paramIndex" : "first", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ named_struct(a, 1, b, spark))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "+named_struct('a', 1, 'b', 'spark')" + } ] +} + + +-- !query +select +X'1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"X'01'\"", + "inputType" : "\"BINARY\"", + "paramIndex" : "first", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ X'01')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "+X'1'" + } ] +} + + +-- !query +select -date '1999-01-01' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"DATE '1999-01-01'\"", + "inputType" : "\"DATE\"", + "paramIndex" : "first", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(- DATE '1999-01-01')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "-date '1999-01-01'" + } ] +} + + +-- !query +select -timestamp '1999-01-01' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"TIMESTAMP '1999-01-01 00:00:00'\"", + "inputType" : "\"TIMESTAMP\"", + "paramIndex" : "first", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(- TIMESTAMP '1999-01-01 00:00:00')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "-timestamp '1999-01-01'" + } ] +} + + +-- !query +select -x'2379ACFe' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"X'2379ACFE'\"", + "inputType" : "\"BINARY\"", + "paramIndex" : "first", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(- X'2379ACFE')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "-x'2379ACFe'" + } ] +} + + +-- !query +select -0, -0.0 +-- !query schema +struct<0:int,0.0:decimal(1,1)> +-- !query output +0 0.0 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/map.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/map.sql.out new file mode 100644 index 000000000000..71f5491287c3 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/map.sql.out @@ -0,0 +1,115 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select element_at(map(1, 'a', 2, 'b'), 5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select map(1, 'a', 2, 'b')[5] +-- !query schema +struct +-- !query output +NULL + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), 5) +-- !query schema +struct +-- !query output +false + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), 1) +-- !query schema +struct +-- !query output +true + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), 5.0) +-- !query schema +struct +-- !query output +false + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), 1.0) +-- !query schema +struct +-- !query output +true + + +-- !query +select map_contains_key(map(1.0, 'a', 2, 'b'), 5) +-- !query schema +struct +-- !query output +false + + +-- !query +select map_contains_key(map(1.0, 'a', 2, 'b'), 1) +-- !query schema +struct +-- !query output +true + + +-- !query +select map_contains_key(map('1', 'a', '2', 'b'), 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "\"MAP\"", + "functionName" : "`map_contains_key`", + "leftType" : "\"MAP\"", + "rightType" : "\"INT\"", + "sqlExpr" : "\"map_contains_key(map(1, a, 2, b), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "map_contains_key(map('1', 'a', '2', 'b'), 1)" + } ] +} + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), '1') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "\"MAP\"", + "functionName" : "`map_contains_key`", + "leftType" : "\"MAP\"", + "rightType" : "\"STRING\"", + "sqlExpr" : "\"map_contains_key(map(1, a, 2, b), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "map_contains_key(map(1, 'a', 2, 'b'), '1')" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/mask-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/mask-functions.sql.out new file mode 100644 index 000000000000..fdaad7168b84 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/mask-functions.sql.out @@ -0,0 +1,552 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT mask('AbCD123-@$#') +-- !query schema +struct +-- !query output +XxXXnnn-@$# + + +-- !query +SELECT mask('AbCD123-@$#', 'Q') +-- !query schema +struct +-- !query output +QxQQnnn-@$# + + +-- !query +SELECT mask('AbCD123-@$#', 'Q', 'q') +-- !query schema +struct +-- !query output +QqQQnnn-@$# + + +-- !query +SELECT mask('AbCD123-@$#', 'Q', 'q', 'd') +-- !query schema +struct +-- !query output +QqQQddd-@$# + + +-- !query +SELECT mask('AbCD123-@$#', 'Q', 'q', 'd', 'o') +-- !query schema +struct +-- !query output +QqQQdddoooo + + +-- !query +SELECT mask('AbCD123-@$#', 'Qa', 'qa', 'da', 'oa') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "upperChar", + "sqlExpr" : "\"mask(AbCD123-@$#, Qa, qa, da, oa)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "mask('AbCD123-@$#', 'Qa', 'qa', 'da', 'oa')" + } ] +} + + +-- !query +SELECT mask('AbCD123-@$#', NULL, 'q', 'd', 'o') +-- !query schema +struct +-- !query output +AqCDdddoooo + + +-- !query +SELECT mask('AbCD123-@$#', NULL, NULL, 'd', 'o') +-- !query schema +struct +-- !query output +AbCDdddoooo + + +-- !query +SELECT mask('AbCD123-@$#', NULL, NULL, NULL, 'o') +-- !query schema +struct +-- !query output +AbCD123oooo + + +-- !query +SELECT mask('AbCD123-@$#', NULL, NULL, NULL, NULL) +-- !query schema +struct +-- !query output +AbCD123-@$# + + +-- !query +SELECT mask(NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT mask(NULL, NULL, 'q', 'd', 'o') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT mask(NULL, NULL, NULL, 'd', 'o') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT mask(NULL, NULL, NULL, NULL, 'o') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT mask('AbCD123-@$#', NULL, NULL, NULL, NULL) +-- !query schema +struct +-- !query output +AbCD123-@$# + + +-- !query +SELECT mask(c1) from values ('AbCD123-@$#') as tab(c1) +-- !query schema +struct +-- !query output +XxXXnnn-@$# + + +-- !query +SELECT mask(c1, 'Q') from values ('AbCD123-@$#') as tab(c1) +-- !query schema +struct +-- !query output +QxQQnnn-@$# + + +-- !query +SELECT mask(c1, 'Q', 'q')from values ('AbCD123-@$#') as tab(c1) +-- !query schema +struct +-- !query output +QqQQnnn-@$# + + +-- !query +SELECT mask(c1, 'Q', 'q', 'd') from values ('AbCD123-@$#') as tab(c1) +-- !query schema +struct +-- !query output +QqQQddd-@$# + + +-- !query +SELECT mask(c1, 'Q', 'q', 'd', 'o') from values ('AbCD123-@$#') as tab(c1) +-- !query schema +struct +-- !query output +QqQQdddoooo + + +-- !query +SELECT mask(c1, NULL, 'q', 'd', 'o') from values ('AbCD123-@$#') as tab(c1) +-- !query schema +struct +-- !query output +AqCDdddoooo + + +-- !query +SELECT mask(c1, NULL, NULL, 'd', 'o') from values ('AbCD123-@$#') as tab(c1) +-- !query schema +struct +-- !query output +AbCDdddoooo + + +-- !query +SELECT mask(c1, NULL, NULL, NULL, 'o') from values ('AbCD123-@$#') as tab(c1) +-- !query schema +struct +-- !query output +AbCD123oooo + + +-- !query +SELECT mask(c1, NULL, NULL, NULL, NULL) from values ('AbCD123-@$#') as tab(c1) +-- !query schema +struct +-- !query output +AbCD123-@$# + + +-- !query +SELECT mask(c1, NULL, 'q', 'd', 'o') from values ('AbCD123-@$#') as tab(c1) +-- !query schema +struct +-- !query output +AqCDdddoooo + + +-- !query +SELECT mask(c1, 'Q', NULL, 'd', 'o') from values ('AbCD123-@$#') as tab(c1) +-- !query schema +struct +-- !query output +QbQQdddoooo + + +-- !query +SELECT mask(c1, 'Q', 'q', NULL, 'o') from values ('AbCD123-@$#') as tab(c1) +-- !query schema +struct +-- !query output +QqQQ123oooo + + +-- !query +SELECT mask(c1, 'Q', 'q', 'd', NULL) from values ('AbCD123-@$#') as tab(c1) +-- !query schema +struct +-- !query output +QqQQddd-@$# + + +-- !query +SELECT mask(NULL, 'Q', 'q', 'd', NULL) from values ('AbCD123-@$#') as tab(c1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT mask('abcd-EFGH-8765-4321') +-- !query schema +struct +-- !query output +xxxx-XXXX-nnnn-nnnn + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', 'Q') +-- !query schema +struct +-- !query output +xxxx-QQQQ-nnnn-nnnn + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', 'Q', 'q') +-- !query schema +struct +-- !query output +qqqq-QQQQ-nnnn-nnnn + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', 'Q', 'q', 'd') +-- !query schema +struct +-- !query output +qqqq-QQQQ-dddd-dddd + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', 'Q', 'q', 'd', '*') +-- !query schema +struct +-- !query output +qqqq*QQQQ*dddd*dddd + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', NULL, 'q', 'd', '*') +-- !query schema +struct +-- !query output +qqqq*EFGH*dddd*dddd + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', NULL, NULL, 'd', '*') +-- !query schema +struct +-- !query output +abcd*EFGH*dddd*dddd + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', NULL, NULL, NULL, '*') +-- !query schema +struct +-- !query output +abcd*EFGH*8765*4321 + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', NULL, NULL, NULL, NULL) +-- !query schema +struct +-- !query output +abcd-EFGH-8765-4321 + + +-- !query +SELECT mask(NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT mask(NULL, NULL, 'q', 'd', '*') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT mask(NULL, NULL, NULL, 'd', '*') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT mask(NULL, NULL, NULL, NULL, '*') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT mask(NULL, NULL, NULL, NULL, NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT mask(c1) from values ('abcd-EFGH-8765-4321') as tab(c1) +-- !query schema +struct +-- !query output +xxxx-XXXX-nnnn-nnnn + + +-- !query +SELECT mask(c1, 'Q') from values ('abcd-EFGH-8765-4321') as tab(c1) +-- !query schema +struct +-- !query output +xxxx-QQQQ-nnnn-nnnn + + +-- !query +SELECT mask(c1, 'Q', 'q')from values ('abcd-EFGH-8765-4321') as tab(c1) +-- !query schema +struct +-- !query output +qqqq-QQQQ-nnnn-nnnn + + +-- !query +SELECT mask(c1, 'Q', 'q', 'd') from values ('abcd-EFGH-8765-4321') as tab(c1) +-- !query schema +struct +-- !query output +qqqq-QQQQ-dddd-dddd + + +-- !query +SELECT mask(c1, 'Q', 'q', 'd', '*') from values ('abcd-EFGH-8765-4321') as tab(c1) +-- !query schema +struct +-- !query output +qqqq*QQQQ*dddd*dddd + + +-- !query +SELECT mask(c1, NULL, 'q', 'd', '*') from values ('abcd-EFGH-8765-4321') as tab(c1) +-- !query schema +struct +-- !query output +qqqq*EFGH*dddd*dddd + + +-- !query +SELECT mask(c1, NULL, NULL, 'd', '*') from values ('abcd-EFGH-8765-4321') as tab(c1) +-- !query schema +struct +-- !query output +abcd*EFGH*dddd*dddd + + +-- !query +SELECT mask(c1, NULL, NULL, NULL, '*') from values ('abcd-EFGH-8765-4321') as tab(c1) +-- !query schema +struct +-- !query output +abcd*EFGH*8765*4321 + + +-- !query +SELECT mask(c1, NULL, NULL, NULL, NULL) from values ('abcd-EFGH-8765-4321') as tab(c1) +-- !query schema +struct +-- !query output +abcd-EFGH-8765-4321 + + +-- !query +SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 'a') as t(c1, replaceArg) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"replaceArg\"", + "inputName" : "`upperChar`", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"mask(c1, replaceArg, x, n, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "mask(c1, replaceArg)" + } ] +} + + +-- !query +SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 'ABC') as t(c1, replaceArg) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"replaceArg\"", + "inputName" : "`upperChar`", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"mask(c1, replaceArg, x, n, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "mask(c1, replaceArg)" + } ] +} + + +-- !query +SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 123) as t(c1, replaceArg) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"replaceArg\"", + "inputType" : "\"INT\"", + "paramIndex" : "second", + "requiredType" : "\"STRING\"", + "sqlExpr" : "\"mask(c1, replaceArg, x, n, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "mask(c1, replaceArg)" + } ] +} + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', 'A', 'w', '') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "digitChar", + "sqlExpr" : "\"mask(abcd-EFGH-8765-4321, A, w, , NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "mask('abcd-EFGH-8765-4321', 'A', 'w', '')" + } ] +} + + +-- !query +SELECT mask('abcd-EFGH-8765-4321', 'A', 'abc') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INPUT_SIZE_NOT_ONE", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "lowerChar", + "sqlExpr" : "\"mask(abcd-EFGH-8765-4321, A, abc, n, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "mask('abcd-EFGH-8765-4321', 'A', 'abc')" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/math.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/math.sql.out new file mode 100644 index 000000000000..e2abcb099130 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/math.sql.out @@ -0,0 +1,904 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT round(25y, 1) +-- !query schema +struct +-- !query output +25 + + +-- !query +SELECT round(25y, 0) +-- !query schema +struct +-- !query output +25 + + +-- !query +SELECT round(25y, -1) +-- !query schema +struct +-- !query output +30 + + +-- !query +SELECT round(25y, -2) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT round(25y, -3) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT round(127y, -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "round(127y, -1)" + } ] +} + + +-- !query +SELECT round(-128y, -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "round(-128y, -1)" + } ] +} + + +-- !query +SELECT round(525s, 1) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT round(525s, 0) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT round(525s, -1) +-- !query schema +struct +-- !query output +530 + + +-- !query +SELECT round(525s, -2) +-- !query schema +struct +-- !query output +500 + + +-- !query +SELECT round(525s, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT round(32767s, -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "round(32767s, -1)" + } ] +} + + +-- !query +SELECT round(-32768s, -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "round(-32768s, -1)" + } ] +} + + +-- !query +SELECT round(525, 1) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT round(525, 0) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT round(525, -1) +-- !query schema +struct +-- !query output +530 + + +-- !query +SELECT round(525, -2) +-- !query schema +struct +-- !query output +500 + + +-- !query +SELECT round(525, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT round(2147483647, -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "round(2147483647, -1)" + } ] +} + + +-- !query +SELECT round(-2147483647, -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "round(-2147483647, -1)" + } ] +} + + +-- !query +SELECT round(525L, 1) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT round(525L, 0) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT round(525L, -1) +-- !query schema +struct +-- !query output +530 + + +-- !query +SELECT round(525L, -2) +-- !query schema +struct +-- !query output +500 + + +-- !query +SELECT round(525L, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT round(9223372036854775807L, -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "round(9223372036854775807L, -1)" + } ] +} + + +-- !query +SELECT round(-9223372036854775808L, -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "round(-9223372036854775808L, -1)" + } ] +} + + +-- !query +SELECT bround(25y, 1) +-- !query schema +struct +-- !query output +25 + + +-- !query +SELECT bround(25y, 0) +-- !query schema +struct +-- !query output +25 + + +-- !query +SELECT bround(25y, -1) +-- !query schema +struct +-- !query output +20 + + +-- !query +SELECT bround(25y, -2) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT bround(25y, -3) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT bround(127y, -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "bround(127y, -1)" + } ] +} + + +-- !query +SELECT bround(-128y, -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "bround(-128y, -1)" + } ] +} + + +-- !query +SELECT bround(525s, 1) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT bround(525s, 0) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT bround(525s, -1) +-- !query schema +struct +-- !query output +520 + + +-- !query +SELECT bround(525s, -2) +-- !query schema +struct +-- !query output +500 + + +-- !query +SELECT bround(525s, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT bround(32767s, -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "bround(32767s, -1)" + } ] +} + + +-- !query +SELECT bround(-32768s, -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "bround(-32768s, -1)" + } ] +} + + +-- !query +SELECT bround(525, 1) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT bround(525, 0) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT bround(525, -1) +-- !query schema +struct +-- !query output +520 + + +-- !query +SELECT bround(525, -2) +-- !query schema +struct +-- !query output +500 + + +-- !query +SELECT bround(525, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT bround(2147483647, -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "bround(2147483647, -1)" + } ] +} + + +-- !query +SELECT bround(-2147483647, -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "bround(-2147483647, -1)" + } ] +} + + +-- !query +SELECT bround(525L, 1) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT bround(525L, 0) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT bround(525L, -1) +-- !query schema +struct +-- !query output +520 + + +-- !query +SELECT bround(525L, -2) +-- !query schema +struct +-- !query output +500 + + +-- !query +SELECT bround(525L, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT bround(9223372036854775807L, -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "bround(9223372036854775807L, -1)" + } ] +} + + +-- !query +SELECT bround(-9223372036854775808L, -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "bround(-9223372036854775808L, -1)" + } ] +} + + +-- !query +SELECT conv('100', 2, 10) +-- !query schema +struct +-- !query output +4 + + +-- !query +SELECT conv(-10, 16, -10) +-- !query schema +struct +-- !query output +-16 + + +-- !query +SELECT conv('9223372036854775808', 10, 16) +-- !query schema +struct +-- !query output +8000000000000000 + + +-- !query +SELECT conv('92233720368547758070', 10, 16) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow in function conv()" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "conv('92233720368547758070', 10, 16)" + } ] +} + + +-- !query +SELECT conv('9223372036854775807', 36, 10) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow in function conv()" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "conv('9223372036854775807', 36, 10)" + } ] +} + + +-- !query +SELECT conv('-9223372036854775807', 36, 10) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : "", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "Overflow in function conv()" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "conv('-9223372036854775807', 36, 10)" + } ] +} + + +-- !query +SELECT BIN(0) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT BIN(25) +-- !query schema +struct +-- !query output +11001 + + +-- !query +SELECT BIN(25L) +-- !query schema +struct +-- !query output +11001 + + +-- !query +SELECT BIN(25.5) +-- !query schema +struct +-- !query output +11001 + + +-- !query +SELECT POSITIVE(0Y) +-- !query schema +struct<(+ 0):tinyint> +-- !query output +0 + + +-- !query +SELECT POSITIVE(25) +-- !query schema +struct<(+ 25):int> +-- !query output +25 + + +-- !query +SELECT POSITIVE(-25L) +-- !query schema +struct<(+ -25):bigint> +-- !query output +-25 + + +-- !query +SELECT POSITIVE(25.5) +-- !query schema +struct<(+ 25.5):decimal(3,1)> +-- !query output +25.5 + + +-- !query +SELECT POSITIVE("25.5") +-- !query schema +struct<(+ 25.5):double> +-- !query output +25.5 + + +-- !query +SELECT POSITIVE("invalid") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'invalid'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "POSITIVE(\"invalid\")" + } ] +} + + +-- !query +SELECT POSITIVE(null) +-- !query schema +struct<(+ NULL):double> +-- !query output +NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/misc-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/misc-functions.sql.out new file mode 100644 index 000000000000..22305e75914c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/misc-functions.sql.out @@ -0,0 +1,332 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query + +select typeof(null) +-- !query schema +struct +-- !query output +void + + +-- !query +select typeof(true) +-- !query schema +struct +-- !query output +boolean + + +-- !query +select typeof(1Y), typeof(1S), typeof(1), typeof(1L) +-- !query schema +struct +-- !query output +tinyint smallint int bigint + + +-- !query +select typeof(cast(1.0 as float)), typeof(1.0D), typeof(1.2) +-- !query schema +struct +-- !query output +float double decimal(2,1) + + +-- !query +select typeof(date '1986-05-23'), typeof(timestamp '1986-05-23'), typeof(interval '23 days') +-- !query schema +struct +-- !query output +date timestamp interval day + + +-- !query +select typeof(x'ABCD'), typeof('SPARK') +-- !query schema +struct +-- !query output +binary string + + +-- !query +select typeof(array(1, 2)), typeof(map(1, 2)), typeof(named_struct('a', 1, 'b', 'spark')) +-- !query schema +struct +-- !query output +array map struct + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +SELECT assert_true(true), assert_true(boolean(1)) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +SELECT assert_true(false) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "USER_RAISED_EXCEPTION", + "sqlState" : "P0001", + "messageParameters" : { + "errorMessage" : "'false' is not true!" + } +} + + +-- !query +SELECT assert_true(boolean(0)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "USER_RAISED_EXCEPTION", + "sqlState" : "P0001", + "messageParameters" : { + "errorMessage" : "'cast(0 as boolean)' is not true!" + } +} + + +-- !query +SELECT assert_true(null) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "USER_RAISED_EXCEPTION", + "sqlState" : "P0001", + "messageParameters" : { + "errorMessage" : "'null' is not true!" + } +} + + +-- !query +SELECT assert_true(boolean(null)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "USER_RAISED_EXCEPTION", + "sqlState" : "P0001", + "messageParameters" : { + "errorMessage" : "'cast(null as boolean)' is not true!" + } +} + + +-- !query +SELECT assert_true(false, 'custom error message') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "USER_RAISED_EXCEPTION", + "sqlState" : "P0001", + "messageParameters" : { + "errorMessage" : "custom error message" + } +} + + +-- !query +CREATE TEMPORARY VIEW tbl_misc AS SELECT * FROM (VALUES (1), (8), (2)) AS T(v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT raise_error('error message') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "USER_RAISED_EXCEPTION", + "sqlState" : "P0001", + "messageParameters" : { + "errorMessage" : "error message" + } +} + + +-- !query +SELECT if(v > 5, raise_error('too big: ' || v), v + 1) FROM tbl_misc +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "USER_RAISED_EXCEPTION", + "sqlState" : "P0001", + "messageParameters" : { + "errorMessage" : "too big: 8" + } +} + + +-- !query +SELECT raise_error('error message', Map()) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`raise_error`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "raise_error('error message', Map())" + } ] +} + + +-- !query +SELECT raise_error('error message', 'some args') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`raise_error`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "raise_error('error message', 'some args')" + } ] +} + + +-- !query +SELECT raise_error() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`raise_error`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "raise_error()" + } ] +} + + +-- !query +SELECT raise_error(NULL) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "USER_RAISED_EXCEPTION", + "sqlState" : "P0001", + "messageParameters" : { + "errorMessage" : "null" + } +} + + +-- !query +SELECT raise_error(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "USER_RAISED_EXCEPTION", + "sqlState" : "P0001", + "messageParameters" : { + "errorMessage" : "1" + } +} + + +-- !query +SET spark.sql.legacy.raiseErrorWithoutErrorClass=true +-- !query schema +struct +-- !query output +spark.sql.legacy.raiseErrorWithoutErrorClass true + + +-- !query +SELECT assert_true(false) +-- !query schema +struct<> +-- !query output +java.lang.RuntimeException +'false' is not true! + + +-- !query +SELECT raise_error('hello') +-- !query schema +struct<> +-- !query output +java.lang.RuntimeException +hello + + +-- !query +SET spark.sql.legacy.raiseErrorWithoutErrorClass=false +-- !query schema +struct +-- !query output +spark.sql.legacy.raiseErrorWithoutErrorClass false diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/mode.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/mode.sql.out new file mode 100644 index 000000000000..70f253066d4f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/mode.sql.out @@ -0,0 +1,489 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW basic_pays AS SELECT * FROM VALUES +('Diane Murphy','Accounting',8435), +('Mary Patterson','Accounting',9998), +('Jeff Firrelli','Accounting',8992), +('William Patterson','Accounting',8870), +('Gerard Bondur','Accounting',11472), +('Anthony Bow','Accounting',6627), +('Leslie Jennings','IT',8113), +('Leslie Thompson','IT',5186), +('Julie Firrelli','Sales',9181), +('Steve Patterson','Sales',9441), +('Foon Yue Tseng','Sales',6660), +('George Vanauf','Sales',10563), +('Loui Bondur','SCM',10449), +('Gerard Hernandez','SCM',6949), +('Pamela Castillo','SCM',11303), +('Larry Bott','SCM',11798), +('Barry Jones','SCM',10586) +AS basic_pays(employee_name, department, salary) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT mode(department), mode(salary) FROM basic_pays +-- !query schema +struct +-- !query output +Accounting 10563 + + +-- !query +SELECT department, mode(salary) FROM basic_pays GROUP BY department ORDER BY department +-- !query schema +struct +-- !query output +Accounting 6627 +IT 5186 +SCM 11303 +Sales 10563 + + +-- !query +SELECT department, mode(DISTINCT salary) FROM basic_pays GROUP BY department ORDER BY department +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.DISTINCT_UNSUPPORTED", + "sqlState" : "42K0K", + "messageParameters" : { + "funcName" : "`mode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 20, + "stopIndex" : 40, + "fragment" : "mode(DISTINCT salary)" + } ] +} + + +-- !query +SELECT mode(col) FROM VALUES (-10), (0), (10) AS tab(col) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT mode(col, false) FROM VALUES (-10), (0), (10) AS tab(col) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT mode(col, true) FROM VALUES (-10), (0), (10) AS tab(col) +-- !query schema +struct +-- !query output +-10 + + +-- !query +SELECT mode(col, 'true') FROM VALUES (-10), (0), (10) AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`mode`", + "inputSql" : "\"true\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "mode(col, 'true')" + } ] +} + + +-- !query +SELECT mode(col, null) FROM VALUES (-10), (0), (10) AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_NULL", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "`deterministic`", + "sqlExpr" : "\"NULL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "mode(col, null)" + } ] +} + + +-- !query +SELECT mode(col, b) FROM VALUES (-10, false), (0, false), (10, false) AS tab(col, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NON_FOLDABLE_ARGUMENT", + "sqlState" : "42K08", + "messageParameters" : { + "funcName" : "`mode`", + "paramName" : "`deterministic`", + "paramType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "mode(col, b)" + } ] +} + + +-- !query +SELECT mode(col) FROM VALUES (map(1, 'a')) AS tab(col) +-- !query schema +struct> +-- !query output +{1:"a"} + + +-- !query +SELECT mode(col, false) FROM VALUES (map(1, 'a')) AS tab(col) +-- !query schema +struct> +-- !query output +{1:"a"} + + +-- !query +SELECT mode(col, true) FROM VALUES (map(1, 'a')) AS tab(col) +-- !query schema +struct> +-- !query output +{1:"a"} + + +-- !query +SELECT + mode() WITHIN GROUP (ORDER BY col), + mode() WITHIN GROUP (ORDER BY col DESC) +FROM VALUES (null), (null), (null) AS tab(col) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +SELECT + mode() WITHIN GROUP (ORDER BY salary), + mode() WITHIN GROUP (ORDER BY salary DESC) +FROM basic_pays +WHERE salary > 20000 +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +SELECT + mode() WITHIN GROUP (ORDER BY salary), + mode() WITHIN GROUP (ORDER BY salary DESC) +FROM basic_pays +-- !query schema +struct +-- !query output +5186 11798 + + +-- !query +SELECT + mode() WITHIN GROUP (ORDER BY salary), + mode() WITHIN GROUP (ORDER BY salary) FILTER (WHERE salary > 10000) +FROM basic_pays +-- !query schema +struct 10000)):int> +-- !query output +5186 10449 + + +-- !query +SELECT + department, + mode() WITHIN GROUP (ORDER BY salary), + mode() WITHIN GROUP (ORDER BY salary) FILTER (WHERE salary > 10000) +FROM basic_pays +GROUP BY department +ORDER BY department +-- !query schema +struct 10000)):int> +-- !query output +Accounting 6627 11472 +IT 5186 NULL +SCM 6949 10449 +Sales 6660 10563 + + +-- !query +SELECT + employee_name, + department, + salary, + mode() WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department) +FROM basic_pays +ORDER BY salary +-- !query schema +struct +-- !query output +Leslie Thompson IT 5186 5186 +Anthony Bow Accounting 6627 6627 +Foon Yue Tseng Sales 6660 6660 +Gerard Hernandez SCM 6949 6949 +Leslie Jennings IT 8113 5186 +Diane Murphy Accounting 8435 6627 +William Patterson Accounting 8870 6627 +Jeff Firrelli Accounting 8992 6627 +Julie Firrelli Sales 9181 6660 +Steve Patterson Sales 9441 6660 +Mary Patterson Accounting 9998 6627 +Loui Bondur SCM 10449 6949 +George Vanauf Sales 10563 6660 +Barry Jones SCM 10586 6949 +Pamela Castillo SCM 11303 6949 +Gerard Bondur Accounting 11472 6627 +Larry Bott SCM 11798 6949 + + +-- !query +SELECT + employee_name, + department, + salary, + mode() WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary) +FROM basic_pays +ORDER BY salary +-- !query schema +struct +-- !query output +Leslie Thompson IT 5186 5186 +Anthony Bow Accounting 6627 6627 +Foon Yue Tseng Sales 6660 6660 +Gerard Hernandez SCM 6949 6949 +Leslie Jennings IT 8113 5186 +Diane Murphy Accounting 8435 6627 +William Patterson Accounting 8870 6627 +Jeff Firrelli Accounting 8992 6627 +Julie Firrelli Sales 9181 6660 +Steve Patterson Sales 9441 6660 +Mary Patterson Accounting 9998 6627 +Loui Bondur SCM 10449 6949 +George Vanauf Sales 10563 6660 +Barry Jones SCM 10586 6949 +Pamela Castillo SCM 11303 6949 +Gerard Bondur Accounting 11472 6627 +Larry Bott SCM 11798 6949 + + +-- !query +SELECT + employee_name, + department, + salary, + mode() WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +FROM basic_pays +ORDER BY salary +-- !query schema +struct +-- !query output +Leslie Thompson IT 5186 5186 +Anthony Bow Accounting 6627 6627 +Foon Yue Tseng Sales 6660 6660 +Gerard Hernandez SCM 6949 6949 +Leslie Jennings IT 8113 5186 +Diane Murphy Accounting 8435 8435 +William Patterson Accounting 8870 8435 +Jeff Firrelli Accounting 8992 8435 +Julie Firrelli Sales 9181 9181 +Steve Patterson Sales 9441 6660 +Mary Patterson Accounting 9998 8435 +Loui Bondur SCM 10449 6949 +George Vanauf Sales 10563 6660 +Barry Jones SCM 10586 6949 +Pamela Castillo SCM 11303 6949 +Gerard Bondur Accounting 11472 6627 +Larry Bott SCM 11798 6949 + + +-- !query +SELECT + employee_name, + department, + salary, + mode() WITHIN GROUP (ORDER BY salary) OVER w +FROM basic_pays +WHERE salary > 8900 +WINDOW w AS (PARTITION BY department) +ORDER BY salary +-- !query schema +struct +-- !query output +Jeff Firrelli Accounting 8992 8992 +Julie Firrelli Sales 9181 9181 +Steve Patterson Sales 9441 9181 +Mary Patterson Accounting 9998 8992 +Loui Bondur SCM 10449 10449 +George Vanauf Sales 10563 9181 +Barry Jones SCM 10586 10449 +Pamela Castillo SCM 11303 10449 +Gerard Bondur Accounting 11472 8992 +Larry Bott SCM 11798 10449 + + +-- !query +SELECT + mode(DISTINCT salary) WITHIN GROUP (ORDER BY salary) +FROM basic_pays +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.DISTINCT_UNSUPPORTED", + "sqlState" : "42K0K", + "messageParameters" : { + "funcName" : "`mode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 61, + "fragment" : "mode(DISTINCT salary) WITHIN GROUP (ORDER BY salary)" + } ] +} + + +-- !query +SELECT + mode() +FROM basic_pays +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.WITHIN_GROUP_MISSING", + "sqlState" : "42K0K", + "messageParameters" : { + "funcName" : "`mode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 15, + "fragment" : "mode()" + } ] +} + + +-- !query +SELECT + mode(salary) WITHIN GROUP (ORDER BY salary) +FROM basic_pays +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.WRONG_NUM_ORDERINGS", + "sqlState" : "42K0K", + "messageParameters" : { + "actualNum" : "1", + "expectedNum" : "0", + "funcName" : "`mode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 52, + "fragment" : "mode(salary) WITHIN GROUP (ORDER BY salary)" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW intervals AS SELECT * FROM VALUES +(0, INTERVAL '0' MONTH, INTERVAL '0' SECOND, INTERVAL '0' MINUTE), +(0, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(0, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(0, INTERVAL '30' MONTH, INTERVAL '30' SECOND, INTERVAL '30' MINUTE), +(0, INTERVAL '40' MONTH, INTERVAL '40' SECOND, INTERVAL '40' MINUTE), +(1, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(1, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(2, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(2, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(2, INTERVAL '25' MONTH, INTERVAL '25' SECOND, INTERVAL '25' MINUTE), +(2, INTERVAL '30' MONTH, INTERVAL '30' SECOND, INTERVAL '30' MINUTE), +(3, INTERVAL '60' MONTH, INTERVAL '60' SECOND, INTERVAL '60' MINUTE), +(4, null, null, null) +AS intervals(k, dt, ym, dt2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + mode() WITHIN GROUP (ORDER BY dt), + mode() WITHIN GROUP (ORDER BY dt DESC) +FROM intervals +-- !query schema +struct +-- !query output +0-10 1-8 + + +-- !query +SELECT + k, + mode() WITHIN GROUP (ORDER BY ym), + mode() WITHIN GROUP (ORDER BY dt DESC) +FROM intervals +GROUP BY k +ORDER BY k +-- !query schema +struct +-- !query output +0 0 00:00:00.000000000 3-4 +1 0 00:00:10.000000000 1-8 +2 0 00:00:10.000000000 2-6 +3 0 00:01:00.000000000 5-0 +4 NULL NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/named-function-arguments.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/named-function-arguments.sql.out new file mode 100644 index 000000000000..e5063dc0cf31 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/named-function-arguments.sql.out @@ -0,0 +1,464 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd') +-- !query schema +struct +-- !query output +QqQQdddoooo + + +-- !query +SELECT mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbCD123-@$#') +-- !query schema +struct +-- !query output +QqQQdddoooo + + +-- !query +SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', digitChar => 'd') +-- !query schema +struct +-- !query output +QqQQddd-@$# + + +-- !query +SELECT mask(lowerChar => 'q', upperChar => 'Q', digitChar => 'd', str => 'AbCD123-@$#') +-- !query schema +struct +-- !query output +QqQQddd-@$# + + +-- !query +create temporary view t2 as select * from values + ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT hex(count_min_sketch(t2d, seed => 1, epsilon => 0.5d, confidence => 0.5d)) FROM t2 +-- !query schema +struct +-- !query output +00000001000000000000000D0000000100000004000000005D8D6AB90000000000000002000000000000000700000000000000010000000000000003 + + +-- !query +SELECT hex(count_min_sketch(seed => 1, epsilon => 0.5d, confidence => 0.5d, column => t2d)) FROM t2 +-- !query schema +struct +-- !query output +00000001000000000000000D0000000100000004000000005D8D6AB90000000000000002000000000000000700000000000000010000000000000003 + + +-- !query +SELECT hex(count_min_sketch(t2d, 0.5d, seed => 1, confidence => 0.5d)) FROM t2 +-- !query schema +struct +-- !query output +00000001000000000000000D0000000100000004000000005D8D6AB90000000000000002000000000000000700000000000000010000000000000003 + + +-- !query +SELECT * FROM explode(collection => array(1, 2)) +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT * FROM explode_outer(collection => map('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +a 1 +b 2 + + +-- !query +SELECT * FROM explode(array(1, 2)), explode(array(3, 4)) +-- !query schema +struct +-- !query output +1 3 +1 4 +2 3 +2 4 + + +-- !query +SELECT * FROM explode(array(1, 2)) AS t, LATERAL explode(array(3 * t.col, 4 * t.col)) +-- !query schema +struct +-- !query output +1 3 +1 4 +2 6 +2 8 + + +-- !query +SELECT num, val, 'Spark' FROM explode(map(1, 'a', 2, 'b')) AS t(num, val) +-- !query schema +struct +-- !query output +1 a Spark +2 b Spark + + +-- !query +SELECT * FROM posexplode(collection => array(1, 2)) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +SELECT * FROM posexplode_outer(collection => map('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +0 a 1 +1 b 2 + + +-- !query +SELECT * FROM posexplode(array(1, 2)), posexplode(array(3, 4)) +-- !query schema +struct +-- !query output +0 1 0 3 +0 1 1 4 +1 2 0 3 +1 2 1 4 + + +-- !query +SELECT * FROM posexplode(array(1, 2)) AS t, LATERAL posexplode(array(3 * t.col, 4 * t.col)) +-- !query schema +struct +-- !query output +0 1 0 3 +0 1 1 4 +1 2 0 6 +1 2 1 8 + + +-- !query +SELECT pos, num, val, 'Spark' FROM posexplode(map(1, 'a', 2, 'b')) AS t(pos, num, val) +-- !query schema +struct +-- !query output +0 1 a Spark +1 2 b Spark + + +-- !query +SELECT * FROM inline(input => array(struct(1, 'a'), struct(2, 'b'))) +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +SELECT * FROM inline_outer(input => array(struct(1, 'a'), struct(2, 'b'))) +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +SELECT * FROM inline(array(struct(1, 'a'), struct(2, 'b'))), inline(array(struct(3, 'c'), struct(4, 'd'))) +-- !query schema +struct +-- !query output +1 a 3 c +1 a 4 d +2 b 3 c +2 b 4 d + + +-- !query +SELECT * FROM inline(array(struct(1, 'a'), struct(2, 'b'))) AS t, LATERAL inline(array(struct(3 * t.col1, 4 * t.col1))) +-- !query schema +struct +-- !query output +1 a 3 4 +2 b 6 8 + + +-- !query +SELECT num, val, 'Spark' FROM inline(array(struct(1, 'a'), struct(2, 'b'))) AS t(num, val) +-- !query schema +struct +-- !query output +1 a Spark +2 b Spark + + +-- !query +SELECT * FROM explode(collection => explode(array(1))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + "sqlState" : "42K0E", + "messageParameters" : { + "expression" : "\"explode(explode(array(1)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 54, + "fragment" : "explode(collection => explode(array(1)))" + } ] +} + + +-- !query +SELECT * FROM explode(collection => explode(collection => array(1))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + "sqlState" : "42K0E", + "messageParameters" : { + "expression" : "\"explode(explode(array(1)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 68, + "fragment" : "explode(collection => explode(collection => array(1)))" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v AS SELECT id FROM range(0, 8) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM explode(collection => TABLE(v)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"functiontablesubqueryargumentexpression()\"", + "inputType" : "\"STRUCT\"", + "paramIndex" : "first", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"explode(functiontablesubqueryargumentexpression())\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 45, + "fragment" : "explode(collection => TABLE(v))" + } ] +} + + +-- !query +SELECT mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNEXPECTED_POSITIONAL_ARGUMENT", + "sqlState" : "4274K", + "messageParameters" : { + "parameterName" : "`lowerChar`", + "routineName" : "`mask`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 98, + "fragment" : "mask(lowerChar => 'q', 'AbCD123-@$#', upperChar => 'Q', otherChar => 'o', digitChar => 'd')" + } ] +} + + +-- !query +SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', digitChar => 'e') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.DOUBLE_NAMED_ARGUMENT_REFERENCE", + "sqlState" : "4274K", + "messageParameters" : { + "parameterName" : "`digitChar`", + "routineName" : "`mask`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 116, + "fragment" : "mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', digitChar => 'e')" + } ] +} + + +-- !query +SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbC') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.BOTH_POSITIONAL_AND_NAMED", + "sqlState" : "4274K", + "messageParameters" : { + "parameterName" : "`str`", + "routineName" : "`mask`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 112, + "fragment" : "mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', str => 'AbC')" + } ] +} + + +-- !query +SELECT mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "REQUIRED_PARAMETER_NOT_FOUND", + "sqlState" : "4274K", + "messageParameters" : { + "index" : "0", + "parameterName" : "`str`", + "routineName" : "`mask`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 83, + "fragment" : "mask(lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd')" + } ] +} + + +-- !query +SELECT mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', cellular => 'automata') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRECOGNIZED_PARAMETER_NAME", + "sqlState" : "4274K", + "messageParameters" : { + "argumentName" : "`cellular`", + "proposal" : "`str` `upperChar` `otherChar`", + "routineName" : "`mask`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 122, + "fragment" : "mask('AbCD123-@$#', lowerChar => 'q', upperChar => 'Q', otherChar => 'o', digitChar => 'd', cellular => 'automata')" + } ] +} + + +-- !query +SELECT encode(str => 'a', charset => 'utf-8') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NAMED_PARAMETERS_NOT_SUPPORTED", + "sqlState" : "4274K", + "messageParameters" : { + "functionName" : "`encode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "encode(str => 'a', charset => 'utf-8')" + } ] +} + + +-- !query +SELECT mask('AbCD123-@$#', 'Q', 'q', 'd', 'o', 'k') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "6", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[1, 2, 3, 4, 5]", + "functionName" : "`mask`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "mask('AbCD123-@$#', 'Q', 'q', 'd', 'o', 'k')" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/natural-join.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/natural-join.sql.out new file mode 100644 index 000000000000..38791e6741fc --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/natural-join.sql.out @@ -0,0 +1,342 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view nt3 as select * from values + ("one", 4), + ("two", 5), + ("one", 6) + as nt3(k, v3) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view nt4 as select * from values + ("one", 7), + ("two", 8), + ("one", 9) + as nt4(k, v4) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM nt1 natural join nt2 +-- !query schema +struct +-- !query output +one 1 1 +one 1 5 +two 2 22 + + +-- !query +SELECT * FROM nt1 natural join nt2 where k = "one" +-- !query schema +struct +-- !query output +one 1 1 +one 1 5 + + +-- !query +SELECT * FROM nt1 natural left join nt2 order by v1, v2 +-- !query schema +struct +-- !query output +one 1 1 +one 1 5 +two 2 22 +three 3 NULL + + +-- !query +SELECT * FROM nt1 natural right join nt2 order by v1, v2 +-- !query schema +struct +-- !query output +one 1 1 +one 1 5 +two 2 22 + + +-- !query +SELECT count(*) FROM nt1 natural full outer join nt2 +-- !query schema +struct +-- !query output +4 + + +-- !query +SELECT k FROM nt1 natural join nt2 +-- !query schema +struct +-- !query output +one +one +two + + +-- !query +SELECT k FROM nt1 natural join nt2 where k = "one" +-- !query schema +struct +-- !query output +one +one + + +-- !query +SELECT nt1.* FROM nt1 natural join nt2 +-- !query schema +struct +-- !query output +one 1 +one 1 +two 2 + + +-- !query +SELECT nt2.* FROM nt1 natural join nt2 +-- !query schema +struct +-- !query output +one 1 +one 5 +two 22 + + +-- !query +SELECT sbq.* from (SELECT * FROM nt1 natural join nt2) sbq +-- !query schema +struct +-- !query output +one 1 1 +one 1 5 +two 2 22 + + +-- !query +SELECT sbq.k from (SELECT * FROM nt1 natural join nt2) sbq +-- !query schema +struct +-- !query output +one +one +two + + +-- !query +SELECT nt1.*, nt2.* FROM nt1 natural join nt2 +-- !query schema +struct +-- !query output +one 1 one 1 +one 1 one 5 +two 2 two 22 + + +-- !query +SELECT *, nt2.k FROM nt1 natural join nt2 +-- !query schema +struct +-- !query output +one 1 1 one +one 1 5 one +two 2 22 two + + +-- !query +SELECT nt1.k, nt2.k FROM nt1 natural join nt2 +-- !query schema +struct +-- !query output +one one +one one +two two + + +-- !query +SELECT k FROM (SELECT nt2.k FROM nt1 natural join nt2) +-- !query schema +struct +-- !query output +one +one +two + + +-- !query +SELECT nt2.k AS key FROM nt1 natural join nt2 ORDER BY key +-- !query schema +struct +-- !query output +one +one +two + + +-- !query +SELECT nt1.k, nt2.k FROM nt1 natural join nt2 where k = "one" +-- !query schema +struct +-- !query output +one one +one one + + +-- !query +SELECT * FROM (SELECT * FROM nt1 natural join nt2) +-- !query schema +struct +-- !query output +one 1 1 +one 1 5 +two 2 22 + + +-- !query +SELECT * FROM (SELECT nt1.*, nt2.* FROM nt1 natural join nt2) +-- !query schema +struct +-- !query output +one 1 one 1 +one 1 one 5 +two 2 two 22 + + +-- !query +SELECT * FROM (SELECT nt1.v1, nt2.k FROM nt1 natural join nt2) +-- !query schema +struct +-- !query output +1 one +1 one +2 two + + +-- !query +SELECT nt2.k FROM (SELECT * FROM nt1 natural join nt2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`nt2`.`k`", + "proposal" : "`k`, `v1`, `v2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "nt2.k" + } ] +} + + +-- !query +SELECT * FROM nt1 natural join nt2 natural join nt3 +-- !query schema +struct +-- !query output +one 1 1 4 +one 1 1 6 +one 1 5 4 +one 1 5 6 +two 2 22 5 + + +-- !query +SELECT nt1.*, nt2.*, nt3.* FROM nt1 natural join nt2 natural join nt3 +-- !query schema +struct +-- !query output +one 1 one 1 one 4 +one 1 one 1 one 6 +one 1 one 5 one 4 +one 1 one 5 one 6 +two 2 two 22 two 5 + + +-- !query +SELECT nt1.*, nt2.*, nt3.* FROM nt1 natural join nt2 join nt3 on nt2.k = nt3.k +-- !query schema +struct +-- !query output +one 1 one 1 one 4 +one 1 one 1 one 6 +one 1 one 5 one 4 +one 1 one 5 one 6 +two 2 two 22 two 5 + + +-- !query +SELECT * FROM nt1 natural join nt2 join nt3 on nt1.k = nt3.k +-- !query schema +struct +-- !query output +one 1 1 one 4 +one 1 1 one 6 +one 1 5 one 4 +one 1 5 one 6 +two 2 22 two 5 + + +-- !query +SELECT * FROM nt1 natural join nt2 join nt3 on nt2.k = nt3.k +-- !query schema +struct +-- !query output +one 1 1 one 4 +one 1 1 one 6 +one 1 5 one 4 +one 1 5 one 6 +two 2 22 two 5 + + +-- !query +SELECT nt1.*, nt2.*, nt3.*, nt4.* FROM nt1 natural join nt2 natural join nt3 natural join nt4 +-- !query schema +struct +-- !query output +one 1 one 1 one 4 one 7 +one 1 one 1 one 4 one 9 +one 1 one 1 one 6 one 7 +one 1 one 1 one 6 one 9 +one 1 one 5 one 4 one 7 +one 1 one 5 one 4 one 9 +one 1 one 5 one 6 one 7 +one 1 one 5 one 6 one 9 +two 2 two 22 two 5 two 8 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/non-excludable-rule.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/non-excludable-rule.sql.out new file mode 100644 index 000000000000..6a683995cc33 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/non-excludable-rule.sql.out @@ -0,0 +1,55 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SET spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ReplaceCTERefWithRepartition +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ReplaceCTERefWithRepartition + + +-- !query +SELECT + (SELECT min(id) FROM range(10)), + (SELECT sum(id) FROM range(10)), + (SELECT count(distinct id) FROM range(10)) +-- !query schema +struct +-- !query output +0 45 10 + + +-- !query +SET spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.Optimizer$OptimizeSubqueries +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.Optimizer$OptimizeSubqueries + + +-- !query +WITH tmp AS ( + SELECT id FROM range(2) + INTERSECT + SELECT id FROM range(4) +) +SELECT id FROM range(3) WHERE id > (SELECT max(id) FROM tmp) +-- !query schema +struct +-- !query output +2 + + +-- !query +SET spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.RewriteLateralSubquery +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.RewriteLateralSubquery + + +-- !query +SELECT * FROM testData, LATERAL (SELECT * FROM testData) LIMIT 1 +-- !query schema +struct +-- !query output +1 1 1 1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/array.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/array.sql.out new file mode 100644 index 000000000000..c1330c620acf --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/array.sql.out @@ -0,0 +1,788 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view data as select * from values + ("one", array(11, 12, 13), array(array(111, 112, 113), array(121, 122, 123))), + ("two", array(21, 22, 23), array(array(211, 212, 213), array(221, 222, 223))) + as data(a, b, c) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from data +-- !query schema +struct,c:array>> +-- !query output +one [11,12,13] [[111,112,113],[121,122,123]] +two [21,22,23] [[211,212,213],[221,222,223]] + + +-- !query +select a, b[0], b[0] + b[1] from data +-- !query schema +struct +-- !query output +one 11 23 +two 21 43 + + +-- !query +select a, c[0][0] + c[0][0 + 1] from data +-- !query schema +struct +-- !query output +one 223 +two 423 + + +-- !query +create temporary view primitive_arrays as select * from values ( + array(true), + array(2Y, 1Y), + array(2S, 1S), + array(2, 1), + array(2L, 1L), + array(9223372036854775809, 9223372036854775808), + array(2.0D, 1.0D), + array(float(2.0), float(1.0)), + array(date '2016-03-14', date '2016-03-13'), + array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000') +) as primitive_arrays( + boolean_array, + tinyint_array, + smallint_array, + int_array, + bigint_array, + decimal_array, + double_array, + float_array, + date_array, + timestamp_array +) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from primitive_arrays +-- !query schema +struct,tinyint_array:array,smallint_array:array,int_array:array,bigint_array:array,decimal_array:array,double_array:array,float_array:array,date_array:array,timestamp_array:array> +-- !query output +[true] [2,1] [2,1] [2,1] [2,1] [9223372036854775809,9223372036854775808] [2.0,1.0] [2.0,1.0] [2016-03-14,2016-03-13] [2016-11-15 20:54:00,2016-11-12 20:54:00] + + +-- !query +select + array_contains(boolean_array, true), array_contains(boolean_array, false), + array_contains(tinyint_array, 2Y), array_contains(tinyint_array, 0Y), + array_contains(smallint_array, 2S), array_contains(smallint_array, 0S), + array_contains(int_array, 2), array_contains(int_array, 0), + array_contains(bigint_array, 2L), array_contains(bigint_array, 0L), + array_contains(decimal_array, 9223372036854775809), array_contains(decimal_array, 1), + array_contains(double_array, 2.0D), array_contains(double_array, 0.0D), + array_contains(float_array, float(2.0)), array_contains(float_array, float(0.0)), + array_contains(date_array, date '2016-03-14'), array_contains(date_array, date '2016-01-01'), + array_contains(timestamp_array, timestamp '2016-11-15 20:54:00.000'), array_contains(timestamp_array, timestamp '2016-01-01 20:54:00.000') +from primitive_arrays +-- !query schema +struct +-- !query output +true false true false true false true false true false true false true false true false true false true false + + +-- !query +select array_contains(b, 11), array_contains(c, array(111, 112, 113)) from data +-- !query schema +struct +-- !query output +false false +true true + + +-- !query +select + sort_array(boolean_array), + sort_array(tinyint_array), + sort_array(smallint_array), + sort_array(int_array), + sort_array(bigint_array), + sort_array(decimal_array), + sort_array(double_array), + sort_array(float_array), + sort_array(date_array), + sort_array(timestamp_array) +from primitive_arrays +-- !query schema +struct,sort_array(tinyint_array, true):array,sort_array(smallint_array, true):array,sort_array(int_array, true):array,sort_array(bigint_array, true):array,sort_array(decimal_array, true):array,sort_array(double_array, true):array,sort_array(float_array, true):array,sort_array(date_array, true):array,sort_array(timestamp_array, true):array> +-- !query output +[true] [1,2] [1,2] [1,2] [1,2] [9223372036854775808,9223372036854775809] [1.0,2.0] [1.0,2.0] [2016-03-13,2016-03-14] [2016-11-12 20:54:00,2016-11-15 20:54:00] + + +-- !query +select sort_array(array('b', 'd'), '1') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"sort_array(array(b, d), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "sort_array(array('b', 'd'), '1')" + } ] +} + + +-- !query +select sort_array(array('b', 'd'), cast(NULL as boolean)) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select + size(boolean_array), + size(tinyint_array), + size(smallint_array), + size(int_array), + size(bigint_array), + size(decimal_array), + size(double_array), + size(float_array), + size(date_array), + size(timestamp_array) +from primitive_arrays +-- !query schema +struct +-- !query output +1 2 2 2 2 2 2 2 2 2 + + +-- !query +select element_at(array(1, 2, 3), 5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select element_at(array(1, 2, 3), -5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select element_at(array(1, 2, 3), 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_INDEX_OF_ZERO", + "sqlState" : "22003" +} + + +-- !query +select elt(4, '123', '456') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select elt(0, '123', '456') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select elt(-1, '123', '456') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select elt(null, '123', '456') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select elt(null, '123', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select elt(1, '123', null) +-- !query schema +struct +-- !query output +123 + + +-- !query +select elt(2, '123', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select array(1, 2, 3)[5] +-- !query schema +struct +-- !query output +NULL + + +-- !query +select array(1, 2, 3)[-1] +-- !query schema +struct +-- !query output +NULL + + +-- !query +select array_size(array()) +-- !query schema +struct +-- !query output +0 + + +-- !query +select array_size(array(true)) +-- !query schema +struct +-- !query output +1 + + +-- !query +select array_size(array(2, 1)) +-- !query schema +struct +-- !query output +2 + + +-- !query +select array_size(NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select array_size(map('a', 1, 'b', 2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"map(a, 1, b, 2)\"", + "inputType" : "\"MAP\"", + "paramIndex" : "first", + "requiredType" : "\"ARRAY\"", + "sqlExpr" : "\"array_size(map(a, 1, b, 2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "array_size(map('a', 1, 'b', 2))" + } ] +} + + +-- !query +select size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10))) +-- !query schema +struct +-- !query output +4 + + +-- !query +select size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10))) +-- !query schema +struct +-- !query output +4 + + +-- !query +select size(arrays_zip(array(1, 2, 3), array(4), null, array(7, 8, 9, 10))) +-- !query schema +struct +-- !query output +-1 + + +-- !query +select isnotnull(arrays_zip(array(), array(4), array(7, 8, 9, 10))) +-- !query schema +struct<(arrays_zip(array(), array(4), array(7, 8, 9, 10)) IS NOT NULL):boolean> +-- !query output +true + + +-- !query +select isnotnull(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10))) +-- !query schema +struct<(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10)) IS NOT NULL):boolean> +-- !query output +true + + +-- !query +select isnotnull(arrays_zip(array(1, 2, 3), NULL, array(4), array(7, 8, 9, 10))) +-- !query schema +struct<(arrays_zip(array(1, 2, 3), NULL, array(4), array(7, 8, 9, 10)) IS NOT NULL):boolean> +-- !query output +false + + +-- !query +select get(array(1, 2, 3), 0) +-- !query schema +struct +-- !query output +1 + + +-- !query +select get(array(1, 2, 3), 3) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select get(array(1, 2, 3), null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select get(array(1, 2, 3), -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select array_insert(array(1, 2, 3), 3, 4) +-- !query schema +struct> +-- !query output +[1,2,4,3] + + +-- !query +select array_insert(array(2, 3, 4), 0, 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_INDEX_OF_ZERO", + "sqlState" : "22003", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "array_insert(array(2, 3, 4), 0, 1)" + } ] +} + + +-- !query +select array_insert(array(2, 3, 4), 1, 1) +-- !query schema +struct> +-- !query output +[1,2,3,4] + + +-- !query +select array_insert(array(1, 3, 4), -2, 2) +-- !query schema +struct> +-- !query output +[1,3,2,4] + + +-- !query +select array_insert(array(1, 2, 3), 3, "4") +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "\"ARRAY\"", + "functionName" : "`array_insert`", + "leftType" : "\"ARRAY\"", + "rightType" : "\"STRING\"", + "sqlExpr" : "\"array_insert(array(1, 2, 3), 3, 4)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "array_insert(array(1, 2, 3), 3, \"4\")" + } ] +} + + +-- !query +select array_insert(cast(NULL as ARRAY), 1, 1) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select array_insert(array(1, 2, 3, NULL), cast(NULL as INT), 4) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select array_insert(array(1, 2, 3, NULL), 4, cast(NULL as INT)) +-- !query schema +struct> +-- !query output +[1,2,3,null,null] + + +-- !query +select array_insert(array(2, 3, NULL, 4), 5, 5) +-- !query schema +struct> +-- !query output +[2,3,null,4,5] + + +-- !query +select array_insert(array(2, 3, NULL, 4), -5, 1) +-- !query schema +struct> +-- !query output +[1,2,3,null,4] + + +-- !query +select array_insert(array(1), 2, cast(2 as tinyint)) +-- !query schema +struct> +-- !query output +[1,2] + + +-- !query +set spark.sql.legacy.negativeIndexInArrayInsert=true +-- !query schema +struct +-- !query output +spark.sql.legacy.negativeIndexInArrayInsert true + + +-- !query +select array_insert(array(1, 3, 4), -2, 2) +-- !query schema +struct> +-- !query output +[1,2,3,4] + + +-- !query +select array_insert(array(2, 3, NULL, 4), -5, 1) +-- !query schema +struct> +-- !query output +[1,null,2,3,null,4] + + +-- !query +set spark.sql.legacy.negativeIndexInArrayInsert=false +-- !query schema +struct +-- !query output +spark.sql.legacy.negativeIndexInArrayInsert false + + +-- !query +select array_compact(id) from values (1) as t(id) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"id\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "\"ARRAY\"", + "sqlExpr" : "\"array_compact(id)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "array_compact(id)" + } ] +} + + +-- !query +select array_compact(array("1", null, "2", null)) +-- !query schema +struct> +-- !query output +["1","2"] + + +-- !query +select array_compact(array("a", "b", "c")) +-- !query schema +struct> +-- !query output +["a","b","c"] + + +-- !query +select array_compact(array(1D, null, 2D, null)) +-- !query schema +struct> +-- !query output +[1.0,2.0] + + +-- !query +select array_compact(array(array(1, 2, 3, null), null, array(4, null, 6))) +-- !query schema +struct>> +-- !query output +[[1,2,3,null],[4,null,6]] + + +-- !query +select array_compact(array(null)) +-- !query schema +struct> +-- !query output +[] + + +-- !query +select array_append(array(1, 2, 3), 4) +-- !query schema +struct> +-- !query output +[1,2,3,4] + + +-- !query +select array_append(array('a', 'b', 'c'), 'd') +-- !query schema +struct> +-- !query output +["a","b","c","d"] + + +-- !query +select array_append(array(1, 2, 3, NULL), NULL) +-- !query schema +struct> +-- !query output +[1,2,3,null,null] + + +-- !query +select array_append(array('a', 'b', 'c', NULL), NULL) +-- !query schema +struct> +-- !query output +["a","b","c",null,null] + + +-- !query +select array_append(CAST(null AS ARRAY), 'a') +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select array_append(CAST(null AS ARRAY), CAST(null as String)) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select array_append(array(), 1) +-- !query schema +struct> +-- !query output +[1] + + +-- !query +select array_append(CAST(array() AS ARRAY), CAST(NULL AS String)) +-- !query schema +struct> +-- !query output +[null] + + +-- !query +select array_append(array(CAST(NULL AS String)), CAST(NULL AS String)) +-- !query schema +struct> +-- !query output +[null,null] + + +-- !query +select array_prepend(array(1, 2, 3), 4) +-- !query schema +struct> +-- !query output +[4,1,2,3] + + +-- !query +select array_prepend(array('a', 'b', 'c'), 'd') +-- !query schema +struct> +-- !query output +["d","a","b","c"] + + +-- !query +select array_prepend(array(1, 2, 3, NULL), NULL) +-- !query schema +struct> +-- !query output +[null,1,2,3,null] + + +-- !query +select array_prepend(array('a', 'b', 'c', NULL), NULL) +-- !query schema +struct> +-- !query output +[null,"a","b","c",null] + + +-- !query +select array_prepend(CAST(null AS ARRAY), 'a') +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select array_prepend(CAST(null AS ARRAY), CAST(null as String)) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select array_prepend(array(), 1) +-- !query schema +struct> +-- !query output +[1] + + +-- !query +select array_prepend(CAST(array() AS ARRAY), CAST(NULL AS String)) +-- !query schema +struct> +-- !query output +[null] + + +-- !query +select array_prepend(array(CAST(NULL AS String)), CAST(NULL AS String)) +-- !query schema +struct> +-- !query output +[null,null] + + +-- !query +select array_union(array(0.0, -0.0, DOUBLE("NaN")), array(0.0, -0.0, DOUBLE("NaN"))) +-- !query schema +struct> +-- !query output +[0.0,NaN] + + +-- !query +select array_distinct(array(0.0, -0.0, -0.0, DOUBLE("NaN"), DOUBLE("NaN"))) +-- !query schema +struct> +-- !query output +[0.0,NaN] diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/cast.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/cast.sql.out new file mode 100644 index 000000000000..6f74c63da354 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/cast.sql.out @@ -0,0 +1,1161 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT CAST('1.23' AS int) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CAST('1.23' AS long) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CAST('-4.56' AS int) +-- !query schema +struct +-- !query output +-4 + + +-- !query +SELECT CAST('-4.56' AS long) +-- !query schema +struct +-- !query output +-4 + + +-- !query +SELECT CAST('abc' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('abc' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('abc' AS float) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('abc' AS double) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('1234567890123' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('12345678901234567890123' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('' AS float) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('' AS double) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST(NULL AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST(NULL AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('123.a' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('123.a' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('123.a' AS float) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('123.a' AS double) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('-2147483648' AS int) +-- !query schema +struct +-- !query output +-2147483648 + + +-- !query +SELECT CAST('-2147483649' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('2147483647' AS int) +-- !query schema +struct +-- !query output +2147483647 + + +-- !query +SELECT CAST('2147483648' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('-9223372036854775808' AS long) +-- !query schema +struct +-- !query output +-9223372036854775808 + + +-- !query +SELECT CAST('-9223372036854775809' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CAST('9223372036854775807' AS long) +-- !query schema +struct +-- !query output +9223372036854775807 + + +-- !query +SELECT CAST('9223372036854775808' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT HEX(CAST('abc' AS binary)) +-- !query schema +struct +-- !query output +616263 + + +-- !query +SELECT HEX(CAST(CAST(123 AS byte) AS binary)) +-- !query schema +struct +-- !query output +7B + + +-- !query +SELECT HEX(CAST(CAST(-123 AS byte) AS binary)) +-- !query schema +struct +-- !query output +85 + + +-- !query +SELECT HEX(CAST(123S AS binary)) +-- !query schema +struct +-- !query output +007B + + +-- !query +SELECT HEX(CAST(-123S AS binary)) +-- !query schema +struct +-- !query output +FF85 + + +-- !query +SELECT HEX(CAST(123 AS binary)) +-- !query schema +struct +-- !query output +0000007B + + +-- !query +SELECT HEX(CAST(-123 AS binary)) +-- !query schema +struct +-- !query output +FFFFFF85 + + +-- !query +SELECT HEX(CAST(123L AS binary)) +-- !query schema +struct +-- !query output +000000000000007B + + +-- !query +SELECT HEX(CAST(-123L AS binary)) +-- !query schema +struct +-- !query output +FFFFFFFFFFFFFF85 + + +-- !query +DESC FUNCTION boolean +-- !query schema +struct +-- !query output +Class: org.apache.spark.sql.catalyst.expressions.Cast +Function: boolean +Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`. + + +-- !query +DESC FUNCTION EXTENDED boolean +-- !query schema +struct +-- !query output +Class: org.apache.spark.sql.catalyst.expressions.Cast +Extended Usage: + No example/argument for boolean. + + Since: 2.0.1 + +Function: boolean +Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`. + + +-- !query +SELECT CAST('interval 3 month 1 hour' AS interval) +-- !query schema +struct +-- !query output +3 months 1 hours + + +-- !query +SELECT CAST("interval '3-1' year to month" AS interval year to month) +-- !query schema +struct +-- !query output +3-1 + + +-- !query +SELECT CAST("interval '3 00:00:01' day to second" AS interval day to second) +-- !query schema +struct +-- !query output +3 00:00:01.000000000 + + +-- !query +SELECT CAST(interval 3 month 1 hour AS string) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 3 month 1 hour" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 35, + "fragment" : "interval 3 month 1 hour" + } ] +} + + +-- !query +SELECT CAST(interval 3 year 1 month AS string) +-- !query schema +struct +-- !query output +INTERVAL '3-1' YEAR TO MONTH + + +-- !query +SELECT CAST(interval 3 day 1 second AS string) +-- !query schema +struct +-- !query output +INTERVAL '3 00:00:01' DAY TO SECOND + + +-- !query +select cast(' 1' as tinyint) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast(' 1\t' as tinyint) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast(' 1' as smallint) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast(' 1' as INT) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast(' 1' as bigint) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast(' 1' as float) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +select cast(' 1 ' as DOUBLE) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +select cast('1.0 ' as DEC) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast('1中文' as tinyint) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('1中文' as smallint) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('1中文' as INT) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('中文1' as bigint) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('1中文' as bigint) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('\t\t true \n\r ' as boolean) +-- !query schema +struct +-- !query output +true + + +-- !query +select cast('\t\n false \t\r' as boolean) +-- !query schema +struct +-- !query output +false + + +-- !query +select cast('\t\n xyz \t\r' as boolean) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('23.45' as decimal(4, 2)) +-- !query schema +struct +-- !query output +23.45 + + +-- !query +select cast('123.45' as decimal(4, 2)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('xyz' as decimal(4, 2)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('2022-01-01' as date) +-- !query schema +struct +-- !query output +2022-01-01 + + +-- !query +select cast('a' as date) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('2022-01-01 00:00:00' as timestamp) +-- !query schema +struct +-- !query output +2022-01-01 00:00:00 + + +-- !query +select cast('a' as timestamp) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('2022-01-01 00:00:00' as timestamp_ntz) +-- !query schema +struct +-- !query output +2022-01-01 00:00:00 + + +-- !query +select cast('a' as timestamp_ntz) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast(cast('inf' as double) as timestamp) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast(cast('inf' as float) as timestamp) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast(interval '1' year as tinyint) +-- !query schema +struct +-- !query output +1 + + +-- !query +select cast(interval '-10-2' year to month as smallint) +-- !query schema +struct +-- !query output +-122 + + +-- !query +select cast(interval '1000' month as int) +-- !query schema +struct +-- !query output +1000 + + +-- !query +select cast(interval -'10.123456' second as tinyint) +-- !query schema +struct +-- !query output +-10 + + +-- !query +select cast(interval '23:59:59' hour to second as smallint) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"INTERVAL HOUR TO SECOND\"", + "targetType" : "\"SMALLINT\"", + "value" : "INTERVAL '23:59:59' HOUR TO SECOND" + } +} + + +-- !query +select cast(interval -'1 02:03:04.123' day to second as int) +-- !query schema +struct +-- !query output +-93784 + + +-- !query +select cast(interval '10' day as bigint) +-- !query schema +struct +-- !query output +10 + + +-- !query +select cast(interval '-1000' month as tinyint) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"INTERVAL MONTH\"", + "targetType" : "\"TINYINT\"", + "value" : "INTERVAL '-1000' MONTH" + } +} + + +-- !query +select cast(interval '1000000' second as smallint) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"INTERVAL SECOND\"", + "targetType" : "\"SMALLINT\"", + "value" : "INTERVAL '1000000' SECOND" + } +} + + +-- !query +select cast(1Y as interval year) +-- !query schema +struct +-- !query output +1-0 + + +-- !query +select cast(-122S as interval year to month) +-- !query schema +struct +-- !query output +-10-2 + + +-- !query +select cast(ym as interval year to month) from values(-122S) as t(ym) +-- !query schema +struct +-- !query output +-10-2 + + +-- !query +select cast(1000 as interval month) +-- !query schema +struct +-- !query output +83-4 + + +-- !query +select cast(-10L as interval second) +-- !query schema +struct +-- !query output +-0 00:00:10.000000000 + + +-- !query +select cast(100Y as interval hour to second) +-- !query schema +struct +-- !query output +0 00:01:40.000000000 + + +-- !query +select cast(dt as interval hour to second) from values(100Y) as t(dt) +-- !query schema +struct +-- !query output +0 00:01:40.000000000 + + +-- !query +select cast(-1000S as interval day to second) +-- !query schema +struct +-- !query output +-0 00:16:40.000000000 + + +-- !query +select cast(10 as interval day) +-- !query schema +struct +-- !query output +10 00:00:00.000000000 + + +-- !query +select cast(2147483647 as interval year) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"INT\"", + "targetType" : "\"INTERVAL YEAR\"", + "value" : "2147483647" + } +} + + +-- !query +select cast(-9223372036854775808L as interval day) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"BIGINT\"", + "targetType" : "\"INTERVAL DAY\"", + "value" : "-9223372036854775808L" + } +} + + +-- !query +select cast(interval '-1' year as decimal(10, 0)) +-- !query schema +struct +-- !query output +-1 + + +-- !query +select cast(interval '1.000001' second as decimal(10, 6)) +-- !query schema +struct +-- !query output +1.000001 + + +-- !query +select cast(interval '08:11:10.001' hour to second as decimal(10, 4)) +-- !query schema +struct +-- !query output +29470.0010 + + +-- !query +select cast(interval '1 01:02:03.1' day to second as decimal(8, 1)) +-- !query schema +struct +-- !query output +90123.1 + + +-- !query +select cast(interval '10.123' second as decimal(4, 2)) +-- !query schema +struct +-- !query output +10.12 + + +-- !query +select cast(interval '10.005' second as decimal(4, 2)) +-- !query schema +struct +-- !query output +10.01 + + +-- !query +select cast(interval '10.123' second as decimal(5, 2)) +-- !query schema +struct +-- !query output +10.12 + + +-- !query +select cast(interval '10.123' second as decimal(1, 0)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "1", + "scale" : "0", + "value" : "10.123000" + } +} + + +-- !query +select cast(10.123456BD as interval day to second) +-- !query schema +struct +-- !query output +0 00:00:10.123456000 + + +-- !query +select cast(80.654321BD as interval hour to minute) +-- !query schema +struct +-- !query output +0 01:20:00.000000000 + + +-- !query +select cast(-10.123456BD as interval year to month) +-- !query schema +struct +-- !query output +-0-10 + + +-- !query +select cast(10.654321BD as interval month) +-- !query schema +struct +-- !query output +0-11 + + +-- !query +SELECT '1.23' :: int +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT 'abc' :: int +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT '12345678901234567890123' :: long +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT '' :: int +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT NULL :: int +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT '123.a' :: int +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT '-2147483648' :: int +-- !query schema +struct +-- !query output +-2147483648 + + +-- !query +SELECT HEX('abc' :: binary) +-- !query schema +struct +-- !query output +616263 + + +-- !query +SELECT HEX((123 :: byte) :: binary) +-- !query schema +struct +-- !query output +7B + + +-- !query +SELECT 'interval 3 month 1 hour' :: interval +-- !query schema +struct +-- !query output +3 months 1 hours + + +-- !query +SELECT interval 3 day 1 second :: string +-- !query schema +struct +-- !query output +INTERVAL '3 00:00:01' DAY TO SECOND + + +-- !query +select ' 1 ' :: DOUBLE +-- !query schema +struct +-- !query output +1.0 + + +-- !query +select '1.0 ' :: DEC +-- !query schema +struct +-- !query output +1 + + +-- !query +select '\t\t true \n\r ' :: boolean +-- !query schema +struct +-- !query output +true + + +-- !query +select '2022-01-01 00:00:00' :: timestamp +-- !query schema +struct +-- !query output +2022-01-01 00:00:00 + + +-- !query +select interval '-10-2' year to month :: smallint +-- !query schema +struct +-- !query output +-122 + + +-- !query +select -10L :: interval second +-- !query schema +struct +-- !query output +-0 00:00:10.000000000 + + +-- !query +select interval '08:11:10.001' hour to second :: decimal(10, 4) +-- !query schema +struct +-- !query output +29470.0010 + + +-- !query +select 10.123456BD :: interval day to second +-- !query schema +struct +-- !query output +0 00:00:10.123456000 + + +-- !query +SELECT '1.23' :: int :: long +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT '2147483648' :: long :: int +-- !query schema +struct +-- !query output +-2147483648 + + +-- !query +SELECT CAST('2147483648' :: long AS int) +-- !query schema +struct +-- !query output +-2147483648 + + +-- !query +SELECT map(1, '123', 2, '456')[1] :: int +-- !query schema +struct +-- !query output +123 + + +-- !query +SELECT '2147483648' :: BINT +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_DATATYPE", + "sqlState" : "0A000", + "messageParameters" : { + "typeName" : "\"BINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 27, + "fragment" : "BINT" + } ] +} + + +-- !query +SELECT '2147483648' :: SELECT +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_DATATYPE", + "sqlState" : "0A000", + "messageParameters" : { + "typeName" : "\"SELECT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 29, + "fragment" : "SELECT" + } ] +} + + +-- !query +SELECT FALSE IS NOT NULL :: string +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'::'", + "hint" : "" + } +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/conditional-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/conditional-functions.sql.out new file mode 100644 index 000000000000..33882561f518 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/conditional-functions.sql.out @@ -0,0 +1,170 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE conditional_t USING PARQUET AS SELECT c1, c2 FROM VALUES(1d, 0),(2d, 1),(null, 1),(CAST('NaN' AS DOUBLE), 0) AS t(c1, c2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT nanvl(c2, c1/c2 + c1/c2) FROM conditional_t +-- !query schema +struct +-- !query output +0.0 +0.0 +1.0 +1.0 + + +-- !query +SELECT nanvl(c2, 1/0) FROM conditional_t +-- !query schema +struct +-- !query output +0.0 +0.0 +1.0 +1.0 + + +-- !query +SELECT nanvl(1-0, 1/0) FROM conditional_t +-- !query schema +struct +-- !query output +1.0 +1.0 +1.0 +1.0 + + +-- !query +SELECT if(c2 >= 0, 1-0, 1/0) from conditional_t +-- !query schema +struct<(IF((c2 >= 0), (1 - 0), (1 / 0))):double> +-- !query output +1.0 +1.0 +1.0 +1.0 + + +-- !query +SELECT if(1 == 1, 1, 1/0) +-- !query schema +struct<(IF((1 = 1), 1, (1 / 0))):double> +-- !query output +1.0 + + +-- !query +SELECT if(1 != 1, 1/0, 1) +-- !query schema +struct<(IF((NOT (1 = 1)), (1 / 0), 1)):double> +-- !query output +1.0 + + +-- !query +SELECT coalesce(c2, 1/0) from conditional_t +-- !query schema +struct +-- !query output +0.0 +0.0 +1.0 +1.0 + + +-- !query +SELECT coalesce(1, 1/0) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT coalesce(null, 1, 1/0) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT case when c2 >= 0 then 1 else 1/0 end from conditional_t +-- !query schema +struct= 0) THEN 1 ELSE (1 / 0) END:double> +-- !query output +1.0 +1.0 +1.0 +1.0 + + +-- !query +SELECT case when 1 < 2 then 1 else 1/0 end +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT case when 1 > 2 then 1/0 else 1 end +-- !query schema +struct 2) THEN (1 / 0) ELSE 1 END:double> +-- !query output +1.0 + + +-- !query +SELECT nullifzero(0), + nullifzero(cast(0 as tinyint)), + nullifzero(cast(0 as bigint)), + nullifzero('0'), + nullifzero(0.0), + nullifzero(1), + nullifzero(null) +-- !query schema +struct +-- !query output +NULL NULL NULL NULL NULL 1 NULL + + +-- !query +SELECT nullifzero('abc') +-- !query schema +struct +-- !query output +abc + + +-- !query +SELECT zeroifnull(null), + zeroifnull(1), + zeroifnull(cast(1 as tinyint)), + zeroifnull(cast(1 as bigint)) +-- !query schema +struct +-- !query output +0 1 1 1 + + +-- !query +SELECT zeroifnull('abc') +-- !query schema +struct +-- !query output +abc + + +-- !query +DROP TABLE conditional_t +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/date.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/date.sql.out new file mode 100644 index 000000000000..c46c200ff026 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/date.sql.out @@ -0,0 +1,1223 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view date_view as select '2011-11-11' date_str, '1' int_str +-- !query schema +struct<> +-- !query output + + + +-- !query +select date '2019-01-01\t' +-- !query schema +struct +-- !query output +2019-01-01 + + +-- !query +select date '2020-01-01中文' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2020-01-01中文'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "date '2020-01-01中文'" + } ] +} + + +-- !query +select make_date(2019, 1, 1), make_date(12, 12, 12) +-- !query schema +struct +-- !query output +2019-01-01 0012-12-12 + + +-- !query +select make_date(2000, 13, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select make_date(2000, 1, 33) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date'015' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'015'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "date'015'" + } ] +} + + +-- !query +select date'2021-4294967297-11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2021-4294967297-11'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "date'2021-4294967297-11'" + } ] +} + + +-- !query +select current_date = current_date +-- !query schema +struct<(current_date() = current_date()):boolean> +-- !query output +true + + +-- !query +select current_date() = current_date() +-- !query schema +struct<(current_date() = current_date()):boolean> +-- !query output +true + + +-- !query +select curdate(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "0", + "functionName" : "`curdate`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "curdate(1)" + } ] +} + + +-- !query +select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null) +-- !query schema +struct +-- !query output +1970-01-01 1972-09-27 NULL + + +-- !query +select UNIX_DATE(DATE('1970-01-01')), UNIX_DATE(DATE('2020-12-04')), UNIX_DATE(null) +-- !query schema +struct +-- !query output +0 18600 NULL + + +-- !query +select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL 2016-12-31 2016-12-31 + + +-- !query +select to_date("16", "dd") +-- !query schema +struct +-- !query output +1970-01-16 + + +-- !query +select to_date("02-29", "MM-dd") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), + dayofweek('1582-10-15 13:10:15'), dayofweek(timestamp_ltz'1582-10-15 13:10:15'), dayofweek(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +7 5 7 NULL 6 6 6 + + +-- !query +select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null), + weekday('1582-10-15 13:10:15'), weekday(timestamp_ltz'1582-10-15 13:10:15'), weekday(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +5 3 5 NULL 4 4 4 + + +-- !query +select year('1500-01-01'), year('1582-10-15 13:10:15'), year(timestamp_ltz'1582-10-15 13:10:15'), year(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +1500 1582 1582 1582 + + +-- !query +select month('1500-01-01'), month('1582-10-15 13:10:15'), month(timestamp_ltz'1582-10-15 13:10:15'), month(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +1 10 10 10 + + +-- !query +select dayOfYear('1500-01-01'), dayOfYear('1582-10-15 13:10:15'), dayOfYear(timestamp_ltz'1582-10-15 13:10:15'), dayOfYear(timestamp_ntz'1582-10-15 13:10:15') +-- !query schema +struct +-- !query output +1 288 288 288 + + +-- !query +select next_day("2015-07-23", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day("2015-07-23", "xx") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select next_day("2015-07-23 12:12:12", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day(timestamp_ltz"2015-07-23 12:12:12", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day(timestamp_ntz"2015-07-23 12:12:12", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day("xx", "Mon") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select next_day(null, "Mon") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select next_day(null, "xx") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_add(date'2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1Y) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1S) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', 1L) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "date_add('2011-11-11', 1L)" + } ] +} + + +-- !query +select date_add('2011-11-11', 1.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_add('2011-11-11', 1.0)" + } ] +} + + +-- !query +select date_add('2011-11-11', 1E1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_add('2011-11-11', 1E1)" + } ] +} + + +-- !query +select date_add('2011-11-11', '1') +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add('2011-11-11', '1.2') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "date_add" + } +} + + +-- !query +select date_add(null, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_add(date'2011-11-11', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_add(timestamp_ltz'2011-11-11 12:12:12', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_add(timestamp_ntz'2011-11-11 12:12:12', 1) +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_sub(date'2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub('2011-11-11', 1) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub('2011-11-11', 1Y) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub('2011-11-11', 1S) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub('2011-11-11', 1L) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "date_sub('2011-11-11', 1L)" + } ] +} + + +-- !query +select date_sub('2011-11-11', 1.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_sub('2011-11-11', 1.0)" + } ] +} + + +-- !query +select date_sub('2011-11-11', 1E1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "date_sub('2011-11-11', 1E1)" + } ] +} + + +-- !query +select date_sub(date'2011-11-11', '1') +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub(date'2011-11-11', '1.2') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "date_sub" + } +} + + +-- !query +select date_sub(null, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_sub(date'2011-11-11', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date_sub(timestamp_ltz'2011-11-11 12:12:12', 1) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_sub(timestamp_ntz'2011-11-11 12:12:12', 1) +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date_add('2011-11-11', int_str) from date_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"int_str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(2011-11-11, int_str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "date_add('2011-11-11', int_str)" + } ] +} + + +-- !query +select date_sub('2011-11-11', int_str) from date_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"int_str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(2011-11-11, int_str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "date_sub('2011-11-11', int_str)" + } ] +} + + +-- !query +select date_add(date_str, 1) from date_view +-- !query schema +struct +-- !query output +2011-11-12 + + +-- !query +select date_sub(date_str, 1) from date_view +-- !query schema +struct +-- !query output +2011-11-10 + + +-- !query +select date '2011-11-11' + 1E1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"10.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(DATE '2011-11-11', 10.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "date '2011-11-11' + 1E1" + } ] +} + + +-- !query +select date '2001-09-28' + 7Y +-- !query schema +struct +-- !query output +2001-10-05 + + +-- !query +select 7S + date '2001-09-28' +-- !query schema +struct +-- !query output +2001-10-05 + + +-- !query +select date '2001-10-01' - 7 +-- !query schema +struct +-- !query output +2001-09-24 + + +-- !query +select date '2001-10-01' - date '2001-09-28' +-- !query schema +struct<(DATE '2001-10-01' - DATE '2001-09-28'):interval day> +-- !query output +3 00:00:00.000000000 + + +-- !query +select date '2001-10-01' - '2001-09-28' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2001-09-28\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(DATE '2001-10-01', 2001-09-28)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "date '2001-10-01' - '2001-09-28'" + } ] +} + + +-- !query +select '2001-10-01' - date '2001-09-28' +-- !query schema +struct<(2001-10-01 - DATE '2001-09-28'):interval day> +-- !query output +3 00:00:00.000000000 + + +-- !query +select date '2001-09-28' - null +-- !query schema +struct +-- !query output +NULL + + +-- !query +select null - date '2019-10-06' +-- !query schema +struct<(NULL - DATE '2019-10-06'):interval day> +-- !query output +NULL + + +-- !query +select date_str - date '2001-09-28' from date_view +-- !query schema +struct<(date_str - DATE '2001-09-28'):interval day> +-- !query output +3696 00:00:00.000000000 + + +-- !query +select date '2001-09-28' - date_str from date_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"date_str\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(DATE '2001-09-28', date_str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "date '2001-09-28' - date_str" + } ] +} + + +-- !query +select date'2011-11-11' + '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "date'2011-11-11' + '1'" + } ] +} + + +-- !query +select '1' + date'2011-11-11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' + date'2011-11-11'" + } ] +} + + +-- !query +select date'2011-11-11' + null +-- !query schema +struct +-- !query output +NULL + + +-- !query +select null + date'2011-11-11' +-- !query schema +struct +-- !query output +NULL + + +-- !query +select date '2012-01-01' - interval '2-2' year to month, + date '2011-11-11' - interval '2' day, + date '2012-01-01' + interval '-2-2' year to month, + date '2011-11-11' + interval '-2' month, + - interval '2-2' year to month + date '2012-01-01', + interval '-2' day + date '2011-11-11' +-- !query schema +struct +-- !query output +2009-11-01 2011-11-09 2009-11-01 2011-09-11 2009-11-01 2011-11-09 + + +-- !query +select to_date('26/October/2015', 'dd/MMMMM/yyyy') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd/MMMMM/yyyy'" + } +} + + +-- !query +select from_json('{"d":"26/October/2015"}', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd/MMMMM/yyyy'" + } +} + + +-- !query +select from_csv('26/October/2015', 'd Date', map('dateFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd/MMMMM/yyyy'" + } +} + + +-- !query +select dateadd(MICROSECOND, 1001, timestamp'2022-02-25 01:02:03.123') +-- !query schema +struct +-- !query output +2022-02-25 01:02:03.124001 + + +-- !query +select date_add(MILLISECOND, -1, timestamp'2022-02-25 01:02:03.456') +-- !query schema +struct +-- !query output +2022-02-25 01:02:03.455 + + +-- !query +select dateadd(SECOND, 58, timestamp'2022-02-25 01:02:03') +-- !query schema +struct +-- !query output +2022-02-25 01:03:01 + + +-- !query +select date_add(MINUTE, -100, date'2022-02-25') +-- !query schema +struct +-- !query output +2022-02-24 22:20:00 + + +-- !query +select dateadd(HOUR, -1, timestamp'2022-02-25 01:02:03') +-- !query schema +struct +-- !query output +2022-02-25 00:02:03 + + +-- !query +select date_add(DAY, 367, date'2022-02-25') +-- !query schema +struct +-- !query output +2023-02-27 00:00:00 + + +-- !query +select dateadd(WEEK, -4, timestamp'2022-02-25 01:02:03') +-- !query schema +struct +-- !query output +2022-01-28 01:02:03 + + +-- !query +select date_add(MONTH, -1, timestamp'2022-02-25 01:02:03') +-- !query schema +struct +-- !query output +2022-01-25 01:02:03 + + +-- !query +select dateadd(QUARTER, 5, date'2022-02-25') +-- !query schema +struct +-- !query output +2023-05-25 00:00:00 + + +-- !query +select date_add(YEAR, 1, date'2022-02-25') +-- !query schema +struct +-- !query output +2023-02-25 00:00:00 + + +-- !query +select dateadd('MICROSECOND', 1001, timestamp'2022-02-25 01:02:03.123') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`dateadd`", + "invalidValue" : "'MICROSECOND'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "dateadd('MICROSECOND', 1001, timestamp'2022-02-25 01:02:03.123')" + } ] +} + + +-- !query +select date_add('QUARTER', 5, date'2022-02-25') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`date_add`", + "invalidValue" : "'QUARTER'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "date_add('QUARTER', 5, date'2022-02-25')" + } ] +} + + +-- !query +select datediff(MICROSECOND, timestamp'2022-02-25 01:02:03.123', timestamp'2022-02-25 01:02:03.124001') +-- !query schema +struct +-- !query output +1001 + + +-- !query +select date_diff(MILLISECOND, timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select datediff(SECOND, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 01:03:01') +-- !query schema +struct +-- !query output +58 + + +-- !query +select date_diff(MINUTE, date'2022-02-25', timestamp'2022-02-24 22:20:00') +-- !query schema +struct +-- !query output +-100 + + +-- !query +select datediff(HOUR, timestamp'2022-02-25 01:02:03', timestamp'2022-02-25 00:02:03') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select date_diff(DAY, date'2022-02-25', timestamp'2023-02-27 00:00:00') +-- !query schema +struct +-- !query output +367 + + +-- !query +select datediff(WEEK, timestamp'2022-02-25 01:02:03', timestamp'2022-01-28 01:02:03') +-- !query schema +struct +-- !query output +-4 + + +-- !query +select date_diff(MONTH, timestamp'2022-02-25 01:02:03', timestamp'2022-01-25 01:02:03') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select datediff(QUARTER, date'2022-02-25', date'2023-05-25') +-- !query schema +struct +-- !query output +5 + + +-- !query +select date_diff(YEAR, date'2022-02-25', date'2023-02-25') +-- !query schema +struct +-- !query output +1 + + +-- !query +select date_diff('MILLISECOND', timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`date_diff`", + "invalidValue" : "'MILLISECOND'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 103, + "fragment" : "date_diff('MILLISECOND', timestamp'2022-02-25 01:02:03.456', timestamp'2022-02-25 01:02:03.455')" + } ] +} + + +-- !query +select datediff('YEAR', date'2022-02-25', date'2023-02-25') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`datediff`", + "invalidValue" : "'YEAR'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "datediff('YEAR', date'2022-02-25', date'2023-02-25')" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/datetime-parsing-invalid.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/datetime-parsing-invalid.sql.out new file mode 100644 index 000000000000..fffbb2a4e017 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/datetime-parsing-invalid.sql.out @@ -0,0 +1,241 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select to_timestamp('294248', 'y') +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select to_timestamp('1', 'yy') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('-12', 'yy') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('123', 'yy') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1', 'yyy') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('1234567', 'yyyyyyy') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'yyyyyyy'" + } +} + + +-- !query +select to_timestamp('366', 'D') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('9', 'DD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('366', 'DD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('9', 'DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('99', 'DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('30-365', 'dd-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('11-365', 'MM-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-366', 'yyyy-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('12-30-365', 'MM-dd-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2020-01-365', 'yyyy-dd-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2020-10-350', 'yyyy-MM-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2020-11-31-366', 'yyyy-MM-dd-DDD') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_csv('2018-366', 'date Date', map('dateFormat', 'yyyy-DDD')) +-- !query schema +struct> +-- !query output +{"date":null} + + +-- !query +select to_date("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_date("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast("Unparseable" as timestamp) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast("Unparseable" as date) +-- !query schema +struct +-- !query output +NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/datetime-special.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/datetime-special.sql.out new file mode 100644 index 000000000000..4f7ecf4b03f4 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/datetime-special.sql.out @@ -0,0 +1,31 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select date'999999-03-18', date'-0001-1-28', date'0015' +-- !query schema +struct +-- !query output ++999999-03-18 -0001-01-28 0015-01-01 + + +-- !query +select make_date(999999, 3, 18), make_date(-1, 1, 28) +-- !query schema +struct +-- !query output ++999999-03-18 -0001-01-28 + + +-- !query +select timestamp'-1969-12-31 16:00:00', timestamp'-0015-03-18 16:00:00', timestamp'-000001', timestamp'99999-03-18T12:03:17' +-- !query schema +struct +-- !query output +-1969-12-31 16:00:00 -0015-03-18 16:00:00 -0001-01-01 00:00:00 +99999-03-18 12:03:17 + + +-- !query +select make_timestamp(-1969, 12, 31, 16, 0, 0.0), make_timestamp(-15, 3, 18, 16, 0, 0.0), make_timestamp(99999, 3, 18, 12, 3, 17.0) +-- !query schema +struct +-- !query output +-1969-12-31 16:00:00 -0015-03-18 16:00:00 +99999-03-18 12:03:17 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/decimalArithmeticOperations.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/decimalArithmeticOperations.sql.out new file mode 100644 index 000000000000..8276168d8bb8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/decimalArithmeticOperations.sql.out @@ -0,0 +1,473 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1.0 as a, 0.0 as b +-- !query schema +struct<> +-- !query output + + + +-- !query +select a / b from t +-- !query schema +struct<(a / b):decimal(8,6)> +-- !query output +NULL + + +-- !query +select a % b from t +-- !query schema +struct<(a % b):decimal(1,1)> +-- !query output +NULL + + +-- !query +select pmod(a, b) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +create table decimals_test(id int, a decimal(38,18), b decimal(38,18)) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into decimals_test values(1, 100.0, 999.0), (2, 12345.123, 12345.123), + (3, 0.1234567891011, 1234.1), (4, 123456789123456789.0, 1.123456789123456789) +-- !query schema +struct<> +-- !query output + + + +-- !query +select id, a+b, a-b, a*b, a/b from decimals_test order by id +-- !query schema +struct +-- !query output +1 1099.00000000000000000 -899.00000000000000000 99900.000000 0.100100 +2 24690.24600000000000000 0.00000000000000000 152402061.885129 1.000000 +3 1234.22345678910110000 -1233.97654321089890000 152.358023 0.000100 +4 123456789123456790.12345678912345679 123456789123456787.87654321087654321 138698367904130467.515623 109890109097814272.043109 + + +-- !query +select id, a*10, b/10 from decimals_test order by id +-- !query schema +struct +-- !query output +1 1000.000000000000000 99.900000000000000000 +2 123451.230000000000000 1234.512300000000000000 +3 1.234567891011000 123.410000000000000000 +4 1234567891234567890.000000000000000 0.112345678912345679 + + +-- !query +select 10.3 * 3.0 +-- !query schema +struct<(10.3 * 3.0):decimal(6,2)> +-- !query output +30.90 + + +-- !query +select 10.3000 * 3.0 +-- !query schema +struct<(10.3000 * 3.0):decimal(9,5)> +-- !query output +30.90000 + + +-- !query +select 10.30000 * 30.0 +-- !query schema +struct<(10.30000 * 30.0):decimal(11,6)> +-- !query output +309.000000 + + +-- !query +select 10.300000000000000000 * 3.000000000000000000 +-- !query schema +struct<(10.300000000000000000 * 3.000000000000000000):decimal(38,34)> +-- !query output +30.9000000000000000000000000000000000 + + +-- !query +select 10.300000000000000000 * 3.0000000000000000000 +-- !query schema +struct<(10.300000000000000000 * 3.0000000000000000000):decimal(38,34)> +-- !query output +30.9000000000000000000000000000000000 + + +-- !query +select 2.35E10 * 1.0 +-- !query schema +struct<(2.35E10 * 1.0):double> +-- !query output +2.35E10 + + +-- !query +select (5e36BD + 0.1) + 5e36BD +-- !query schema +struct<((5000000000000000000000000000000000000 + 0.1) + 5000000000000000000000000000000000000):decimal(38,1)> +-- !query output +NULL + + +-- !query +select (-4e36BD - 0.1) - 7e36BD +-- !query schema +struct<((-4000000000000000000000000000000000000 - 0.1) - 7000000000000000000000000000000000000):decimal(38,1)> +-- !query output +NULL + + +-- !query +select 12345678901234567890.0 * 12345678901234567890.0 +-- !query schema +struct<(12345678901234567890.0 * 12345678901234567890.0):decimal(38,2)> +-- !query output +NULL + + +-- !query +select 1e35BD / 0.1 +-- !query schema +struct<(100000000000000000000000000000000000 / 0.1):decimal(38,6)> +-- !query output +NULL + + +-- !query +select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD +-- !query schema +struct<(1234567890123456789000000000000 * 12345678901234567890000000):decimal(38,0)> +-- !query output +NULL + + +-- !query +select 12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345 +-- !query schema +struct<(12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345):decimal(38,6)> +-- !query output +10012345678912345678912345678911.246907 + + +-- !query +select 123456789123456789.1234567890 * 1.123456789123456789 +-- !query schema +struct<(123456789123456789.1234567890 * 1.123456789123456789):decimal(38,18)> +-- !query output +138698367904130467.654320988515622621 + + +-- !query +select 12345678912345.123456789123 / 0.000000012345678 +-- !query schema +struct<(12345678912345.123456789123 / 1.2345678E-8):decimal(38,9)> +-- !query output +1000000073899961059796.725866332 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) + CAST(90 AS DECIMAL(3, 1)) +-- !query schema +struct +-- !query output +100.0 +20.0 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) - CAST(-90 AS DECIMAL(3, 1)) +-- !query schema +struct +-- !query output +100.0 +20.0 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) * CAST(10 AS DECIMAL(3, 1)) +-- !query schema +struct +-- !query output +100.00 +20.00 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(3, 1)) / CAST(10 AS DECIMAL(3, 1)) +-- !query schema +struct +-- !query output +1.000000 +20.000000 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(10, 2)) % CAST(3 AS DECIMAL(5, 1)) +-- !query schema +struct +-- !query output +1.00 +20.00 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT pmod(CAST(10 AS DECIMAL(10, 2)), CAST(3 AS DECIMAL(5, 1))) +-- !query schema +struct +-- !query output +1.00 +20.00 + + +-- !query +SELECT CAST(20 AS DECIMAL(4, 1)) +UNION ALL +SELECT CAST(10 AS DECIMAL(10, 2)) div CAST(3 AS DECIMAL(5, 1)) +-- !query schema +struct +-- !query output +20.0 +3.0 + + +-- !query +set spark.sql.decimalOperations.allowPrecisionLoss=false +-- !query schema +struct +-- !query output +spark.sql.decimalOperations.allowPrecisionLoss false + + +-- !query +select /*+ COALESCE(1) */ id, a+b, a-b, a*b, a/b from decimals_test order by id +-- !query schema +struct +-- !query output +1 1099.000000000000000000 -899.000000000000000000 NULL 0.100100100100100100 +2 24690.246000000000000000 0.000000000000000000 NULL 1.000000000000000000 +3 1234.223456789101100000 -1233.976543210898900000 NULL 0.000100037913541123 +4 123456789123456790.123456789123456789 123456789123456787.876543210876543211 NULL 109890109097814272.043109406191131436 + + +-- !query +select id, a*10, b/10 from decimals_test order by id +-- !query schema +struct +-- !query output +1 1000.000000000000000000 99.9000000000000000000 +2 123451.230000000000000000 1234.5123000000000000000 +3 1.234567891011000000 123.4100000000000000000 +4 1234567891234567890.000000000000000000 0.1123456789123456789 + + +-- !query +select 10.3 * 3.0 +-- !query schema +struct<(10.3 * 3.0):decimal(6,2)> +-- !query output +30.90 + + +-- !query +select 10.3000 * 3.0 +-- !query schema +struct<(10.3000 * 3.0):decimal(9,5)> +-- !query output +30.90000 + + +-- !query +select 10.30000 * 30.0 +-- !query schema +struct<(10.30000 * 30.0):decimal(11,6)> +-- !query output +309.000000 + + +-- !query +select 10.300000000000000000 * 3.000000000000000000 +-- !query schema +struct<(10.300000000000000000 * 3.000000000000000000):decimal(38,36)> +-- !query output +30.900000000000000000000000000000000000 + + +-- !query +select 10.300000000000000000 * 3.0000000000000000000 +-- !query schema +struct<(10.300000000000000000 * 3.0000000000000000000):decimal(38,37)> +-- !query output +NULL + + +-- !query +select 2.35E10 * 1.0 +-- !query schema +struct<(2.35E10 * 1.0):double> +-- !query output +2.35E10 + + +-- !query +select (5e36BD + 0.1) + 5e36BD +-- !query schema +struct<((5000000000000000000000000000000000000 + 0.1) + 5000000000000000000000000000000000000):decimal(38,1)> +-- !query output +NULL + + +-- !query +select (-4e36BD - 0.1) - 7e36BD +-- !query schema +struct<((-4000000000000000000000000000000000000 - 0.1) - 7000000000000000000000000000000000000):decimal(38,1)> +-- !query output +NULL + + +-- !query +select 12345678901234567890.0 * 12345678901234567890.0 +-- !query schema +struct<(12345678901234567890.0 * 12345678901234567890.0):decimal(38,2)> +-- !query output +NULL + + +-- !query +select 1e35BD / 0.1 +-- !query schema +struct<(100000000000000000000000000000000000 / 0.1):decimal(38,3)> +-- !query output +NULL + + +-- !query +select 1.2345678901234567890E30BD * 1.2345678901234567890E25BD +-- !query schema +struct<(1234567890123456789000000000000 * 12345678901234567890000000):decimal(38,0)> +-- !query output +NULL + + +-- !query +select 12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345 +-- !query schema +struct<(12345678912345678912345678912.1234567 + 9999999999999999999999999999999.12345):decimal(38,7)> +-- !query output +NULL + + +-- !query +select 123456789123456789.1234567890 * 1.123456789123456789 +-- !query schema +struct<(123456789123456789.1234567890 * 1.123456789123456789):decimal(38,28)> +-- !query output +NULL + + +-- !query +select 12345678912345.123456789123 / 0.000000012345678 +-- !query schema +struct<(12345678912345.123456789123 / 1.2345678E-8):decimal(38,18)> +-- !query output +NULL + + +-- !query +select 1.0123456789012345678901234567890123456e36BD / 0.1 +-- !query schema +struct<(1012345678901234567890123456789012345.6 / 0.1):decimal(38,2)> +-- !query output +NULL + + +-- !query +select 1.0123456789012345678901234567890123456e35BD / 1.0 +-- !query schema +struct<(101234567890123456789012345678901234.56 / 1.0):decimal(38,3)> +-- !query output +NULL + + +-- !query +select 1.0123456789012345678901234567890123456e34BD / 1.0 +-- !query schema +struct<(10123456789012345678901234567890123.456 / 1.0):decimal(38,3)> +-- !query output +10123456789012345678901234567890123.456 + + +-- !query +select 1.0123456789012345678901234567890123456e33BD / 1.0 +-- !query schema +struct<(1012345678901234567890123456789012.3456 / 1.0):decimal(38,4)> +-- !query output +1012345678901234567890123456789012.3456 + + +-- !query +select 1.0123456789012345678901234567890123456e32BD / 1.0 +-- !query schema +struct<(101234567890123456789012345678901.23456 / 1.0):decimal(38,5)> +-- !query output +101234567890123456789012345678901.23456 + + +-- !query +select 1.0123456789012345678901234567890123456e31BD / 1.0 +-- !query schema +struct<(10123456789012345678901234567890.123456 / 1.0):decimal(38,6)> +-- !query output +10123456789012345678901234567890.123456 + + +-- !query +select 1.0123456789012345678901234567890123456e31BD / 0.1 +-- !query schema +struct<(10123456789012345678901234567890.123456 / 0.1):decimal(38,6)> +-- !query output +NULL + + +-- !query +select 1.0123456789012345678901234567890123456e31BD / 10.0 +-- !query schema +struct<(10123456789012345678901234567890.123456 / 10.0):decimal(38,7)> +-- !query output +1012345678901234567890123456789.0123456 + + +-- !query +drop table decimals_test +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/double-quoted-identifiers.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/double-quoted-identifiers.sql.out new file mode 100644 index 000000000000..81a98a60590f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/double-quoted-identifiers.sql.out @@ -0,0 +1,443 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT 1 FROM "not_exist" +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +USE SCHEMA "not_exist" +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +ALTER TABLE "not_exist" ADD COLUMN not_exist int +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +ALTER TABLE not_exist ADD COLUMN "not_exist" int +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +SELECT 1 AS "not_exist" FROM not_exist +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +SELECT 1 FROM not_exist AS X("hello") +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"hello\"'", + "hint" : "" + } +} + + +-- !query +SELECT "not_exist"() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +SELECT "not_exist".not_exist() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"not_exist\"'", + "hint" : "" + } +} + + +-- !query +SELECT 1 FROM `hello` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`hello`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 21, + "fragment" : "`hello`" + } ] +} + + +-- !query +USE SCHEMA `not_exist` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException +{ + "errorClass" : "SCHEMA_NOT_FOUND", + "sqlState" : "42704", + "messageParameters" : { + "schemaName" : "`spark_catalog`.`not_exist`" + } +} + + +-- !query +ALTER TABLE `not_exist` ADD COLUMN not_exist int +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 23, + "fragment" : "`not_exist`" + } ] +} + + +-- !query +ALTER TABLE not_exist ADD COLUMN `not_exist` int +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 21, + "fragment" : "not_exist" + } ] +} + + +-- !query +SELECT 1 AS `not_exist` FROM `not_exist` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 40, + "fragment" : "`not_exist`" + } ] +} + + +-- !query +SELECT 1 FROM not_exist AS X(`hello`) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`not_exist`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 23, + "fragment" : "not_exist" + } ] +} + + +-- !query +SELECT `not_exist`() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`not_exist`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "`not_exist`()" + } ] +} + + +-- !query +SELECT `not_exist`.not_exist() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`not_exist`.`not_exist`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "`not_exist`.not_exist()" + } ] +} + + +-- !query +SELECT "hello" +-- !query schema +struct +-- !query output +hello + + +-- !query +CREATE TEMPORARY VIEW v(c1 COMMENT "hello") AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW v +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT INTERVAL "1" YEAR +-- !query schema +struct +-- !query output +1-0 + + +-- !query +SELECT 'hello' +-- !query schema +struct +-- !query output +hello + + +-- !query +CREATE TEMPORARY VIEW v(c1 COMMENT 'hello') AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW v +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT INTERVAL '1' YEAR +-- !query schema +struct +-- !query output +1-0 + + +-- !query +CREATE SCHEMA "myschema" +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"myschema\"'", + "hint" : "" + } +} + + +-- !query +CREATE TEMPORARY VIEW "myview"("c1") AS + WITH "v"("a") AS (SELECT 1) SELECT "a" FROM "v" +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"myview\"'", + "hint" : "" + } +} + + +-- !query +SELECT "a1" AS "a2" FROM "myview" AS "atab"("a1") +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"a2\"'", + "hint" : "" + } +} + + +-- !query +DROP TABLE "myview" +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"myview\"'", + "hint" : "" + } +} + + +-- !query +DROP SCHEMA "myschema" +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'\"myschema\"'", + "hint" : "" + } +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/higher-order-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/higher-order-functions.sql.out new file mode 100644 index 000000000000..7bfc35a61e09 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/higher-order-functions.sql.out @@ -0,0 +1,350 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create or replace temporary view nested as values + (1, array(32, 97), array(array(12, 99), array(123, 42), array(1))), + (2, array(77, -76), array(array(6, 96, 65), array(-1, -2))), + (3, array(12), array(array(17))) + as t(x, ys, zs) +-- !query schema +struct<> +-- !query output + + + +-- !query +select upper(x -> x) as v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_LAMBDA_FUNCTION_CALL.NON_HIGHER_ORDER_FUNCTION", + "sqlState" : "42K0D", + "messageParameters" : { + "class" : "org.apache.spark.sql.catalyst.expressions.Upper" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "upper(x -> x)" + } ] +} + + +-- !query +select ceil(x -> x) as v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_LAMBDA_FUNCTION_CALL.NON_HIGHER_ORDER_FUNCTION", + "sqlState" : "42K0D", + "messageParameters" : { + "class" : "org.apache.spark.sql.catalyst.expressions.CeilExpressionBuilder$" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "ceil(x -> x)" + } ] +} + + +-- !query +select transform(zs, z -> z) as v from nested +-- !query schema +struct>> +-- !query output +[[12,99],[123,42],[1]] +[[17]] +[[6,96,65],[-1,-2]] + + +-- !query +select transform(ys, y -> y * y) as v from nested +-- !query schema +struct> +-- !query output +[1024,9409] +[144] +[5929,5776] + + +-- !query +select transform(ys, (y, i) -> y + i) as v from nested +-- !query schema +struct> +-- !query output +[12] +[32,98] +[77,-75] + + +-- !query +select transform(zs, z -> concat(ys, z)) as v from nested +-- !query schema +struct>> +-- !query output +[[12,17]] +[[32,97,12,99],[32,97,123,42],[32,97,1]] +[[77,-76,6,96,65],[77,-76,-1,-2]] + + +-- !query +select transform(ys, 0) as v from nested +-- !query schema +struct> +-- !query output +[0,0] +[0,0] +[0] + + +-- !query +select transform(cast(null as array), x -> x + 1) as v +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select filter(ys, y -> y > 30) as v from nested +-- !query schema +struct> +-- !query output +[32,97] +[77] +[] + + +-- !query +select filter(cast(null as array), y -> true) as v +-- !query schema +struct> +-- !query output +NULL + + +-- !query +select transform(zs, z -> filter(z, zz -> zz > 50)) as v from nested +-- !query schema +struct>> +-- !query output +[[96,65],[]] +[[99],[123],[]] +[[]] + + +-- !query +select aggregate(ys, 0, (y, a) -> y + a + x) as v from nested +-- !query schema +struct +-- !query output +131 +15 +5 + + +-- !query +select aggregate(ys, (0 as sum, 0 as n), (acc, x) -> (acc.sum + x, acc.n + 1), acc -> acc.sum / acc.n) as v from nested +-- !query schema +struct +-- !query output +0.5 +12.0 +64.5 + + +-- !query +select transform(zs, z -> aggregate(z, 1, (acc, val) -> acc * val * size(z))) as v from nested +-- !query schema +struct> +-- !query output +[1010880,8] +[17] +[4752,20664,1] + + +-- !query +select aggregate(cast(null as array), 0, (a, y) -> a + y + 1, a -> a + 2) as v +-- !query schema +struct +-- !query output +NULL + + +-- !query +select reduce(ys, 0, (y, a) -> y + a + x) as v from nested +-- !query schema +struct +-- !query output +131 +15 +5 + + +-- !query +select reduce(ys, (0 as sum, 0 as n), (acc, x) -> (acc.sum + x, acc.n + 1), acc -> acc.sum / acc.n) as v from nested +-- !query schema +struct +-- !query output +0.5 +12.0 +64.5 + + +-- !query +select transform(zs, z -> reduce(z, 1, (acc, val) -> acc * val * size(z))) as v from nested +-- !query schema +struct> +-- !query output +[1010880,8] +[17] +[4752,20664,1] + + +-- !query +select reduce(cast(null as array), 0, (a, y) -> a + y + 1, a -> a + 2) as v +-- !query schema +struct +-- !query output +NULL + + +-- !query +select exists(ys, y -> y > 30) as v from nested +-- !query schema +struct +-- !query output +false +true +true + + +-- !query +select exists(cast(null as array), y -> y > 30) as v +-- !query schema +struct +-- !query output +NULL + + +-- !query +select zip_with(ys, zs, (a, b) -> a + size(b)) as v from nested +-- !query schema +struct> +-- !query output +[13] +[34,99,null] +[80,-74] + + +-- !query +select zip_with(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) -> concat(x, y)) as v +-- !query schema +struct> +-- !query output +["ad","be","cf"] + + +-- !query +select zip_with(array('a'), array('d', null, 'f'), (x, y) -> coalesce(x, y)) as v +-- !query schema +struct> +-- !query output +["a",null,"f"] + + +-- !query +create or replace temporary view nested as values + (1, map(1, 1, 2, 2, 3, 3)), + (2, map(4, 4, 5, 5, 6, 6)) + as t(x, ys) +-- !query schema +struct<> +-- !query output + + + +-- !query +select transform_keys(ys, (k, v) -> k) as v from nested +-- !query schema +struct> +-- !query output +{1:1,2:2,3:3} +{4:4,5:5,6:6} + + +-- !query +select transform_keys(ys, (k, v) -> k + 1) as v from nested +-- !query schema +struct> +-- !query output +{2:1,3:2,4:3} +{5:4,6:5,7:6} + + +-- !query +select transform_keys(ys, (k, v) -> k + v) as v from nested +-- !query schema +struct> +-- !query output +{10:5,12:6,8:4} +{2:1,4:2,6:3} + + +-- !query +select transform_values(ys, (k, v) -> v) as v from nested +-- !query schema +struct> +-- !query output +{1:1,2:2,3:3} +{4:4,5:5,6:6} + + +-- !query +select transform_values(ys, (k, v) -> v + 1) as v from nested +-- !query schema +struct> +-- !query output +{1:2,2:3,3:4} +{4:5,5:6,6:7} + + +-- !query +select transform_values(ys, (k, v) -> k + v) as v from nested +-- !query schema +struct> +-- !query output +{1:2,2:4,3:6} +{4:8,5:10,6:12} + + +-- !query +select transform(ys, all -> all * all) as v from values (array(32, 97)) as t(ys) +-- !query schema +struct> +-- !query output +[1024,9409] + + +-- !query +select transform(ys, (all, i) -> all + i) as v from values (array(32, 97)) as t(ys) +-- !query schema +struct> +-- !query output +[32,98] + + +-- !query +select aggregate(split('abcdefgh',''), array(array('')), (acc, x) -> array(array(x))) +-- !query schema +struct>> +-- !query output +[["h"]] diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/interval.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/interval.sql.out new file mode 100644 index 000000000000..a8a0423bdb3e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/interval.sql.out @@ -0,0 +1,3712 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15') +-- !query schema +struct<((TIMESTAMP '2019-10-15 10:11:12.001002' - DATE '2019-10-15') * 3):interval day to second> +-- !query output +1 06:33:36.003006000 + + +-- !query +select interval 4 month 2 weeks 3 microseconds * 1.5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 4 month 2 weeks 3 microseconds" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "interval 4 month 2 weeks 3 microseconds" + } ] +} + + +-- !query +select interval 2 years 4 months +-- !query schema +struct +-- !query output +2-4 + + +-- !query +select interval 2 weeks 3 microseconds * 1.5 +-- !query schema +struct<(INTERVAL '14 00:00:00.000003' DAY TO SECOND * 1.5):interval day to second> +-- !query output +21 00:00:00.000005000 + + +-- !query +select (timestamp'2019-10-15' - timestamp'2019-10-14') / 1.5 +-- !query schema +struct<((TIMESTAMP '2019-10-15 00:00:00' - TIMESTAMP '2019-10-14 00:00:00') / 1.5):interval day to second> +-- !query output +0 16:00:00.000000000 + + +-- !query +select interval 2147483647 month * 2 +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +integer overflow + + +-- !query +select interval 2147483647 month / 0.5 +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Overflow + + +-- !query +select interval 2147483647 day * 2 +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select interval 2147483647 day / 0.5 +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select interval 2 second * '2' +-- !query schema +struct<(INTERVAL '02' SECOND * 2):interval day to second> +-- !query output +0 00:00:04.000000000 + + +-- !query +select interval 2 second / '2' +-- !query schema +struct<(INTERVAL '02' SECOND / 2):interval day to second> +-- !query output +0 00:00:01.000000000 + + +-- !query +select interval 2 year * '2' +-- !query schema +struct<(INTERVAL '2' YEAR * 2):interval year to month> +-- !query output +4-0 + + +-- !query +select interval 2 year / '2' +-- !query schema +struct<(INTERVAL '2' YEAR / 2):interval year to month> +-- !query output +1-0 + + +-- !query +select interval 2 second * 'a' +-- !query schema +struct<(INTERVAL '02' SECOND * a):interval day to second> +-- !query output +NULL + + +-- !query +select interval 2 second / 'a' +-- !query schema +struct<(INTERVAL '02' SECOND / a):interval day to second> +-- !query output +NULL + + +-- !query +select interval 2 year * 'a' +-- !query schema +struct<(INTERVAL '2' YEAR * a):interval year to month> +-- !query output +NULL + + +-- !query +select interval 2 year / 'a' +-- !query schema +struct<(INTERVAL '2' YEAR / a):interval year to month> +-- !query output +NULL + + +-- !query +select '2' * interval 2 second +-- !query schema +struct<(INTERVAL '02' SECOND * 2):interval day to second> +-- !query output +0 00:00:04.000000000 + + +-- !query +select '2' * interval 2 year +-- !query schema +struct<(INTERVAL '2' YEAR * 2):interval year to month> +-- !query output +4-0 + + +-- !query +select 'a' * interval 2 second +-- !query schema +struct<(INTERVAL '02' SECOND * a):interval day to second> +-- !query output +NULL + + +-- !query +select 'a' * interval 2 year +-- !query schema +struct<(INTERVAL '2' YEAR * a):interval year to month> +-- !query output +NULL + + +-- !query +select '2' / interval 2 second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL SECOND\"", + "sqlExpr" : "\"(2 / INTERVAL '02' SECOND)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'2' / interval 2 second" + } ] +} + + +-- !query +select '2' / interval 2 year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "'2' / interval 2 year" + } ] +} + + +-- !query +select interval '2 seconds' / 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "interval '2 seconds' / 0" + } ] +} + + +-- !query +select interval '2 seconds' / null +-- !query schema +struct<(INTERVAL '02' SECOND / NULL):interval day to second> +-- !query output +NULL + + +-- !query +select interval '2 seconds' * null +-- !query schema +struct<(INTERVAL '02' SECOND * NULL):interval day to second> +-- !query output +NULL + + +-- !query +select null * interval '2 seconds' +-- !query schema +struct<(INTERVAL '02' SECOND * NULL):interval day to second> +-- !query output +NULL + + +-- !query +select interval '2' year / 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "interval '2' year / 0" + } ] +} + + +-- !query +select interval '2' year / null +-- !query schema +struct<(INTERVAL '2' YEAR / NULL):interval year to month> +-- !query output +NULL + + +-- !query +select interval '2' year * null +-- !query schema +struct<(INTERVAL '2' YEAR * NULL):interval year to month> +-- !query output +NULL + + +-- !query +select null * interval '2' year +-- !query schema +struct<(INTERVAL '2' YEAR * NULL):interval year to month> +-- !query output +NULL + + +-- !query +select 2 / interval '2' year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "2 / interval '2' year" + } ] +} + + +-- !query +select 2 / interval '2' hour +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"INTERVAL HOUR\"", + "sqlExpr" : "\"(2 / INTERVAL '02' HOUR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "2 / interval '2' hour" + } ] +} + + +-- !query +select null / interval '2' year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(NULL / INTERVAL '2' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "null / interval '2' year" + } ] +} + + +-- !query +select null / interval '2' hour +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"INTERVAL HOUR\"", + "sqlExpr" : "\"(NULL / INTERVAL '02' HOUR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "null / interval '2' hour" + } ] +} + + +-- !query +select -interval '-1 month 1 day -1 second' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '-1 month 1 day -1 second'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 43, + "fragment" : "interval '-1 month 1 day -1 second'" + } ] +} + + +-- !query +select -interval '-1 year 1 month' +-- !query schema +struct<(- INTERVAL '-0-11' YEAR TO MONTH):interval year to month> +-- !query output +0-11 + + +-- !query +select -interval '-1 day 1 hour -1 minute 1 second' +-- !query schema +struct<(- INTERVAL '-0 23:00:59' DAY TO SECOND):interval day to second> +-- !query output +0 23:00:59.000000000 + + +-- !query +select -interval -1 month 1 day -1 second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval -1 month 1 day -1 second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 41, + "fragment" : "interval -1 month 1 day -1 second" + } ] +} + + +-- !query +select -interval -1 year 1 month +-- !query schema +struct<(- INTERVAL '-0-11' YEAR TO MONTH):interval year to month> +-- !query output +0-11 + + +-- !query +select -interval -1 day 1 hour -1 minute 1 second +-- !query schema +struct<(- INTERVAL '-0 23:00:59' DAY TO SECOND):interval day to second> +-- !query output +0 23:00:59.000000000 + + +-- !query +select +interval '-1 month 1 day -1 second' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '-1 month 1 day -1 second'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 43, + "fragment" : "interval '-1 month 1 day -1 second'" + } ] +} + + +-- !query +select +interval '-1 year 1 month' +-- !query schema +struct<(+ INTERVAL '-0-11' YEAR TO MONTH):interval year to month> +-- !query output +-0-11 + + +-- !query +select +interval '-1 day 1 hour -1 minute 1 second' +-- !query schema +struct<(+ INTERVAL '-0 23:00:59' DAY TO SECOND):interval day to second> +-- !query output +-0 23:00:59.000000000 + + +-- !query +select +interval -1 month 1 day -1 second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval -1 month 1 day -1 second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 9, + "stopIndex" : 41, + "fragment" : "interval -1 month 1 day -1 second" + } ] +} + + +-- !query +select +interval -1 year 1 month +-- !query schema +struct<(+ INTERVAL '-0-11' YEAR TO MONTH):interval year to month> +-- !query output +-0-11 + + +-- !query +select +interval -1 day 1 hour -1 minute 1 second +-- !query schema +struct<(+ INTERVAL '-0 23:00:59' DAY TO SECOND):interval day to second> +-- !query output +-0 23:00:59.000000000 + + +-- !query +select interval -'1-1' year to month +-- !query schema +struct +-- !query output +-1-1 + + +-- !query +select interval -'-1-1' year to month +-- !query schema +struct +-- !query output +1-1 + + +-- !query +select interval +'-1-1' year to month +-- !query schema +struct +-- !query output +-1-1 + + +-- !query +select interval - '1 2:3:4.001' day to second +-- !query schema +struct +-- !query output +-1 02:03:04.001000000 + + +-- !query +select interval +'1 2:3:4.001' day to second +-- !query schema +struct +-- !query output +1 02:03:04.001000000 + + +-- !query +select interval -'-1 2:3:4.001' day to second +-- !query schema +struct +-- !query output +1 02:03:04.001000000 + + +-- !query +select interval -'1' year +-- !query schema +struct +-- !query output +-1-0 + + +-- !query +select interval -'-1' year +-- !query schema +struct +-- !query output +1-0 + + +-- !query +select interval -'11' month +-- !query schema +struct +-- !query output +-0-11 + + +-- !query +select interval -'-11' month +-- !query schema +struct +-- !query output +0-11 + + +-- !query +select interval -'1' day +-- !query schema +struct +-- !query output +-1 00:00:00.000000000 + + +-- !query +select interval -'-1' day +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select interval -'23' hour +-- !query schema +struct +-- !query output +-0 23:00:00.000000000 + + +-- !query +select interval -'-23' hour +-- !query schema +struct +-- !query output +0 23:00:00.000000000 + + +-- !query +select interval -'59' minute +-- !query schema +struct +-- !query output +-0 00:59:00.000000000 + + +-- !query +select interval -'-59' minute +-- !query schema +struct +-- !query output +0 00:59:00.000000000 + + +-- !query +select interval -'59' second +-- !query schema +struct +-- !query output +-0 00:00:59.000000000 + + +-- !query +select interval -'-59' second +-- !query schema +struct +-- !query output +0 00:00:59.000000000 + + +-- !query +select make_interval(1) +-- !query schema +struct +-- !query output +1 years + + +-- !query +select make_interval(1, 2) +-- !query schema +struct +-- !query output +1 years 2 months + + +-- !query +select make_interval(1, 2, 3) +-- !query schema +struct +-- !query output +1 years 2 months 21 days + + +-- !query +select make_interval(1, 2, 3, 4) +-- !query schema +struct +-- !query output +1 years 2 months 25 days + + +-- !query +select make_interval(1, 2, 3, 4, 5) +-- !query schema +struct +-- !query output +1 years 2 months 25 days 5 hours + + +-- !query +select make_interval(1, 2, 3, 4, 5, 6) +-- !query schema +struct +-- !query output +1 years 2 months 25 days 5 hours 6 minutes + + +-- !query +select make_interval(1, 2, 3, 4, 5, 6, 7.008009) +-- !query schema +struct +-- !query output +1 years 2 months 25 days 5 hours 6 minutes 7.008009 seconds + + +-- !query +select make_interval(1, 2, 3, 4, 0, 0, 123456789012.123456) +-- !query schema +struct +-- !query output +1 years 2 months 25 days 34293552 hours 30 minutes 12.123456 seconds + + +-- !query +select make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select make_dt_interval(1) +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select make_dt_interval(1, 2) +-- !query schema +struct +-- !query output +1 02:00:00.000000000 + + +-- !query +select make_dt_interval(1, 2, 3) +-- !query schema +struct +-- !query output +1 02:03:00.000000000 + + +-- !query +select make_dt_interval(1, 2, 3, 4.005006) +-- !query schema +struct +-- !query output +1 02:03:04.005006000 + + +-- !query +select make_dt_interval(1, 0, 0, 123456789012.123456) +-- !query schema +struct +-- !query output +1428899 00:30:12.123456000 + + +-- !query +select make_dt_interval(2147483647) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "make_dt_interval(2147483647)" + } ] +} + + +-- !query +select make_ym_interval(1) +-- !query schema +struct +-- !query output +1-0 + + +-- !query +select make_ym_interval(1, 2) +-- !query schema +struct +-- !query output +1-2 + + +-- !query +select make_ym_interval(0, 1) +-- !query schema +struct +-- !query output +0-1 + + +-- !query +select make_ym_interval(178956970, 7) +-- !query schema +struct +-- !query output +178956970-7 + + +-- !query +select make_ym_interval(178956970, 8) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "make_ym_interval(178956970, 8)" + } ] +} + + +-- !query +select make_ym_interval(-178956970, -8) +-- !query schema +struct +-- !query output +-178956970-8 + + +-- !query +select make_ym_interval(-178956970, -9) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "make_ym_interval(-178956970, -9)" + } ] +} + + +-- !query +select cast('1 second' as interval) +-- !query schema +struct +-- !query output +1 seconds + + +-- !query +select cast('+1 second' as interval) +-- !query schema +struct +-- !query output +1 seconds + + +-- !query +select cast('-1 second' as interval) +-- !query schema +struct +-- !query output +-1 seconds + + +-- !query +select cast('+ 1 second' as interval) +-- !query schema +struct +-- !query output +1 seconds + + +-- !query +select cast('- 1 second' as interval) +-- !query schema +struct +-- !query output +-1 seconds + + +-- !query +select cast('- -1 second' as interval) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cast('- +1 second' as interval) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select interval 13.123456789 seconds, interval -13.123456789 second +-- !query schema +struct +-- !query output +0 00:00:13.123456000 -0 00:00:13.123456000 + + +-- !query +select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond" + } ] +} + + +-- !query +select interval 1 year 2 month +-- !query schema +struct +-- !query output +1-2 + + +-- !query +select interval 4 day 5 hour 6 minute 7 seconds +-- !query schema +struct +-- !query output +4 05:06:07.000000000 + + +-- !query +select interval 3 week 8 millisecond 9 microsecond +-- !query schema +struct +-- !query output +21 00:00:00.008009000 + + +-- !query +select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0029", + "messageParameters" : { + "literal" : "interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 92, + "fragment" : "interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second" + } ] +} + + +-- !query +select interval '30' year '25' month +-- !query schema +struct +-- !query output +32-1 + + +-- !query +select interval '-100' day '40' hour '80' minute '299.889987299' second +-- !query schema +struct +-- !query output +-98 06:35:00.110013000 + + +-- !query +select interval '0-0' year to month +-- !query schema +struct +-- !query output +0-0 + + +-- !query +select interval '0 0:0:0' day to second +-- !query schema +struct +-- !query output +0 00:00:00.000000000 + + +-- !query +select interval '0 0:0:0.1' day to second +-- !query schema +struct +-- !query output +0 00:00:00.100000000 + + +-- !query +select interval '10-9' year to month +-- !query schema +struct +-- !query output +10-9 + + +-- !query +select interval '20 15' day to hour +-- !query schema +struct +-- !query output +20 15:00:00.000000000 + + +-- !query +select interval '20 15:40' day to minute +-- !query schema +struct +-- !query output +20 15:40:00.000000000 + + +-- !query +select interval '20 15:40:32.99899999' day to second +-- !query schema +struct +-- !query output +20 15:40:32.998999000 + + +-- !query +select interval '15:40' hour to minute +-- !query schema +struct +-- !query output +0 15:40:00.000000000 + + +-- !query +select interval '15:40:32.99899999' hour to second +-- !query schema +struct +-- !query output +0 15:40:32.998999000 + + +-- !query +select interval '40:32.99899999' minute to second +-- !query schema +struct +-- !query output +0 00:40:32.998999000 + + +-- !query +select interval '40:32' minute to second +-- !query schema +struct +-- !query output +0 00:40:32.000000000 + + +-- !query +select interval 30 day day +-- !query schema +struct +-- !query output +30 00:00:00.000000000 + + +-- !query +select interval 30 days days +-- !query schema +struct +-- !query output +30 00:00:00.000000000 + + +-- !query +select interval '20 15:40:32.99899999' day to hour +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "20 15:40:32.99899999", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR`", + "typeName" : "interval day to hour" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 50, + "fragment" : "'20 15:40:32.99899999' day to hour" + } ] +} + + +-- !query +select interval '20 15:40:32.99899999' day to minute +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "20 15:40:32.99899999", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]d h:m`, `INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE`", + "typeName" : "interval day to minute" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 52, + "fragment" : "'20 15:40:32.99899999' day to minute" + } ] +} + + +-- !query +select interval '15:40:32.99899999' hour to minute +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "15:40:32.99899999", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE`", + "typeName" : "interval hour to minute" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 50, + "fragment" : "'15:40:32.99899999' hour to minute" + } ] +} + + +-- !query +select interval '15:40.99899999' hour to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "15:40.99899999", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND`", + "typeName" : "interval hour to second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 47, + "fragment" : "'15:40.99899999' hour to second" + } ] +} + + +-- !query +select interval '15:40' hour to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "15:40", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND`", + "typeName" : "interval hour to second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 38, + "fragment" : "'15:40' hour to second" + } ] +} + + +-- !query +select interval '20 40:32.99899999' minute to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "20 40:32.99899999", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND`", + "typeName" : "interval minute to second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 52, + "fragment" : "'20 40:32.99899999' minute to second" + } ] +} + + +-- !query +select interval 10 nanoseconds +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.INVALID_UNIT", + "sqlState" : "22006", + "messageParameters" : { + "input" : " 10 nanoseconds", + "unit" : "nanoseconds" + } +} + + +-- !query +select map(1, interval 1 day, 2, interval 3 week) +-- !query schema +struct> +-- !query output +{1:1 00:00:00.000000000,2:21 00:00:00.000000000} + + +-- !query +select map(1, interval 1 day, 2, interval 2 day) +-- !query schema +struct> +-- !query output +{1:1 00:00:00.000000000,2:2 00:00:00.000000000} + + +-- !query +select map(1, interval 1 year, 2, interval 2 month) +-- !query schema +struct> +-- !query output +{1:1-0,2:0-2} + + +-- !query +select map(1, interval 1 month, 2, interval 2 month) +-- !query schema +struct> +-- !query output +{1:0-1,2:0-2} + + +-- !query +select map(1, interval 1 week, 2, interval 2 day) +-- !query schema +struct> +-- !query output +{1:7 00:00:00.000000000,2:2 00:00:00.000000000} + + +-- !query +select map(1, interval 2 millisecond, 3, interval 3 microsecond) +-- !query schema +struct> +-- !query output +{1:0 00:00:00.002000000,3:0 00:00:00.000003000} + + +-- !query +select interval 'interval 3 year 1 month' +-- !query schema +struct +-- !query output +3-1 + + +-- !query +select interval '3 year 1 month' +-- !query schema +struct +-- !query output +3-1 + + +-- !query +SELECT interval 'interval 2 weeks 2 days 1 hour 3 minutes 2 seconds 100 millisecond 200 microseconds' +-- !query schema +struct +-- !query output +16 01:03:02.100200000 + + +-- !query +SELECT interval '2 weeks 2 days 1 hour 3 minutes 2 seconds 100 millisecond 200 microseconds' +-- !query schema +struct +-- !query output +16 01:03:02.100200000 + + +-- !query +select interval 1 fake_unit +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'fake_unit'", + "hint" : "" + } +} + + +-- !query +select interval 1 year to month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0027", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 31, + "fragment" : "1 year to month" + } ] +} + + +-- !query +select interval '1' year to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNSUPPORTED_FROM_TO_EXPRESSION", + "sqlState" : "22006", + "messageParameters" : { + "from" : "year", + "input" : "1", + "to" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 34, + "fragment" : "'1' year to second" + } ] +} + + +-- !query +select interval '10-9' year to month '2-1' year to month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "interval '10-9' year to month '2-1' year to month" + } ] +} + + +-- !query +select interval '10-9' year to month '12:11:10' hour to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "interval '10-9' year to month '12:11:10' hour to second" + } ] +} + + +-- !query +select interval '1 15:11' day to minute '12:11:10' hour to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 65, + "fragment" : "interval '1 15:11' day to minute '12:11:10' hour to second" + } ] +} + + +-- !query +select interval 1 year '2-1' year to month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "interval 1 year '2-1' year to month" + } ] +} + + +-- !query +select interval 1 year '12:11:10' hour to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "interval 1 year '12:11:10' hour to second" + } ] +} + + +-- !query +select interval '10-9' year to month '1' year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "interval '10-9' year to month '1' year" + } ] +} + + +-- !query +select interval '12:11:10' hour to second '1' year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0024", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "interval '12:11:10' hour to second '1' year" + } ] +} + + +-- !query +select interval (-30) day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval (-30)" + } ] +} + + +-- !query +select interval (a + 1) day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "interval (a + 1)" + } ] +} + + +-- !query +select interval 30 day day day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'day'", + "hint" : ": extra input 'day'" + } +} + + +-- !query +select interval (-30) days +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval (-30)" + } ] +} + + +-- !query +select interval (a + 1) days +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`interval`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "interval (a + 1)" + } ] +} + + +-- !query +select interval 30 days days days +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'days'", + "hint" : ": extra input 'days'" + } +} + + +-- !query +SELECT INTERVAL '178956970-7' YEAR TO MONTH +-- !query schema +struct +-- !query output +178956970-7 + + +-- !query +SELECT INTERVAL '178956970-8' YEAR TO MONTH +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.INTERVAL_PARSING", + "sqlState" : "22006", + "messageParameters" : { + "input" : "178956970-8", + "interval" : "year-month" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 43, + "fragment" : "'178956970-8' YEAR TO MONTH" + } ] +} + + +-- !query +SELECT INTERVAL '-178956970-8' YEAR TO MONTH +-- !query schema +struct +-- !query output +-178956970-8 + + +-- !query +SELECT INTERVAL -'178956970-8' YEAR TO MONTH +-- !query schema +struct +-- !query output +-178956970-8 + + +-- !query +select + interval '2-2' year to month + interval '3' month, + interval '2' year - interval '3-3' year to month, + interval '99 11:22:33.123456789' day to second + interval '10 9:8' day to minute, + interval '22:33.123456789' minute to second - interval '10' day +-- !query schema +struct<(INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' MONTH):interval year to month,(INTERVAL '2' YEAR - INTERVAL '3-3' YEAR TO MONTH):interval year to month,(INTERVAL '99 11:22:33.123456' DAY TO SECOND + INTERVAL '10 09:08' DAY TO MINUTE):interval day to second,(INTERVAL '22:33.123456' MINUTE TO SECOND - INTERVAL '10' DAY):interval day to second> +-- !query output +2-5 -1-3 109 20:30:33.123456000 -9 23:37:26.876544000 + + +-- !query +select + interval '2' year + '3-3 year to month', + interval '2' year - '3 month', + '3-2 year to month' + interval '2-2' year to month, + '3 year' - interval '2-2' year to month, + interval '99 11:22:33.123456789' day to second + '12:12 hour to second', + interval '99 11:22:33.123456789' day to second - '12 hour', + '4 day' + interval '10' day, + '4 22 day to hour' - interval '10' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3 year to month)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 48, + "fragment" : "interval '2' year + '3-3 year to month'" + } ] +} + + +-- !query +select + interval '2' year + null, + interval '2' year - null, + interval '2' hour + null, + interval '2' hour - null, + null + interval '2' year, + null - interval '2' year, + null + interval '2' hour, + null - interval '2' hour +-- !query schema +struct<(INTERVAL '2' YEAR + NULL):interval year,(INTERVAL '2' YEAR - NULL):interval year,(INTERVAL '02' HOUR + NULL):interval hour,(INTERVAL '02' HOUR - NULL):interval hour,(NULL + INTERVAL '2' YEAR):interval year,(NULL - INTERVAL '2' YEAR):interval year,(NULL + INTERVAL '02' HOUR):interval hour,(NULL - INTERVAL '02' HOUR):interval hour> +-- !query output +NULL NULL NULL NULL NULL NULL NULL NULL + + +-- !query +select interval '2' year + '3-3' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "interval '2' year + '3-3'" + } ] +} + + +-- !query +select interval '2' year - '4' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR - 4)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year - '4'" + } ] +} + + +-- !query +select '4 11:11' - interval '4 22:12' day to minute +-- !query schema +struct<4 11:11 - INTERVAL '4 22:12' DAY TO MINUTE:string> +-- !query output +NULL + + +-- !query +select '4 12:12:12' + interval '4 22:12' day to minute +-- !query schema +struct<4 12:12:12 + INTERVAL '4 22:12' DAY TO MINUTE:string> +-- !query output +NULL + + +-- !query +create temporary view interval_view as select '1' str +-- !query schema +struct<> +-- !query output + + + +-- !query +select interval '2' year + str from interval_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR + str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year + str" + } ] +} + + +-- !query +select interval '2' year - str from interval_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '2' YEAR - str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' year - str" + } ] +} + + +-- !query +select str - interval '4 22:12' day to minute from interval_view +-- !query schema +struct +-- !query output +NULL + + +-- !query +select str + interval '4 22:12' day to minute from interval_view +-- !query schema +struct +-- !query output +NULL + + +-- !query +select interval '2-2' year to month + interval '3' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"", + "inputType" : "\"INTERVAL YEAR TO MONTH\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '2-2' year to month + interval '3' day" + } ] +} + + +-- !query +select interval '3' day + interval '2-2' year to month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"", + "inputType" : "\"INTERVAL YEAR TO MONTH\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '3' day + interval '2-2' year to month" + } ] +} + + +-- !query +select interval '2-2' year to month - interval '3' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"", + "inputType" : "\"INTERVAL YEAR TO MONTH\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + (- INTERVAL '3' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '2-2' year to month - interval '3' day" + } ] +} + + +-- !query +select interval '3' day - interval '2-2' year to month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"INTERVAL YEAR TO MONTH\"", + "sqlExpr" : "\"(INTERVAL '3' DAY - INTERVAL '2-2' YEAR TO MONTH)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "interval '3' day - interval '2-2' year to month" + } ] +} + + +-- !query +select 1 - interval '2' second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"1 + (- INTERVAL '02' SECOND)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "1 - interval '2' second" + } ] +} + + +-- !query +select 1 + interval '2' month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"INTERVAL MONTH\"", + "sqlExpr" : "\"(1 + INTERVAL '2' MONTH)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "1 + interval '2' month" + } ] +} + + +-- !query +select interval '2' second + 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"1 + INTERVAL '02' SECOND\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "interval '2' second + 1" + } ] +} + + +-- !query +select interval '2' month - 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL MONTH\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(INTERVAL '2' MONTH - 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "interval '2' month - 1" + } ] +} + + +-- !query +select interval '\t interval 1 day' +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select interval 'interval \t 1\tday' +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select interval 'interval\t1\tday' +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select interval '1\t' day +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select interval '1 ' day +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +select interval '2-2\t' year to month +-- !query schema +struct +-- !query output +2-2 + + +-- !query +select interval '-\t2-2\t' year to month +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING", + "sqlState" : "22006", + "messageParameters" : { + "input" : "-\t2-2\t", + "intervalStr" : "year-month", + "supportedFormat" : "`[+|-]y-m`, `INTERVAL [+|-]'[+|-]y-m' YEAR TO MONTH`", + "typeName" : "interval year to month" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 40, + "fragment" : "'-\\t2-2\\t' year to month" + } ] +} + + +-- !query +select interval '\n0 12:34:46.789\t' day to second +-- !query schema +struct +-- !query output +0 12:34:46.789000000 + + +-- !query +select interval '\n-\t10\t 12:34:46.789\t' day to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "\n-\t10\t 12:34:46.789\t", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]d h:m:s.n`, `INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND`", + "typeName" : "interval day to second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 56, + "fragment" : "'\\n-\\t10\\t 12:34:46.789\\t' day to second" + } ] +} + + +-- !query +select interval '中文 interval 1 day' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'中文 interval 1 day'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "interval '中文 interval 1 day'" + } ] +} + + +-- !query +select interval 'interval中文 1 day' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'interval中文 1 day'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "interval 'interval中文 1 day'" + } ] +} + + +-- !query +select interval 'interval 1中文day' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'interval 1中文day'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "interval 'interval 1中文day'" + } ] +} + + +-- !query +select -(a) from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITHOUT_SUGGESTION", + "sqlState" : "22015" +} + + +-- !query +select a - b from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITH_SUGGESTION", + "sqlState" : "22015", + "messageParameters" : { + "functionName" : "`try_subtract`" + } +} + + +-- !query +select b + interval '1 month' from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITH_SUGGESTION", + "sqlState" : "22015", + "messageParameters" : { + "functionName" : "`try_add`" + } +} + + +-- !query +select a * 1.1 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Overflow + + +-- !query +select a / 0.5 from values (interval '-2147483648 months', interval '2147483647 months') t(a, b) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Overflow + + +-- !query +SELECT + from_csv('1, 1 day', 'a INT, b interval'), + from_csv('1, 1', 'a INT, b interval day'), + to_csv(from_csv('1, 1 day', 'a INT, b interval')), + to_csv(from_csv('1, 1', 'a INT, b interval day')), + to_csv(named_struct('a', interval 32 hour, 'b', interval 70 minute)), + from_csv(to_csv(named_struct('a', interval 32 hour, 'b', interval 70 minute)), 'a interval hour, b interval minute') +-- !query schema +struct,from_csv(1, 1):struct,to_csv(from_csv(1, 1 day)):string,to_csv(from_csv(1, 1)):string,to_csv(named_struct(a, INTERVAL '32' HOUR, b, INTERVAL '70' MINUTE)):string,from_csv(to_csv(named_struct(a, INTERVAL '32' HOUR, b, INTERVAL '70' MINUTE))):struct> +-- !query output +{"a":1,"b":1 days} {"a":1,"b":1 00:00:00.000000000} 1,1 days 1,INTERVAL '1' DAY INTERVAL '32' HOUR,INTERVAL '70' MINUTE {"a":1 08:00:00.000000000,"b":0 01:10:00.000000000} + + +-- !query +SELECT + from_json('{"a":"1 days"}', 'a interval'), + from_csv('1, 1', 'a INT, b interval year'), + to_json(from_json('{"a":"1 days"}', 'a interval')), + to_csv(from_csv('1, 1', 'a INT, b interval year')), + to_csv(named_struct('a', interval 32 year, 'b', interval 10 month)), + from_csv(to_csv(named_struct('a', interval 32 year, 'b', interval 10 month)), 'a interval year, b interval month') +-- !query schema +struct,from_csv(1, 1):struct,to_json(from_json({"a":"1 days"})):string,to_csv(from_csv(1, 1)):string,to_csv(named_struct(a, INTERVAL '32' YEAR, b, INTERVAL '10' MONTH)):string,from_csv(to_csv(named_struct(a, INTERVAL '32' YEAR, b, INTERVAL '10' MONTH))):struct> +-- !query output +{"a":1 days} {"a":1,"b":1-0} {"a":"1 days"} 1,INTERVAL '1' YEAR INTERVAL '32' YEAR,INTERVAL '10' MONTH {"a":32-0,"b":0-10} + + +-- !query +SELECT + from_json('{"a":"1"}', 'a interval day'), + to_json(from_json('{"a":"1"}', 'a interval day')), + to_json(map('a', interval 100 day 130 minute)), + from_json(to_json(map('a', interval 100 day 130 minute)), 'a interval day to minute') +-- !query schema +struct,to_json(from_json({"a":"1"})):string,to_json(map(a, INTERVAL '100 02:10' DAY TO MINUTE)):string,from_json(to_json(map(a, INTERVAL '100 02:10' DAY TO MINUTE))):struct> +-- !query output +{"a":1 00:00:00.000000000} {"a":"INTERVAL '1' DAY"} {"a":"INTERVAL '100 02:10' DAY TO MINUTE"} {"a":100 02:10:00.000000000} + + +-- !query +SELECT + from_json('{"a":"1"}', 'a interval year'), + to_json(from_json('{"a":"1"}', 'a interval year')), + to_json(map('a', interval 32 year 10 month)), + from_json(to_json(map('a', interval 32 year 10 month)), 'a interval year to month') +-- !query schema +struct,to_json(from_json({"a":"1"})):string,to_json(map(a, INTERVAL '32-10' YEAR TO MONTH)):string,from_json(to_json(map(a, INTERVAL '32-10' YEAR TO MONTH))):struct> +-- !query output +{"a":1-0} {"a":"INTERVAL '1' YEAR"} {"a":"INTERVAL '32-10' YEAR TO MONTH"} {"a":32-10} + + +-- !query +select interval '+' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'+'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "interval '+'" + } ] +} + + +-- !query +select interval '+.' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'+.'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "interval '+.'" + } ] +} + + +-- !query +select interval '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "interval '1'" + } ] +} + + +-- !query +select interval '1.2' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1.2'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval '1.2'" + } ] +} + + +-- !query +select interval '- 2' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'- 2'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "interval '- 2'" + } ] +} + + +-- !query +select interval '1 day -' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1 day -'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "interval '1 day -'" + } ] +} + + +-- !query +select interval '1 day 1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1 day 1'", + "valueType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "interval '1 day 1'" + } ] +} + + +-- !query +select interval '1 day 2' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0026", + "messageParameters" : { + "value" : "1 day 2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 29, + "fragment" : "'1 day 2' day" + } ] +} + + +-- !query +select interval 'interval 1' day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0026", + "messageParameters" : { + "value" : "interval 1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 32, + "fragment" : "'interval 1' day" + } ] +} + + +-- !query +select interval '-\t 1' day +-- !query schema +struct +-- !query output +-1 00:00:00.000000000 + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / 2 +-- !query schema +struct<(INTERVAL '-178956970-8' YEAR TO MONTH / 2):interval year to month> +-- !query output +-89478485-4 + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / 5 +-- !query schema +struct<(INTERVAL '-178956970-8' YEAR TO MONTH / 5):interval year to month> +-- !query output +-35791394-2 + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITH_SUGGESTION", + "sqlState" : "22015", + "messageParameters" : { + "functionName" : "`try_divide`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "(INTERVAL '-178956970-8' YEAR TO MONTH) / -1" + } ] +} + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1L +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITH_SUGGESTION", + "sqlState" : "22015", + "messageParameters" : { + "functionName" : "`try_divide`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "(INTERVAL '-178956970-8' YEAR TO MONTH) / -1L" + } ] +} + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1.0 +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Overflow + + +-- !query +SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1.0D +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +rounded value is out of range for input 2.147483648E9 and rounding mode HALF_UP + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / 2 +-- !query schema +struct<(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / 2):interval day to second> +-- !query output +-53375995 14:00:27.387904000 + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / 5 +-- !query schema +struct<(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND / 5):interval day to second> +-- !query output +-21350398 05:36:10.955162000 + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITH_SUGGESTION", + "sqlState" : "22015", + "messageParameters" : { + "functionName" : "`try_divide`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 65, + "fragment" : "(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1" + } ] +} + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1L +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW.WITH_SUGGESTION", + "sqlState" : "22015", + "messageParameters" : { + "functionName" : "`try_divide`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 66, + "fragment" : "(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1L" + } ] +} + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1.0 +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Overflow + + +-- !query +SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1.0D +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +rounded value is out of range for input 9.223372036854776E18 and rounding mode HALF_UP + + +-- !query +SELECT INTERVAL '106751991 04' DAY TO HOUR +-- !query schema +struct +-- !query output +106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '106751991 04:00' DAY TO MINUTE +-- !query schema +struct +-- !query output +106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '106751991 04:00:54.775807' DAY TO SECOND +-- !query schema +struct +-- !query output +106751991 04:00:54.775807000 + + +-- !query +SELECT INTERVAL '2562047788:00' HOUR TO MINUTE +-- !query schema +struct +-- !query output +106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '2562047788:00:54.775807' HOUR TO SECOND +-- !query schema +struct +-- !query output +106751991 04:00:54.775807000 + + +-- !query +SELECT INTERVAL '153722867280:54.775807' MINUTE TO SECOND +-- !query schema +struct +-- !query output +106751991 04:00:54.775807000 + + +-- !query +SELECT INTERVAL '-106751991 04' DAY TO HOUR +-- !query schema +struct +-- !query output +-106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '-106751991 04:00' DAY TO MINUTE +-- !query schema +struct +-- !query output +-106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND +-- !query schema +struct +-- !query output +-106751991 04:00:54.775808000 + + +-- !query +SELECT INTERVAL '-2562047788:00' HOUR TO MINUTE +-- !query schema +struct +-- !query output +-106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '-2562047788:00:54.775808' HOUR TO SECOND +-- !query schema +struct +-- !query output +-106751991 04:00:54.775808000 + + +-- !query +SELECT INTERVAL '-153722867280:54.775808' MINUTE TO SECOND +-- !query schema +struct +-- !query output +-106751991 04:00:54.775808000 + + +-- !query +SELECT INTERVAL '106751992 04' DAY TO HOUR +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: day 106751992 outside range [0, 106751991]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 42, + "fragment" : "'106751992 04' DAY TO HOUR" + } ] +} + + +-- !query +SELECT INTERVAL '-106751992 04' DAY TO HOUR +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: day 106751992 outside range [0, 106751991]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 43, + "fragment" : "'-106751992 04' DAY TO HOUR" + } ] +} + + +-- !query +SELECT INTERVAL '2562047789:00' HOUR TO MINUTE +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: hour 2562047789 outside range [0, 2562047788]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 46, + "fragment" : "'2562047789:00' HOUR TO MINUTE" + } ] +} + + +-- !query +SELECT INTERVAL '-2562047789:00' HOUR TO MINUTE +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: hour 2562047789 outside range [0, 2562047788]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 47, + "fragment" : "'-2562047789:00' HOUR TO MINUTE" + } ] +} + + +-- !query +SELECT INTERVAL '153722867281:54.775808' MINUTE TO SECOND +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: minute 153722867281 outside range [0, 153722867280]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 57, + "fragment" : "'153722867281:54.775808' MINUTE TO SECOND" + } ] +} + + +-- !query +SELECT INTERVAL '-153722867281:54.775808' MINUTE TO SECOND +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0063", + "messageParameters" : { + "msg" : "requirement failed: minute 153722867281 outside range [0, 153722867280]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 58, + "fragment" : "'-153722867281:54.775808' MINUTE TO SECOND" + } ] +} + + +-- !query +SELECT INTERVAL '178956970' YEAR +-- !query schema +struct +-- !query output +178956970-0 + + +-- !query +SELECT INTERVAL '-178956970' YEAR +-- !query schema +struct +-- !query output +-178956970-0 + + +-- !query +SELECT INTERVAL '2147483647' MONTH +-- !query schema +struct +-- !query output +178956970-7 + + +-- !query +SELECT INTERVAL '-2147483647' MONTH +-- !query schema +struct +-- !query output +-178956970-7 + + +-- !query +SELECT INTERVAL '106751991' DAY +-- !query schema +struct +-- !query output +106751991 00:00:00.000000000 + + +-- !query +SELECT INTERVAL '-106751991' DAY +-- !query schema +struct +-- !query output +-106751991 00:00:00.000000000 + + +-- !query +SELECT INTERVAL '2562047788' HOUR +-- !query schema +struct +-- !query output +106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '-2562047788' HOUR +-- !query schema +struct +-- !query output +-106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '153722867280' MINUTE +-- !query schema +struct +-- !query output +106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '-153722867280' MINUTE +-- !query schema +struct +-- !query output +-106751991 04:00:00.000000000 + + +-- !query +SELECT INTERVAL '54.775807' SECOND +-- !query schema +struct +-- !query output +0 00:00:54.775807000 + + +-- !query +SELECT INTERVAL '-54.775807' SECOND +-- !query schema +struct +-- !query output +-0 00:00:54.775807000 + + +-- !query +SELECT INTERVAL '1' DAY > INTERVAL '1' HOUR +-- !query schema +struct<(INTERVAL '1' DAY > INTERVAL '01' HOUR):boolean> +-- !query output +true + + +-- !query +SELECT INTERVAL '1 02' DAY TO HOUR = INTERVAL '02:10:55' HOUR TO SECOND +-- !query schema +struct<(INTERVAL '1 02' DAY TO HOUR = INTERVAL '02:10:55' HOUR TO SECOND):boolean> +-- !query output +false + + +-- !query +SELECT INTERVAL '1' YEAR < INTERVAL '1' MONTH +-- !query schema +struct<(INTERVAL '1' YEAR < INTERVAL '1' MONTH):boolean> +-- !query output +false + + +-- !query +SELECT INTERVAL '-1-1' YEAR TO MONTH = INTERVAL '-13' MONTH +-- !query schema +struct<(INTERVAL '-1-1' YEAR TO MONTH = INTERVAL '-13' MONTH):boolean> +-- !query output +true + + +-- !query +SELECT INTERVAL 1 MONTH > INTERVAL 20 DAYS +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL MONTH\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(INTERVAL '1' MONTH > INTERVAL '20' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "INTERVAL 1 MONTH > INTERVAL 20 DAYS" + } ] +} + + +-- !query +SELECT INTERVAL '1' DAY < '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY < 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "INTERVAL '1' DAY < '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' DAY = '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY = 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "INTERVAL '1' DAY = '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' DAY > '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL DAY\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' DAY > 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "INTERVAL '1' DAY > '1'" + } ] +} + + +-- !query +SELECT '1' < INTERVAL '1' DAY +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 < INTERVAL '1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' < INTERVAL '1' DAY" + } ] +} + + +-- !query +SELECT '1' = INTERVAL '1' DAY +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 = INTERVAL '1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' = INTERVAL '1' DAY" + } ] +} + + +-- !query +SELECT '1' > INTERVAL '1' DAY +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(1 > INTERVAL '1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "'1' > INTERVAL '1' DAY" + } ] +} + + +-- !query +SELECT INTERVAL '1' YEAR < '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR < 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "INTERVAL '1' YEAR < '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' YEAR = '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR = 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "INTERVAL '1' YEAR = '1'" + } ] +} + + +-- !query +SELECT INTERVAL '1' YEAR > '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL YEAR\"", + "right" : "\"STRING\"", + "sqlExpr" : "\"(INTERVAL '1' YEAR > 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "INTERVAL '1' YEAR > '1'" + } ] +} + + +-- !query +SELECT '1' < INTERVAL '1' YEAR +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 < INTERVAL '1' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'1' < INTERVAL '1' YEAR" + } ] +} + + +-- !query +SELECT '1' = INTERVAL '1' YEAR +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 = INTERVAL '1' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'1' = INTERVAL '1' YEAR" + } ] +} + + +-- !query +SELECT '1' > INTERVAL '1' YEAR +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"STRING\"", + "right" : "\"INTERVAL YEAR\"", + "sqlExpr" : "\"(1 > INTERVAL '1' YEAR)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "'1' > INTERVAL '1' YEAR" + } ] +} + + +-- !query +SELECT array(INTERVAL '1' YEAR, INTERVAL '1' MONTH) +-- !query schema +struct> +-- !query output +[1-0,0-1] + + +-- !query +SELECT array(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE) +-- !query schema +struct> +-- !query output +[1 00:00:00.000000000,0 01:01:00.000000000] + + +-- !query +SELECT array(INTERVAL 1 MONTH, INTERVAL 20 DAYS) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"INTERVAL MONTH\" or \"INTERVAL DAY\")", + "functionName" : "`array`", + "sqlExpr" : "\"array(INTERVAL '1' MONTH, INTERVAL '20' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "array(INTERVAL 1 MONTH, INTERVAL 20 DAYS)" + } ] +} + + +-- !query +SELECT coalesce(INTERVAL '1' YEAR, INTERVAL '1' MONTH) +-- !query schema +struct +-- !query output +1-0 + + +-- !query +SELECT coalesce(INTERVAL '1' DAY, INTERVAL '01:01' HOUR TO MINUTE) +-- !query schema +struct +-- !query output +1 00:00:00.000000000 + + +-- !query +SELECT coalesce(INTERVAL 1 MONTH, INTERVAL 20 DAYS) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"INTERVAL MONTH\" or \"INTERVAL DAY\")", + "functionName" : "`coalesce`", + "sqlExpr" : "\"coalesce(INTERVAL '1' MONTH, INTERVAL '20' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "coalesce(INTERVAL 1 MONTH, INTERVAL 20 DAYS)" + } ] +} + + +-- !query +SELECT abs(INTERVAL '-10' YEAR) +-- !query schema +struct +-- !query output +10-0 + + +-- !query +SELECT abs(INTERVAL -'1 02:03:04.123' DAY TO SECOND) +-- !query schema +struct +-- !query output +1 02:03:04.123000000 + + +-- !query +SELECT div(INTERVAL '1-1' YEAR TO MONTH, INTERVAL '1' YEAR) +-- !query schema +struct<(INTERVAL '1-1' YEAR TO MONTH div INTERVAL '1' YEAR):bigint> +-- !query output +1 + + +-- !query +SELECT div(INTERVAL '1-1' YEAR TO MONTH, INTERVAL '-1' MONTH) +-- !query schema +struct<(INTERVAL '1-1' YEAR TO MONTH div INTERVAL '-1' MONTH):bigint> +-- !query output +-13 + + +-- !query +SELECT div(INTERVAL '1 06' DAY TO HOUR, INTERVAL '1' DAY) +-- !query schema +struct<(INTERVAL '1 06' DAY TO HOUR div INTERVAL '1' DAY):bigint> +-- !query output +1 + + +-- !query +SELECT div(INTERVAL '1 06' DAY TO HOUR, INTERVAL '-1' HOUR) +-- !query schema +struct<(INTERVAL '1 06' DAY TO HOUR div INTERVAL '-01' HOUR):bigint> +-- !query output +-30 + + +-- !query +SELECT div(INTERVAL '1' MONTH, INTERVAL '-1' DAY) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INTERVAL MONTH\"", + "right" : "\"INTERVAL DAY\"", + "sqlExpr" : "\"(INTERVAL '1' MONTH div INTERVAL '-1' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "div(INTERVAL '1' MONTH, INTERVAL '-1' DAY)" + } ] +} + + +-- !query +SELECT signum(INTERVAL '-10' YEAR) +-- !query schema +struct +-- !query output +-1.0 + + +-- !query +SELECT signum(INTERVAL '10' MONTH) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT signum(INTERVAL '0-0' YEAR TO MONTH) +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT signum(INTERVAL '-10' DAY) +-- !query schema +struct +-- !query output +-1.0 + + +-- !query +SELECT signum(INTERVAL '10' HOUR) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT signum(INTERVAL '0 0:0:0' DAY TO SECOND) +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT width_bucket(INTERVAL '0' YEAR, INTERVAL '0' YEAR, INTERVAL '10' YEAR, 10) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT width_bucket(INTERVAL '-1' YEAR, INTERVAL -'1-2' YEAR TO MONTH, INTERVAL '1-2' YEAR TO MONTH, 10) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT width_bucket(INTERVAL '0' DAY, INTERVAL '0' DAY, INTERVAL '10' DAY, 10) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT width_bucket(INTERVAL '-59' MINUTE, INTERVAL -'1 01' DAY TO HOUR, INTERVAL '1 2:3:4.001' DAY TO SECOND, 10) +-- !query schema +struct +-- !query output +5 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/keywords.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/keywords.sql.out new file mode 100644 index 000000000000..fb4c72a88cad --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/keywords.sql.out @@ -0,0 +1,391 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT * from SQL_KEYWORDS() +-- !query schema +struct +-- !query output +ADD false +AFTER false +AGGREGATE false +ALL false +ALTER false +ALWAYS false +ANALYZE false +AND false +ANTI false +ANY false +ANY_VALUE false +ARCHIVE false +ARRAY false +AS false +ASC false +AT false +ATOMIC false +AUTHORIZATION false +BEGIN false +BETWEEN false +BIGINT false +BINARY false +BINDING false +BOOLEAN false +BOTH false +BUCKET false +BUCKETS false +BY false +BYTE false +CACHE false +CALL false +CALLED false +CASCADE false +CASE false +CAST false +CATALOG false +CATALOGS false +CHANGE false +CHAR false +CHARACTER false +CHECK false +CLEAR false +CLUSTER false +CLUSTERED false +CODEGEN false +COLLATE false +COLLATION false +COLLECTION false +COLUMN false +COLUMNS false +COMMENT false +COMMIT false +COMPACT false +COMPACTIONS false +COMPENSATION false +COMPUTE false +CONCATENATE false +CONDITION false +CONSTRAINT false +CONTAINS false +CONTINUE false +COST false +CREATE false +CROSS false +CUBE false +CURRENT false +CURRENT_DATE false +CURRENT_TIME false +CURRENT_TIMESTAMP false +CURRENT_USER false +DATA false +DATABASE false +DATABASES false +DATE false +DATEADD false +DATEDIFF false +DATE_ADD false +DATE_DIFF false +DAY false +DAYOFYEAR false +DAYS false +DBPROPERTIES false +DEC false +DECIMAL false +DECLARE false +DEFAULT false +DEFINED false +DEFINER false +DELETE false +DELIMITED false +DESC false +DESCRIBE false +DETERMINISTIC false +DFS false +DIRECTORIES false +DIRECTORY false +DISTINCT false +DISTRIBUTE false +DIV false +DO false +DOUBLE false +DROP false +ELSE false +ELSEIF false +END false +ESCAPE false +ESCAPED false +EVOLUTION false +EXCEPT false +EXCHANGE false +EXCLUDE false +EXECUTE false +EXISTS false +EXIT false +EXPLAIN false +EXPORT false +EXTEND false +EXTENDED false +EXTERNAL false +EXTRACT false +FALSE false +FETCH false +FIELDS false +FILEFORMAT false +FILTER false +FIRST false +FLOAT false +FOLLOWING false +FOR false +FOREIGN false +FORMAT false +FORMATTED false +FOUND false +FROM false +FULL false +FUNCTION false +FUNCTIONS false +GENERATED false +GLOBAL false +GRANT false +GROUP false +GROUPING false +HANDLER false +HAVING false +HOUR false +HOURS false +IDENTIFIER false +IDENTITY false +IF false +IGNORE false +ILIKE false +IMMEDIATE false +IMPORT false +IN false +INCLUDE false +INCREMENT false +INDEX false +INDEXES false +INNER false +INPATH false +INPUT false +INPUTFORMAT false +INSERT false +INT false +INTEGER false +INTERSECT false +INTERVAL false +INTO false +INVOKER false +IS false +ITEMS false +ITERATE false +JOIN false +JSON false +KEYS false +LANGUAGE false +LAST false +LATERAL false +LAZY false +LEADING false +LEAVE false +LEFT false +LIKE false +LIMIT false +LINES false +LIST false +LOAD false +LOCAL false +LOCATION false +LOCK false +LOCKS false +LOGICAL false +LONG false +LOOP false +MACRO false +MAP false +MATCHED false +MERGE false +MICROSECOND false +MICROSECONDS false +MILLISECOND false +MILLISECONDS false +MINUS false +MINUTE false +MINUTES false +MODIFIES false +MONTH false +MONTHS false +MSCK false +NAME false +NAMESPACE false +NAMESPACES false +NANOSECOND false +NANOSECONDS false +NATURAL false +NO false +NONE false +NOT false +NULL false +NULLS false +NUMERIC false +OF false +OFFSET false +ON false +ONLY false +OPTION false +OPTIONS false +OR false +ORDER false +OUT false +OUTER false +OUTPUTFORMAT false +OVER false +OVERLAPS false +OVERLAY false +OVERWRITE false +PARTITION false +PARTITIONED false +PARTITIONS false +PERCENT false +PIVOT false +PLACING false +POSITION false +PRECEDING false +PRIMARY false +PRINCIPALS false +PROPERTIES false +PURGE false +QUARTER false +QUERY false +RANGE false +READS false +REAL false +RECORDREADER false +RECORDWRITER false +RECOVER false +RECURSIVE false +REDUCE false +REFERENCES false +REFRESH false +RENAME false +REPAIR false +REPEAT false +REPEATABLE false +REPLACE false +RESET false +RESPECT false +RESTRICT false +RETURN false +RETURNS false +REVOKE false +RIGHT false +ROLE false +ROLES false +ROLLBACK false +ROLLUP false +ROW false +ROWS false +SCHEMA false +SCHEMAS false +SECOND false +SECONDS false +SECURITY false +SELECT false +SEMI false +SEPARATED false +SERDE false +SERDEPROPERTIES false +SESSION_USER false +SET false +SETS false +SHORT false +SHOW false +SINGLE false +SKEWED false +SMALLINT false +SOME false +SORT false +SORTED false +SOURCE false +SPECIFIC false +SQL false +SQLEXCEPTION false +SQLSTATE false +START false +STATISTICS false +STORED false +STRATIFY false +STRING false +STRUCT false +SUBSTR false +SUBSTRING false +SYNC false +SYSTEM_TIME false +SYSTEM_VERSION false +TABLE false +TABLES false +TABLESAMPLE false +TARGET false +TBLPROPERTIES false +TERMINATED false +THEN false +TIME false +TIMEDIFF false +TIMESTAMP false +TIMESTAMPADD false +TIMESTAMPDIFF false +TIMESTAMP_LTZ false +TIMESTAMP_NTZ false +TINYINT false +TO false +TOUCH false +TRAILING false +TRANSACTION false +TRANSACTIONS false +TRANSFORM false +TRIM false +TRUE false +TRUNCATE false +TRY_CAST false +TYPE false +UNARCHIVE false +UNBOUNDED false +UNCACHE false +UNION false +UNIQUE false +UNKNOWN false +UNLOCK false +UNPIVOT false +UNSET false +UNTIL false +UPDATE false +USE false +USER false +USING false +VALUE false +VALUES false +VAR false +VARCHAR false +VARIABLE false +VARIANT false +VERSION false +VIEW false +VIEWS false +VOID false +WEEK false +WEEKS false +WHEN false +WHERE false +WHILE false +WINDOW false +WITH false +WITHIN false +X false +YEAR false +YEARS false +ZONE false + + +-- !query +SELECT keyword from SQL_KEYWORDS() WHERE reserved +-- !query schema +struct +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/literals.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/literals.sql.out new file mode 100644 index 000000000000..4e4c70cc333b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/literals.sql.out @@ -0,0 +1,787 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select null, Null, nUll +-- !query schema +struct +-- !query output +NULL NULL NULL + + +-- !query +select true, tRue, false, fALse +-- !query schema +struct +-- !query output +true true false false + + +-- !query +select 1Y +-- !query schema +struct<1:tinyint> +-- !query output +1 + + +-- !query +select 127Y, -128Y +-- !query schema +struct<127:tinyint,-128:tinyint> +-- !query output +127 -128 + + +-- !query +select 128Y +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_NUMERIC_LITERAL_RANGE", + "sqlState" : "22003", + "messageParameters" : { + "maxValue" : "127", + "minValue" : "-128", + "rawStrippedQualifier" : "128", + "typeName" : "tinyint" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 11, + "fragment" : "128Y" + } ] +} + + +-- !query +select 1S +-- !query schema +struct<1:smallint> +-- !query output +1 + + +-- !query +select 32767S, -32768S +-- !query schema +struct<32767:smallint,-32768:smallint> +-- !query output +32767 -32768 + + +-- !query +select 32768S +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_NUMERIC_LITERAL_RANGE", + "sqlState" : "22003", + "messageParameters" : { + "maxValue" : "32767", + "minValue" : "-32768", + "rawStrippedQualifier" : "32768", + "typeName" : "smallint" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : "32768S" + } ] +} + + +-- !query +select 1L, 2147483648L +-- !query schema +struct<1:bigint,2147483648:bigint> +-- !query output +1 2147483648 + + +-- !query +select 9223372036854775807L, -9223372036854775808L +-- !query schema +struct<9223372036854775807:bigint,-9223372036854775808:bigint> +-- !query output +9223372036854775807 -9223372036854775808 + + +-- !query +select 9223372036854775808L +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_NUMERIC_LITERAL_RANGE", + "sqlState" : "22003", + "messageParameters" : { + "maxValue" : "9223372036854775807", + "minValue" : "-9223372036854775808", + "rawStrippedQualifier" : "9223372036854775808", + "typeName" : "bigint" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "9223372036854775808L" + } ] +} + + +-- !query +select 1, -1 +-- !query schema +struct<1:int,-1:int> +-- !query output +1 -1 + + +-- !query +select 2147483647, -2147483648 +-- !query schema +struct<2147483647:int,-2147483648:int> +-- !query output +2147483647 -2147483648 + + +-- !query +select 9223372036854775807, -9223372036854775808 +-- !query schema +struct<9223372036854775807:bigint,-9223372036854775808:bigint> +-- !query output +9223372036854775807 -9223372036854775808 + + +-- !query +select 9223372036854775808, -9223372036854775809 +-- !query schema +struct<9223372036854775808:decimal(19,0),-9223372036854775809:decimal(19,0)> +-- !query output +9223372036854775808 -9223372036854775809 + + +-- !query +select 1234567890123456789012345678901234567890 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION", + "sqlState" : "22003", + "messageParameters" : { + "maxPrecision" : "38", + "precision" : "40" + } +} + + +-- !query +select 1234567890123456789012345678901234567890.0 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION", + "sqlState" : "22003", + "messageParameters" : { + "maxPrecision" : "38", + "precision" : "41" + } +} + + +-- !query +select 1F, 1.2F, .10f, 0.10f +-- !query schema +struct<1.0:float,1.2:float,0.1:float,0.1:float> +-- !query output +1.0 1.2 0.1 0.1 + + +-- !query +select -1F, -1.2F, -.10F, -0.10F +-- !query schema +struct<-1.0:float,-1.2:float,-0.1:float,-0.1:float> +-- !query output +-1.0 -1.2 -0.1 -0.1 + + +-- !query +select -3.4028235E39f +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_NUMERIC_LITERAL_RANGE", + "sqlState" : "22003", + "messageParameters" : { + "maxValue" : "3.4028234663852886E+38", + "minValue" : "-3.4028234663852886E+38", + "rawStrippedQualifier" : "-3.4028235E39", + "typeName" : "float" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "-3.4028235E39f" + } ] +} + + +-- !query +select 1D, 1.2D, 1e10, 1.5e5, .10D, 0.10D, .1e5, .9e+2, 0.9e+2, 900e-1, 9.e+1 +-- !query schema +struct<1.0:double,1.2:double,1.0E10:double,150000.0:double,0.1:double,0.1:double,10000.0:double,90.0:double,90.0:double,90.0:double,90.0:double> +-- !query output +1.0 1.2 1.0E10 150000.0 0.1 0.1 10000.0 90.0 90.0 90.0 90.0 + + +-- !query +select -1D, -1.2D, -1e10, -1.5e5, -.10D, -0.10D, -.1e5 +-- !query schema +struct<-1.0:double,-1.2:double,-1.0E10:double,-150000.0:double,-0.1:double,-0.1:double,-10000.0:double> +-- !query output +-1.0 -1.2 -1.0E10 -150000.0 -0.1 -0.1 -10000.0 + + +-- !query +select .e3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'.'", + "hint" : "" + } +} + + +-- !query +select 1E309, -1E309 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_NUMERIC_LITERAL_RANGE", + "sqlState" : "22003", + "messageParameters" : { + "maxValue" : "1.7976931348623157E+308", + "minValue" : "-1.7976931348623157E+308", + "rawStrippedQualifier" : "1E309", + "typeName" : "double" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "1E309" + } ] +} + + +-- !query +select 0.3, -0.8, .5, -.18, 0.1111, .1111 +-- !query schema +struct<0.3:decimal(1,1),-0.8:decimal(1,1),0.5:decimal(1,1),-0.18:decimal(2,2),0.1111:decimal(4,4),0.1111:decimal(4,4)> +-- !query output +0.3 -0.8 0.5 -0.18 0.1111 0.1111 + + +-- !query +select 0.3 F, 0.4 D, 0.5 BD +-- !query schema +struct +-- !query output +0.3 0.4 0.5 + + +-- !query +select 123456789012345678901234567890123456789e10d, 123456789012345678901234567890123456789.1e10d +-- !query schema +struct<1.2345678901234568E48:double,1.2345678901234568E48:double> +-- !query output +1.2345678901234568E48 1.2345678901234568E48 + + +-- !query +select "Hello Peter!", 'hello lee!' +-- !query schema +struct +-- !query output +Hello Peter! hello lee! + + +-- !query +select 'hello' 'world', 'hello' " " 'lee' +-- !query schema +struct +-- !query output +helloworld hello lee + + +-- !query +select "hello 'peter'" +-- !query schema +struct +-- !query output +hello 'peter' + + +-- !query +select 'pattern%', 'no-pattern\%', 'pattern\\%', 'pattern\\\%' +-- !query schema +struct +-- !query output +pattern% no-pattern\% pattern\% pattern\\% + + +-- !query +select '\'', '"', '\n', '\r', '\t', 'Z' +-- !query schema +struct<':string,":string, +:string, :string, :string,Z:string> +-- !query output +' " + Z + + +-- !query +select '\110\145\154\154\157\041' +-- !query schema +struct +-- !query output +Hello! + + +-- !query +select '\u0057\u006F\u0072\u006C\u0064\u0020\u003A\u0029' +-- !query schema +struct +-- !query output +World :) + + +-- !query +select dAte '2016-03-12' +-- !query schema +struct +-- !query output +2016-03-12 + + +-- !query +select date 'mar 11 2016' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'mar 11 2016'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date 'mar 11 2016'" + } ] +} + + +-- !query +select tImEstAmp '2016-03-11 20:54:00.000' +-- !query schema +struct +-- !query output +2016-03-11 20:54:00 + + +-- !query +select timestamp '2016-33-11 20:54:00.000' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2016-33-11 20:54:00.000'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "timestamp '2016-33-11 20:54:00.000'" + } ] +} + + +-- !query +select GEO '(10,-6)' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_TYPED_LITERAL", + "sqlState" : "0A000", + "messageParameters" : { + "supportedTypes" : "\"DATE\", \"TIMESTAMP_NTZ\", \"TIMESTAMP_LTZ\", \"TIMESTAMP\", \"INTERVAL\", \"X\"", + "unsupportedType" : "\"GEO\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "GEO '(10,-6)'" + } ] +} + + +-- !query +select 90912830918230182310293801923652346786BD, 123.0E-28BD, 123.08BD +-- !query schema +struct<90912830918230182310293801923652346786:decimal(38,0),1.230E-26:decimal(29,29),123.08:decimal(5,2)> +-- !query output +90912830918230182310293801923652346786 0.00000000000000000000000001230 123.08 + + +-- !query +select 1.20E-38BD +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION", + "sqlState" : "22003", + "messageParameters" : { + "maxPrecision" : "38", + "precision" : "40" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "1.20E-38BD" + } ] +} + + +-- !query +select x'2379ACFe' +-- !query schema +struct +-- !query output +#y�� + + +-- !query +select X'XuZ' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'XuZ'", + "valueType" : "\"X\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : "X'XuZ'" + } ] +} + + +-- !query +SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 +-- !query schema +struct<3.14:decimal(3,2),-3.14:decimal(3,2),3.14E8:double,3.14E-8:double,-3.14E8:double,-3.14E-8:double,3.14E8:double,3.14E8:double,3.14E-8:double> +-- !query output +3.14 -3.14 3.14E8 3.14E-8 -3.14E8 -3.14E-8 3.14E8 3.14E8 3.14E-8 + + +-- !query +select +date '1999-01-01' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"DATE '1999-01-01'\"", + "inputType" : "\"DATE\"", + "paramIndex" : "first", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ DATE '1999-01-01')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "+date '1999-01-01'" + } ] +} + + +-- !query +select +timestamp '1999-01-01' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"TIMESTAMP '1999-01-01 00:00:00'\"", + "inputType" : "\"TIMESTAMP\"", + "paramIndex" : "first", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ TIMESTAMP '1999-01-01 00:00:00')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "+timestamp '1999-01-01'" + } ] +} + + +-- !query +select +interval '1 day' +-- !query schema +struct<(+ INTERVAL '1' DAY):interval day> +-- !query output +1 00:00:00.000000000 + + +-- !query +select +map(1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"map(1, 2)\"", + "inputType" : "\"MAP\"", + "paramIndex" : "first", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ map(1, 2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "+map(1, 2)" + } ] +} + + +-- !query +select +array(1,2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"array(1, 2)\"", + "inputType" : "\"ARRAY\"", + "paramIndex" : "first", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ array(1, 2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "+array(1,2)" + } ] +} + + +-- !query +select +named_struct('a', 1, 'b', 'spark') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"named_struct(a, 1, b, spark)\"", + "inputType" : "\"STRUCT\"", + "paramIndex" : "first", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ named_struct(a, 1, b, spark))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "+named_struct('a', 1, 'b', 'spark')" + } ] +} + + +-- !query +select +X'1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"X'01'\"", + "inputType" : "\"BINARY\"", + "paramIndex" : "first", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(+ X'01')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "+X'1'" + } ] +} + + +-- !query +select -date '1999-01-01' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"DATE '1999-01-01'\"", + "inputType" : "\"DATE\"", + "paramIndex" : "first", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(- DATE '1999-01-01')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "-date '1999-01-01'" + } ] +} + + +-- !query +select -timestamp '1999-01-01' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"TIMESTAMP '1999-01-01 00:00:00'\"", + "inputType" : "\"TIMESTAMP\"", + "paramIndex" : "first", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(- TIMESTAMP '1999-01-01 00:00:00')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "-timestamp '1999-01-01'" + } ] +} + + +-- !query +select -x'2379ACFe' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"X'2379ACFE'\"", + "inputType" : "\"BINARY\"", + "paramIndex" : "first", + "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(- X'2379ACFE')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "-x'2379ACFe'" + } ] +} + + +-- !query +select -0, -0.0 +-- !query schema +struct<0:int,0.0:decimal(1,1)> +-- !query output +0 0.0 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/map.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/map.sql.out new file mode 100644 index 000000000000..71f5491287c3 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/map.sql.out @@ -0,0 +1,115 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select element_at(map(1, 'a', 2, 'b'), 5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select map(1, 'a', 2, 'b')[5] +-- !query schema +struct +-- !query output +NULL + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), 5) +-- !query schema +struct +-- !query output +false + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), 1) +-- !query schema +struct +-- !query output +true + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), 5.0) +-- !query schema +struct +-- !query output +false + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), 1.0) +-- !query schema +struct +-- !query output +true + + +-- !query +select map_contains_key(map(1.0, 'a', 2, 'b'), 5) +-- !query schema +struct +-- !query output +false + + +-- !query +select map_contains_key(map(1.0, 'a', 2, 'b'), 1) +-- !query schema +struct +-- !query output +true + + +-- !query +select map_contains_key(map('1', 'a', '2', 'b'), 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "\"MAP\"", + "functionName" : "`map_contains_key`", + "leftType" : "\"MAP\"", + "rightType" : "\"INT\"", + "sqlExpr" : "\"map_contains_key(map(1, a, 2, b), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "map_contains_key(map('1', 'a', '2', 'b'), 1)" + } ] +} + + +-- !query +select map_contains_key(map(1, 'a', 2, 'b'), '1') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "\"MAP\"", + "functionName" : "`map_contains_key`", + "leftType" : "\"MAP\"", + "rightType" : "\"STRING\"", + "sqlExpr" : "\"map_contains_key(map(1, a, 2, b), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "map_contains_key(map(1, 'a', 2, 'b'), '1')" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/math.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/math.sql.out new file mode 100644 index 000000000000..09f438393328 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/math.sql.out @@ -0,0 +1,583 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT round(25y, 1) +-- !query schema +struct +-- !query output +25 + + +-- !query +SELECT round(25y, 0) +-- !query schema +struct +-- !query output +25 + + +-- !query +SELECT round(25y, -1) +-- !query schema +struct +-- !query output +30 + + +-- !query +SELECT round(25y, -2) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT round(25y, -3) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT round(127y, -1) +-- !query schema +struct +-- !query output +-126 + + +-- !query +SELECT round(-128y, -1) +-- !query schema +struct +-- !query output +126 + + +-- !query +SELECT round(525s, 1) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT round(525s, 0) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT round(525s, -1) +-- !query schema +struct +-- !query output +530 + + +-- !query +SELECT round(525s, -2) +-- !query schema +struct +-- !query output +500 + + +-- !query +SELECT round(525s, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT round(32767s, -1) +-- !query schema +struct +-- !query output +-32766 + + +-- !query +SELECT round(-32768s, -1) +-- !query schema +struct +-- !query output +32766 + + +-- !query +SELECT round(525, 1) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT round(525, 0) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT round(525, -1) +-- !query schema +struct +-- !query output +530 + + +-- !query +SELECT round(525, -2) +-- !query schema +struct +-- !query output +500 + + +-- !query +SELECT round(525, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT round(2147483647, -1) +-- !query schema +struct +-- !query output +-2147483646 + + +-- !query +SELECT round(-2147483647, -1) +-- !query schema +struct +-- !query output +2147483646 + + +-- !query +SELECT round(525L, 1) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT round(525L, 0) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT round(525L, -1) +-- !query schema +struct +-- !query output +530 + + +-- !query +SELECT round(525L, -2) +-- !query schema +struct +-- !query output +500 + + +-- !query +SELECT round(525L, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT round(9223372036854775807L, -1) +-- !query schema +struct +-- !query output +-9223372036854775806 + + +-- !query +SELECT round(-9223372036854775808L, -1) +-- !query schema +struct +-- !query output +9223372036854775806 + + +-- !query +SELECT bround(25y, 1) +-- !query schema +struct +-- !query output +25 + + +-- !query +SELECT bround(25y, 0) +-- !query schema +struct +-- !query output +25 + + +-- !query +SELECT bround(25y, -1) +-- !query schema +struct +-- !query output +20 + + +-- !query +SELECT bround(25y, -2) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT bround(25y, -3) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT bround(127y, -1) +-- !query schema +struct +-- !query output +-126 + + +-- !query +SELECT bround(-128y, -1) +-- !query schema +struct +-- !query output +126 + + +-- !query +SELECT bround(525s, 1) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT bround(525s, 0) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT bround(525s, -1) +-- !query schema +struct +-- !query output +520 + + +-- !query +SELECT bround(525s, -2) +-- !query schema +struct +-- !query output +500 + + +-- !query +SELECT bround(525s, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT bround(32767s, -1) +-- !query schema +struct +-- !query output +-32766 + + +-- !query +SELECT bround(-32768s, -1) +-- !query schema +struct +-- !query output +32766 + + +-- !query +SELECT bround(525, 1) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT bround(525, 0) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT bround(525, -1) +-- !query schema +struct +-- !query output +520 + + +-- !query +SELECT bround(525, -2) +-- !query schema +struct +-- !query output +500 + + +-- !query +SELECT bround(525, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT bround(2147483647, -1) +-- !query schema +struct +-- !query output +-2147483646 + + +-- !query +SELECT bround(-2147483647, -1) +-- !query schema +struct +-- !query output +2147483646 + + +-- !query +SELECT bround(525L, 1) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT bround(525L, 0) +-- !query schema +struct +-- !query output +525 + + +-- !query +SELECT bround(525L, -1) +-- !query schema +struct +-- !query output +520 + + +-- !query +SELECT bround(525L, -2) +-- !query schema +struct +-- !query output +500 + + +-- !query +SELECT bround(525L, -3) +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT bround(9223372036854775807L, -1) +-- !query schema +struct +-- !query output +-9223372036854775806 + + +-- !query +SELECT bround(-9223372036854775808L, -1) +-- !query schema +struct +-- !query output +9223372036854775806 + + +-- !query +SELECT conv('100', 2, 10) +-- !query schema +struct +-- !query output +4 + + +-- !query +SELECT conv(-10, 16, -10) +-- !query schema +struct +-- !query output +-16 + + +-- !query +SELECT conv('9223372036854775808', 10, 16) +-- !query schema +struct +-- !query output +8000000000000000 + + +-- !query +SELECT conv('92233720368547758070', 10, 16) +-- !query schema +struct +-- !query output +FFFFFFFFFFFFFFFF + + +-- !query +SELECT conv('9223372036854775807', 36, 10) +-- !query schema +struct +-- !query output +18446744073709551615 + + +-- !query +SELECT conv('-9223372036854775807', 36, 10) +-- !query schema +struct +-- !query output +18446744073709551615 + + +-- !query +SELECT BIN(0) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT BIN(25) +-- !query schema +struct +-- !query output +11001 + + +-- !query +SELECT BIN(25L) +-- !query schema +struct +-- !query output +11001 + + +-- !query +SELECT BIN(25.5) +-- !query schema +struct +-- !query output +11001 + + +-- !query +SELECT POSITIVE(0Y) +-- !query schema +struct<(+ 0):tinyint> +-- !query output +0 + + +-- !query +SELECT POSITIVE(25) +-- !query schema +struct<(+ 25):int> +-- !query output +25 + + +-- !query +SELECT POSITIVE(-25L) +-- !query schema +struct<(+ -25):bigint> +-- !query output +-25 + + +-- !query +SELECT POSITIVE(25.5) +-- !query schema +struct<(+ 25.5):decimal(3,1)> +-- !query output +25.5 + + +-- !query +SELECT POSITIVE("25.5") +-- !query schema +struct<(+ 25.5):double> +-- !query output +25.5 + + +-- !query +SELECT POSITIVE("invalid") +-- !query schema +struct<(+ invalid):double> +-- !query output +NULL + + +-- !query +SELECT POSITIVE(null) +-- !query schema +struct<(+ NULL):double> +-- !query output +NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/parse-schema-string.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/parse-schema-string.sql.out new file mode 100644 index 000000000000..793b545b3241 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/parse-schema-string.sql.out @@ -0,0 +1,31 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select from_csv('1', 'create INT') +-- !query schema +struct> +-- !query output +{"create":1} + + +-- !query +select from_csv('1', 'cube INT') +-- !query schema +struct> +-- !query output +{"cube":1} + + +-- !query +select from_json('{"create":1}', 'create INT') +-- !query schema +struct> +-- !query output +{"create":1} + + +-- !query +select from_json('{"cube":1}', 'cube INT') +-- !query schema +struct> +-- !query output +{"cube":1} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/string-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/string-functions.sql.out new file mode 100644 index 000000000000..d28ce4645ef6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/string-functions.sql.out @@ -0,0 +1,2279 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select concat_ws() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 0", + "functionName" : "`concat_ws`" + } +} + + +-- !query +select format_string() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 0", + "functionName" : "`format_string`" + } +} + + +-- !query +select 'a' || 'b' || 'c' +-- !query schema +struct +-- !query output +abc + + +-- !query +select replace('abc', 'b', '123') +-- !query schema +struct +-- !query output +a123c + + +-- !query +select replace('abc', 'b') +-- !query schema +struct +-- !query output +ac + + +-- !query +select length(uuid()), (uuid() <> uuid()) +-- !query schema +struct +-- !query output +36 true + + +-- !query +select position('bar' in 'foobarbar'), position(null, 'foobarbar'), position('aaads', null) +-- !query schema +struct +-- !query output +4 NULL NULL + + +-- !query +select left("abcd", 2), left("abcd", 5), left("abcd", '2'), left("abcd", null) +-- !query schema +struct +-- !query output +ab abcd ab NULL + + +-- !query +select left(null, -2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select left("abcd", -2), left("abcd", 0), left("abcd", 'a') +-- !query schema +struct +-- !query output + NULL + + +-- !query +select right("abcd", 2), right("abcd", 5), right("abcd", '2'), right("abcd", null) +-- !query schema +struct +-- !query output +cd abcd cd NULL + + +-- !query +select right(null, -2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select right("abcd", -2), right("abcd", 0), right("abcd", 'a') +-- !query schema +struct +-- !query output + NULL + + +-- !query +SELECT split('aa1cc2ee3', '[1-9]+') +-- !query schema +struct> +-- !query output +["aa","cc","ee",""] + + +-- !query +SELECT split('aa1cc2ee3', '[1-9]+', 2) +-- !query schema +struct> +-- !query output +["aa","cc2ee3"] + + +-- !query +SELECT split('hello', '') +-- !query schema +struct> +-- !query output +["h","e","l","l","o"] + + +-- !query +SELECT split('', '') +-- !query schema +struct> +-- !query output +[""] + + +-- !query +SELECT split('abc', null) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +SELECT split(null, 'b') +-- !query schema +struct> +-- !query output +NULL + + +-- !query +SELECT split_part('11.12.13', '.', 2) +-- !query schema +struct +-- !query output +12 + + +-- !query +SELECT split_part('11.12.13', '.', -1) +-- !query schema +struct +-- !query output +13 + + +-- !query +SELECT split_part('11.12.13', '.', -3) +-- !query schema +struct +-- !query output +11 + + +-- !query +SELECT split_part('11.12.13', '', 1) +-- !query schema +struct +-- !query output +11.12.13 + + +-- !query +SELECT split_part('11ab12ab13', 'ab', 1) +-- !query schema +struct +-- !query output +11 + + +-- !query +SELECT split_part('11.12.13', '.', 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_INDEX_OF_ZERO", + "sqlState" : "22003" +} + + +-- !query +SELECT split_part('11.12.13', '.', 4) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT split_part('11.12.13', '.', 5) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT split_part('11.12.13', '.', -5) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT split_part(null, '.', 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT split_part(str, delimiter, partNum) FROM VALUES ('11.12.13', '.', 3) AS v1(str, delimiter, partNum) +-- !query schema +struct +-- !query output +13 + + +-- !query +SELECT substr('Spark SQL', 5) +-- !query schema +struct +-- !query output +k SQL + + +-- !query +SELECT substr('Spark SQL', -3) +-- !query schema +struct +-- !query output +SQL + + +-- !query +SELECT substr('Spark SQL', 5, 1) +-- !query schema +struct +-- !query output +k + + +-- !query +SELECT substr('Spark SQL' from 5) +-- !query schema +struct +-- !query output +k SQL + + +-- !query +SELECT substr('Spark SQL' from -3) +-- !query schema +struct +-- !query output +SQL + + +-- !query +SELECT substr('Spark SQL' from 5 for 1) +-- !query schema +struct +-- !query output +k + + +-- !query +SELECT substring('Spark SQL', 5) +-- !query schema +struct +-- !query output +k SQL + + +-- !query +SELECT substring('Spark SQL', -3) +-- !query schema +struct +-- !query output +SQL + + +-- !query +SELECT substring('Spark SQL', 5, 1) +-- !query schema +struct +-- !query output +k + + +-- !query +SELECT substring('Spark SQL' from 5) +-- !query schema +struct +-- !query output +k SQL + + +-- !query +SELECT substring('Spark SQL' from -3) +-- !query schema +struct +-- !query output +SQL + + +-- !query +SELECT substring('Spark SQL' from 5 for 1) +-- !query schema +struct +-- !query output +k + + +-- !query +SELECT trim(" xyz "), ltrim(" xyz "), rtrim(" xyz ") +-- !query schema +struct +-- !query output +xyz xyz xyz + + +-- !query +SELECT trim(BOTH 'xyz' FROM 'yxTomxx'), trim('xyz' FROM 'yxTomxx') +-- !query schema +struct +-- !query output +Tom Tom + + +-- !query +SELECT trim(BOTH 'x' FROM 'xxxbarxxx'), trim('x' FROM 'xxxbarxxx') +-- !query schema +struct +-- !query output +bar bar + + +-- !query +SELECT trim(LEADING 'xyz' FROM 'zzzytest') +-- !query schema +struct +-- !query output +test + + +-- !query +SELECT trim(LEADING 'xyz' FROM 'zzzytestxyz') +-- !query schema +struct +-- !query output +testxyz + + +-- !query +SELECT trim(LEADING 'xy' FROM 'xyxXxyLAST WORD') +-- !query schema +struct +-- !query output +XxyLAST WORD + + +-- !query +SELECT trim(TRAILING 'xyz' FROM 'testxxzx') +-- !query schema +struct +-- !query output +test + + +-- !query +SELECT trim(TRAILING 'xyz' FROM 'xyztestxxzx') +-- !query schema +struct +-- !query output +xyztest + + +-- !query +SELECT trim(TRAILING 'xy' FROM 'TURNERyxXxy') +-- !query schema +struct +-- !query output +TURNERyxX + + +-- !query +SELECT btrim('xyxtrimyyx', 'xy') +-- !query schema +struct +-- !query output +trim + + +-- !query +SELECT btrim(encode(" xyz ", 'utf-8')) +-- !query schema +struct +-- !query output +xyz + + +-- !query +SELECT btrim(encode('yxTomxx', 'utf-8'), encode('xyz', 'utf-8')) +-- !query schema +struct +-- !query output +Tom + + +-- !query +SELECT btrim(encode('xxxbarxxx', 'utf-8'), encode('x', 'utf-8')) +-- !query schema +struct +-- !query output +bar + + +-- !query +SELECT lpad('hi', 'invalid_length') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT rpad('hi', 'invalid_length') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT hex(lpad(unhex(''), 5)) +-- !query schema +struct +-- !query output +0000000000 + + +-- !query +SELECT hex(lpad(unhex('aabb'), 5)) +-- !query schema +struct +-- !query output +000000AABB + + +-- !query +SELECT hex(lpad(unhex('aabbcc'), 2)) +-- !query schema +struct +-- !query output +AABB + + +-- !query +SELECT hex(lpad(unhex('123'), 2)) +-- !query schema +struct +-- !query output +0123 + + +-- !query +SELECT hex(lpad(unhex('12345'), 2)) +-- !query schema +struct +-- !query output +0123 + + +-- !query +SELECT hex(lpad(unhex(''), 5, unhex('1f'))) +-- !query schema +struct +-- !query output +1F1F1F1F1F + + +-- !query +SELECT hex(lpad(unhex('aa'), 5, unhex('1f'))) +-- !query schema +struct +-- !query output +1F1F1F1FAA + + +-- !query +SELECT hex(lpad(unhex('aa'), 6, unhex('1f'))) +-- !query schema +struct +-- !query output +1F1F1F1F1FAA + + +-- !query +SELECT hex(lpad(unhex(''), 5, unhex('1f2e'))) +-- !query schema +struct +-- !query output +1F2E1F2E1F + + +-- !query +SELECT hex(lpad(unhex('aa'), 5, unhex('1f2e'))) +-- !query schema +struct +-- !query output +1F2E1F2EAA + + +-- !query +SELECT hex(lpad(unhex('aa'), 6, unhex('1f2e'))) +-- !query schema +struct +-- !query output +1F2E1F2E1FAA + + +-- !query +SELECT hex(lpad(unhex(''), 6, unhex(''))) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT hex(lpad(unhex('aabbcc'), 6, unhex(''))) +-- !query schema +struct +-- !query output +AABBCC + + +-- !query +SELECT hex(lpad(unhex('aabbcc'), 2, unhex('ff'))) +-- !query schema +struct +-- !query output +AABB + + +-- !query +SELECT hex(rpad(unhex(''), 5)) +-- !query schema +struct +-- !query output +0000000000 + + +-- !query +SELECT hex(rpad(unhex('aabb'), 5)) +-- !query schema +struct +-- !query output +AABB000000 + + +-- !query +SELECT hex(rpad(unhex('aabbcc'), 2)) +-- !query schema +struct +-- !query output +AABB + + +-- !query +SELECT hex(rpad(unhex('123'), 2)) +-- !query schema +struct +-- !query output +0123 + + +-- !query +SELECT hex(rpad(unhex('12345'), 2)) +-- !query schema +struct +-- !query output +0123 + + +-- !query +SELECT hex(rpad(unhex(''), 5, unhex('1f'))) +-- !query schema +struct +-- !query output +1F1F1F1F1F + + +-- !query +SELECT hex(rpad(unhex('aa'), 5, unhex('1f'))) +-- !query schema +struct +-- !query output +AA1F1F1F1F + + +-- !query +SELECT hex(rpad(unhex('aa'), 6, unhex('1f'))) +-- !query schema +struct +-- !query output +AA1F1F1F1F1F + + +-- !query +SELECT hex(rpad(unhex(''), 5, unhex('1f2e'))) +-- !query schema +struct +-- !query output +1F2E1F2E1F + + +-- !query +SELECT hex(rpad(unhex('aa'), 5, unhex('1f2e'))) +-- !query schema +struct +-- !query output +AA1F2E1F2E + + +-- !query +SELECT hex(rpad(unhex('aa'), 6, unhex('1f2e'))) +-- !query schema +struct +-- !query output +AA1F2E1F2E1F + + +-- !query +SELECT hex(rpad(unhex(''), 6, unhex(''))) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT hex(rpad(unhex('aabbcc'), 6, unhex(''))) +-- !query schema +struct +-- !query output +AABBCC + + +-- !query +SELECT hex(rpad(unhex('aabbcc'), 2, unhex('ff'))) +-- !query schema +struct +-- !query output +AABB + + +-- !query +SELECT lpad('abc', 5, x'57') +-- !query schema +struct +-- !query output +WWabc + + +-- !query +SELECT lpad(x'57', 5, 'abc') +-- !query schema +struct +-- !query output +abcaW + + +-- !query +SELECT rpad('abc', 5, x'57') +-- !query schema +struct +-- !query output +abcWW + + +-- !query +SELECT rpad(x'57', 5, 'abc') +-- !query schema +struct +-- !query output +Wabca + + +-- !query +set spark.sql.legacy.javaCharsets=true +-- !query schema +struct +-- !query output +spark.sql.legacy.javaCharsets true + + +-- !query +select encode('hello', 'WINDOWS-1252') +-- !query schema +struct +-- !query output +hello + + +-- !query +select encode(scol, ecol) from values('hello', 'WINDOWS-1252') as t(scol, ecol) +-- !query schema +struct +-- !query output +hello + + +-- !query +set spark.sql.legacy.javaCharsets=false +-- !query schema +struct +-- !query output +spark.sql.legacy.javaCharsets false + + +-- !query +select encode('hello', 'WINDOWS-1252') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.CHARSET", + "sqlState" : "22023", + "messageParameters" : { + "charset" : "WINDOWS-1252", + "charsets" : "iso-8859-1, us-ascii, utf-16, utf-16be, utf-16le, utf-32, utf-8", + "functionName" : "`encode`", + "parameter" : "`charset`" + } +} + + +-- !query +select encode(scol, ecol) from values('hello', 'WINDOWS-1252') as t(scol, ecol) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.CHARSET", + "sqlState" : "22023", + "messageParameters" : { + "charset" : "WINDOWS-1252", + "charsets" : "iso-8859-1, us-ascii, utf-16, utf-16be, utf-16le, utf-32, utf-8", + "functionName" : "`encode`", + "parameter" : "`charset`" + } +} + + +-- !query +select encode('hello', 'Windows-xxx') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.CHARSET", + "sqlState" : "22023", + "messageParameters" : { + "charset" : "Windows-xxx", + "charsets" : "iso-8859-1, us-ascii, utf-16, utf-16be, utf-16le, utf-32, utf-8", + "functionName" : "`encode`", + "parameter" : "`charset`" + } +} + + +-- !query +select encode(scol, ecol) from values('hello', 'Windows-xxx') as t(scol, ecol) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.CHARSET", + "sqlState" : "22023", + "messageParameters" : { + "charset" : "Windows-xxx", + "charsets" : "iso-8859-1, us-ascii, utf-16, utf-16be, utf-16le, utf-32, utf-8", + "functionName" : "`encode`", + "parameter" : "`charset`" + } +} + + +-- !query +set spark.sql.legacy.codingErrorAction=true +-- !query schema +struct +-- !query output +spark.sql.legacy.codingErrorAction true + + +-- !query +select encode('渭城朝雨浥轻尘', 'US-ASCII') +-- !query schema +struct +-- !query output +??????? + + +-- !query +select encode(scol, ecol) from values('渭城朝雨浥轻尘', 'US-ASCII') as t(scol, ecol) +-- !query schema +struct +-- !query output +??????? + + +-- !query +set spark.sql.legacy.codingErrorAction=false +-- !query schema +struct +-- !query output +spark.sql.legacy.codingErrorAction false + + +-- !query +select encode('客舍青青柳色新', 'US-ASCII') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "MALFORMED_CHARACTER_CODING", + "sqlState" : "22000", + "messageParameters" : { + "charset" : "US-ASCII", + "function" : "`encode`" + } +} + + +-- !query +select encode(scol, ecol) from values('客舍青青柳色新', 'US-ASCII') as t(scol, ecol) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "MALFORMED_CHARACTER_CODING", + "sqlState" : "22000", + "messageParameters" : { + "charset" : "US-ASCII", + "function" : "`encode`" + } +} + + +-- !query +select encode(decode(encode('白日依山尽,黄河入海流。欲穷千里目,更上一层楼。', 'UTF-16'), 'UTF-16'), 'UTF-8') +-- !query schema +struct +-- !query output +白日依山尽,黄河入海流。欲穷千里目,更上一层楼。 + + +-- !query +select encode(decode(encode('南山經之首曰䧿山。其首曰招搖之山,臨於西海之上。', 'UTF-16'), 'UTF-16'), 'UTF-8') +-- !query schema +struct +-- !query output +南山經之首曰䧿山。其首曰招搖之山,臨於西海之上。 + + +-- !query +select encode(decode(encode('세계에서 가장 인기 있는 빅데이터 처리 프레임워크인 Spark', 'UTF-16'), 'UTF-16'), 'UTF-8') +-- !query schema +struct +-- !query output +세계에서 가장 인기 있는 빅데이터 처리 프레임워크인 Spark + + +-- !query +select encode(decode(encode('το Spark είναι το πιο δημοφιλές πλαίσιο επεξεργασίας μεγάλων δεδομένων παγκοσμίως', 'UTF-16'), 'UTF-16'), 'UTF-8') +-- !query schema +struct +-- !query output +το Spark είναι το πιο δημοφιλές πλαίσιο επεξεργασίας μεγάλων δεδομένων παγκοσμίως + + +-- !query +select encode(decode(encode('Sparkは世界で最も人気のあるビッグデータ処理フレームワークである。', 'UTF-16'), 'UTF-16'), 'UTF-8') +-- !query schema +struct +-- !query output +Sparkは世界で最も人気のあるビッグデータ処理フレームワークである。 + + +-- !query +select decode() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "2", + "functionName" : "`decode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "decode()" + } ] +} + + +-- !query +select decode(encode('abc', 'utf-8')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "2", + "functionName" : "`decode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "decode(encode('abc', 'utf-8'))" + } ] +} + + +-- !query +select decode(encode('abc', 'utf-8'), 'utf-8') +-- !query schema +struct +-- !query output +abc + + +-- !query +select decode(encode('大千世界', 'utf-32'), 'utf-32') +-- !query schema +struct +-- !query output +大千世界 + + +-- !query +select decode(1, 1, 'Southlake') +-- !query schema +struct +-- !query output +Southlake + + +-- !query +select decode(2, 1, 'Southlake') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select decode(2, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle', 'Non domestic') +-- !query schema +struct +-- !query output +San Francisco + + +-- !query +select decode(6, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle', 'Non domestic') +-- !query schema +struct +-- !query output +Non domestic + + +-- !query +select decode(6, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select decode(null, 6, 'Spark', NULL, 'SQL', 4, 'rocks') +-- !query schema +struct +-- !query output +SQL + + +-- !query +select decode(null, 6, 'Spark', NULL, 'SQL', 4, 'rocks', NULL, '.') +-- !query schema +struct +-- !query output +SQL + + +-- !query +select decode(X'68656c6c6f', 'Windows-xxx') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.CHARSET", + "sqlState" : "22023", + "messageParameters" : { + "charset" : "Windows-xxx", + "charsets" : "iso-8859-1, us-ascii, utf-16, utf-16be, utf-16le, utf-32, utf-8", + "functionName" : "`decode`", + "parameter" : "`charset`" + } +} + + +-- !query +select decode(scol, ecol) from values(X'68656c6c6f', 'Windows-xxx') as t(scol, ecol) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.CHARSET", + "sqlState" : "22023", + "messageParameters" : { + "charset" : "Windows-xxx", + "charsets" : "iso-8859-1, us-ascii, utf-16, utf-16be, utf-16le, utf-32, utf-8", + "functionName" : "`decode`", + "parameter" : "`charset`" + } +} + + +-- !query +set spark.sql.legacy.javaCharsets=true +-- !query schema +struct +-- !query output +spark.sql.legacy.javaCharsets true + + +-- !query +select decode(X'68656c6c6f', 'WINDOWS-1252') +-- !query schema +struct +-- !query output +hello + + +-- !query +select decode(scol, ecol) from values(X'68656c6c6f', 'WINDOWS-1252') as t(scol, ecol) +-- !query schema +struct +-- !query output +hello + + +-- !query +set spark.sql.legacy.javaCharsets=false +-- !query schema +struct +-- !query output +spark.sql.legacy.javaCharsets false + + +-- !query +select decode(X'68656c6c6f', 'WINDOWS-1252') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.CHARSET", + "sqlState" : "22023", + "messageParameters" : { + "charset" : "WINDOWS-1252", + "charsets" : "iso-8859-1, us-ascii, utf-16, utf-16be, utf-16le, utf-32, utf-8", + "functionName" : "`decode`", + "parameter" : "`charset`" + } +} + + +-- !query +select decode(scol, ecol) from values(X'68656c6c6f', 'WINDOWS-1252') as t(scol, ecol) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.CHARSET", + "sqlState" : "22023", + "messageParameters" : { + "charset" : "WINDOWS-1252", + "charsets" : "iso-8859-1, us-ascii, utf-16, utf-16be, utf-16le, utf-32, utf-8", + "functionName" : "`decode`", + "parameter" : "`charset`" + } +} + + +-- !query +set spark.sql.legacy.codingErrorAction=true +-- !query schema +struct +-- !query output +spark.sql.legacy.codingErrorAction true + + +-- !query +select decode(X'E58A9DE5909BE69BB4E5B0BDE4B880E69DAFE98592', 'US-ASCII') +-- !query schema +struct +-- !query output +��������������������� + + +-- !query +select decode(scol, ecol) from values(X'E58A9DE5909BE69BB4E5B0BDE4B880E69DAFE98592', 'US-ASCII') as t(scol, ecol) +-- !query schema +struct +-- !query output +��������������������� + + +-- !query +set spark.sql.legacy.codingErrorAction=false +-- !query schema +struct +-- !query output +spark.sql.legacy.codingErrorAction false + + +-- !query +select decode(X'E8A5BFE587BAE998B3E585B3E697A0E69585E4BABA', 'US-ASCII') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "MALFORMED_CHARACTER_CODING", + "sqlState" : "22000", + "messageParameters" : { + "charset" : "US-ASCII", + "function" : "`decode`" + } +} + + +-- !query +select decode(scol, ecol) from values(X'E8A5BFE587BAE998B3E585B3E697A0E69585E4BABA', 'US-ASCII') as t(scol, ecol) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "MALFORMED_CHARACTER_CODING", + "sqlState" : "22000", + "messageParameters" : { + "charset" : "US-ASCII", + "function" : "`decode`" + } +} + + +-- !query +SELECT CONTAINS(null, 'Spark') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CONTAINS('Spark SQL', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CONTAINS(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CONTAINS('Spark SQL', 'Spark') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT CONTAINS('Spark SQL', 'SQL') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT CONTAINS('Spark SQL', 'SPARK') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT startswith('Spark SQL', 'ark') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT startswith('Spark SQL', 'Spa') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT startswith(null, 'Spark') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT startswith('Spark', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT startswith(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT endswith('Spark SQL', 'QL') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT endswith('Spark SQL', 'Spa') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT endswith(null, 'Spark') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT endswith('Spark', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT endswith(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT contains(x'537061726b2053514c', x'537061726b') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT contains(x'', x'') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT contains(x'537061726b2053514c', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT contains(12, '1') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT contains(true, 'ru') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT contains(x'12', 12) +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT contains(true, false) +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT startswith(x'537061726b2053514c', x'537061726b') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT startswith(x'537061726b2053514c', x'') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT startswith(x'', x'') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT startswith(x'537061726b2053514c', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT endswith(x'537061726b2053514c', x'53516c') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT endsWith(x'537061726b2053514c', x'537061726b') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT endsWith(x'537061726b2053514c', x'') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT endsWith(x'', x'') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT endsWith(x'537061726b2053514c', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_number('454', '000') +-- !query schema +struct +-- !query output +454 + + +-- !query +select to_number('454.2', '000.0') +-- !query schema +struct +-- !query output +454.2 + + +-- !query +select to_number('12,454', '00,000') +-- !query schema +struct +-- !query output +12454 + + +-- !query +select to_number('$78.12', '$00.00') +-- !query schema +struct +-- !query output +78.12 + + +-- !query +select to_number('+454', 'S000') +-- !query schema +struct +-- !query output +454 + + +-- !query +select to_number('-454', 'S000') +-- !query schema +struct +-- !query output +-454 + + +-- !query +select to_number('12,454.8-', '00,000.9MI') +-- !query schema +struct +-- !query output +-12454.8 + + +-- !query +select to_number('00,454.8-', '00,000.9MI') +-- !query schema +struct +-- !query output +-454.8 + + +-- !query +select to_number('<00,454.8>', '00,000.9PR') +-- !query schema +struct, 00,000.9PR):decimal(6,1)> +-- !query output +-454.8 + + +-- !query +select to_binary('', 'base64') +-- !query schema +struct +-- !query output + + + +-- !query +select to_binary(' ', 'base64') +-- !query schema +struct +-- !query output + + + +-- !query +select to_binary(' ab cd ', 'base64') +-- !query schema +struct +-- !query output +i� + + +-- !query +select to_binary(' ab c=', 'base64') +-- !query schema +struct +-- !query output +i� + + +-- !query +select to_binary(' ab cdef= = ', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "' ab cdef= = '", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary( + concat(' b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0', + 'd2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl'), 'base64') +-- !query schema +struct +-- !query output +one two three four five six seven eight nine ten eleven twelve thirteen fourteen fivteen sixteen seventeen eightee + + +-- !query +select to_binary('a', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "'a'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('a?', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "'a?'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('abcde', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "'abcde'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('abcd=', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "'abcd='", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('a===', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "'a==='", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('ab==f', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "'ab==f'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary( + '∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β)', 'utf-8') +-- !query schema +struct +-- !query output +∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β) + + +-- !query +select to_binary('大千世界', 'utf8') +-- !query schema +struct +-- !query output +大千世界 + + +-- !query +select to_binary('', 'utf-8') +-- !query schema +struct +-- !query output + + + +-- !query +select to_binary(' ', 'utf8') +-- !query schema +struct +-- !query output + + + +-- !query +select to_binary('737472696E67') +-- !query schema +struct +-- !query output +string + + +-- !query +select to_binary('737472696E67', 'hex') +-- !query schema +struct +-- !query output +string + + +-- !query +select to_binary('') +-- !query schema +struct +-- !query output + + + +-- !query +select to_binary('1', 'hex') +-- !query schema +struct +-- !query output + + + +-- !query +select to_binary('FF') +-- !query schema +struct +-- !query output +� + + +-- !query +select to_binary('123', 'hex') +-- !query schema +struct +-- !query output +# + + +-- !query +select to_binary('12345', 'hex') +-- !query schema +struct +-- !query output +#E + + +-- !query +select to_binary('GG') +-- !query schema +struct +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'HEX'", + "str" : "'GG'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('01 AF', 'hex') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'HEX'", + "str" : "'01 AF'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('abc', concat('utf', '-8')) +-- !query schema +struct +-- !query output +abc + + +-- !query +select to_binary(' ab cdef= = ', substr('base64whynot', 0, 6)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "' ab cdef= = '", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary(' ab cdef= = ', replace('HEX0', '0')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'HEX'", + "str" : "' ab cdef= = '", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('abc', 'Hex') +-- !query schema +struct +-- !query output + +� + + +-- !query +select to_binary('abc', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_binary(null, 'utf-8') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_binary(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_binary(null, cast(null as string)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_binary('abc', 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE", + "sqlState" : "42K09", + "messageParameters" : { + "inputName" : "fmt", + "inputValue" : "'1'", + "requireType" : "case-insensitive \"STRING\"", + "sqlExpr" : "\"to_binary(abc, 1)\"", + "validValues" : "'hex', 'utf-8', 'utf8', or 'base64'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "to_binary('abc', 1)" + } ] +} + + +-- !query +select to_binary('abc', 'invalidFormat') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE", + "sqlState" : "42K09", + "messageParameters" : { + "inputName" : "fmt", + "inputValue" : "'invalidformat'", + "requireType" : "case-insensitive \"STRING\"", + "sqlExpr" : "\"to_binary(abc, invalidFormat)\"", + "validValues" : "'hex', 'utf-8', 'utf8', or 'base64'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "to_binary('abc', 'invalidFormat')" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW fmtTable(fmtField) AS SELECT * FROM VALUES ('invalidFormat') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT to_binary('abc', fmtField) FROM fmtTable +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"fmtField\"", + "inputName" : "`fmt`", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"to_binary(abc, fmtField)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "to_binary('abc', fmtField)" + } ] +} + + +-- !query +DROP VIEW IF EXISTS fmtTable +-- !query schema +struct<> +-- !query output + + + +-- !query +select luhn_check('4111111111111111') +-- !query schema +struct +-- !query output +true + + +-- !query +select luhn_check('5500000000000004') +-- !query schema +struct +-- !query output +true + + +-- !query +select luhn_check('340000000000009') +-- !query schema +struct +-- !query output +true + + +-- !query +select luhn_check('6011000000000004') +-- !query schema +struct +-- !query output +true + + +-- !query +select luhn_check('6011000000000005') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check('378282246310006') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check('0') +-- !query schema +struct +-- !query output +true + + +-- !query +select luhn_check('4111111111111111 ') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check('4111111 111111111') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check(' 4111111111111111') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check('') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check(' ') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check('510B105105105106') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check('ABCDED') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select luhn_check(6011111111111117) +-- !query schema +struct +-- !query output +true + + +-- !query +select luhn_check(6011111111111118) +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check(123.456) +-- !query schema +struct +-- !query output +false + + +-- !query +select is_valid_utf8('') +-- !query schema +struct +-- !query output +true + + +-- !query +select is_valid_utf8('abc') +-- !query schema +struct +-- !query output +true + + +-- !query +select is_valid_utf8(x'80') +-- !query schema +struct +-- !query output +false + + +-- !query +select make_valid_utf8('') +-- !query schema +struct +-- !query output + + + +-- !query +select make_valid_utf8('abc') +-- !query schema +struct +-- !query output +abc + + +-- !query +select make_valid_utf8(x'80') +-- !query schema +struct +-- !query output +� + + +-- !query +select validate_utf8('') +-- !query schema +struct +-- !query output + + + +-- !query +select validate_utf8('abc') +-- !query schema +struct +-- !query output +abc + + +-- !query +select validate_utf8(x'80') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_UTF8_STRING", + "sqlState" : "22029", + "messageParameters" : { + "str" : "\\x80" + } +} + + +-- !query +select try_validate_utf8('') +-- !query schema +struct +-- !query output + + + +-- !query +select try_validate_utf8('abc') +-- !query schema +struct +-- !query output +abc + + +-- !query +select try_validate_utf8(x'80') +-- !query schema +struct +-- !query output +NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/timestamp.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/timestamp.sql.out new file mode 100644 index 000000000000..0e0b014a3b16 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/timestamp.sql.out @@ -0,0 +1,1215 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select timestamp '2019-01-01\t' +-- !query schema +struct +-- !query output +2019-01-01 00:00:00 + + +-- !query +select timestamp '2019-01-01中文' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2019-01-01中文'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "timestamp '2019-01-01中文'" + } ] +} + + +-- !query +select timestamp'4294967297' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'4294967297'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "timestamp'4294967297'" + } ] +} + + +-- !query +select timestamp'2021-01-01T12:30:4294967297.123456' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2021-01-01T12:30:4294967297.123456'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'" + } ] +} + + +-- !query +select current_timestamp = current_timestamp +-- !query schema +struct<(current_timestamp() = current_timestamp()):boolean> +-- !query output +true + + +-- !query +select current_timestamp() = current_timestamp() +-- !query schema +struct<(current_timestamp() = current_timestamp()):boolean> +-- !query output +true + + +-- !query +select localtimestamp() = localtimestamp() +-- !query schema +struct<(localtimestamp() = localtimestamp()):boolean> +-- !query output +true + + +-- !query +SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678) +-- !query schema +struct +-- !query output +2021-07-11 06:30:45.678 + + +-- !query +SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678, 'CET') +-- !query schema +struct +-- !query output +2021-07-10 21:30:45.678 + + +-- !query +SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 1) +-- !query schema +struct +-- !query output +0001-01-01 01:01:01 + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 60) +-- !query schema +struct +-- !query output +0001-01-01 01:02:00 + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 61) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 59.999999) +-- !query schema +struct +-- !query output +0001-01-01 01:01:59.999999 + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00 1931-01-07 00:30:00 NULL + + +-- !query +select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23)) +-- !query schema +struct +-- !query output +1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 + + +-- !query +select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00.123 1931-01-07 00:29:59.877 NULL + + +-- !query +select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00.123123 1931-01-07 00:29:59.876877 NULL + + +-- !query +select TIMESTAMP_SECONDS(1230219000123123) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_SECONDS(-1230219000123123) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_MILLIS(92233720368547758) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_MILLIS(-92233720368547758) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_SECONDS(0.1234567) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Rounding necessary + + +-- !query +select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567)) +-- !query schema +struct +-- !query output +1969-12-31 16:00:00.123456 1969-12-31 16:00:00.123456 + + +-- !query +create temporary view ttf1 as select * from values + (1, 2), + (2, 3) + as ttf1(`current_date`, `current_timestamp`) +-- !query schema +struct<> +-- !query output + + + +-- !query +select typeof(current_date), typeof(current_timestamp) from ttf1 +-- !query schema +struct +-- !query output +int int +int int + + +-- !query +create temporary view ttf2 as select * from values + (1, 2), + (2, 3) + as ttf2(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2 +-- !query schema +struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean,a:int,b:int> +-- !query output +true true 1 2 +true true 2 3 + + +-- !query +select a, b from ttf2 order by a, current_date +-- !query schema +struct +-- !query output +1 2 +2 3 + + +-- !query +select UNIX_SECONDS(timestamp'2020-12-01 14:30:08Z'), UNIX_SECONDS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_SECONDS(null) +-- !query schema +struct +-- !query output +1606833008 1606833008 NULL + + +-- !query +select UNIX_MILLIS(timestamp'2020-12-01 14:30:08Z'), UNIX_MILLIS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MILLIS(null) +-- !query schema +struct +-- !query output +1606833008000 1606833008999 NULL + + +-- !query +select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MICROS(null) +-- !query schema +struct +-- !query output +1606833008000000 1606833008999999 NULL + + +-- !query +select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 + + +-- !query +select to_timestamp(1) +-- !query schema +struct +-- !query output +1969-12-31 16:00:01 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 03:11:12.123 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 08:11:12.12345 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.123456 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.123456 + + +-- !query +select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.123 + + +-- !query +select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12 + + +-- !query +select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:00 + + +-- !query +select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12345 + + +-- !query +select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +0019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'") +-- !query schema +struct +-- !query output +2019-10-06 00:00:00 + + +-- !query +select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd") +-- !query schema +struct +-- !query output +2019-10-06 00:00:00 + + +-- !query +select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12 + + +-- !query +select to_timestamp("2019-10-06T10:11:12'", "yyyy-MM-dd'T'HH:mm:ss''") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("16", "dd") +-- !query schema +struct +-- !query output +1970-01-16 00:00:00 + + +-- !query +select to_timestamp("02-29", "MM-dd") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("2019 40", "yyyy mm") +-- !query schema +struct +-- !query output +2019-01-01 00:40:00 + + +-- !query +select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss") +-- !query schema +struct +-- !query output +2019-01-01 10:10:10 + + +-- !query +select timestamp'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query schema +struct<(TIMESTAMP '2011-11-11 11:11:11' - TIMESTAMP '2011-11-11 11:11:10'):interval day to second> +-- !query output +0 00:00:01.000000000 + + +-- !query +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' +-- !query schema +struct<(DATE '2020-01-01' - TIMESTAMP '2019-10-06 10:11:12.345678'):interval day to second> +-- !query output +86 13:48:47.654322000 + + +-- !query +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' +-- !query schema +struct<(TIMESTAMP '2019-10-06 10:11:12.345678' - DATE '2020-01-01'):interval day to second> +-- !query output +-86 13:48:47.654322000 + + +-- !query +select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2011-11-11 11:11:10\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - 2011-11-11 11:11:10)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'" + } ] +} + + +-- !query +select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2011-11-11 11:11:11\"", + "inputType" : "\"STRING\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(2011-11-11 11:11:11 - TIMESTAMP '2011-11-11 11:11:10')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' - null +-- !query schema +struct<(TIMESTAMP '2011-11-11 11:11:11' - NULL):interval day to second> +-- !query output +NULL + + +-- !query +select null - timestamp'2011-11-11 11:11:11' +-- !query schema +struct<(NULL - TIMESTAMP '2011-11-11 11:11:11'):interval day to second> +-- !query output +NULL + + +-- !query +create temporary view ts_view as select '2011-11-11 11:11:11' str +-- !query schema +struct<> +-- !query output + + + +-- !query +select str - timestamp'2011-11-11 11:11:11' from ts_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(str - TIMESTAMP '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "str - timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' - str from ts_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "timestamp'2011-11-11 11:11:11' - str" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "timestamp'2011-11-11 11:11:11' + '1'" + } ] +} + + +-- !query +select '1' + timestamp'2011-11-11 11:11:11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "'1' + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + null +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"VOID\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "timestamp'2011-11-11 11:11:11' + null" + } ] +} + + +-- !query +select null + timestamp'2011-11-11 11:11:11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "null + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + interval '2' day, + timestamp'2011-11-11 11:11:11' - interval '2-2' year to month, + timestamp'2011-11-11 11:11:11' + interval '-2' second, + timestamp'2011-11-11 11:11:11' - interval '12:12:12.123456789' hour to second, + - interval 2 years + timestamp'2011-11-11 11:11:11', + interval '1 12' day to hour + timestamp'2011-11-11 11:11:11' +-- !query schema +struct +-- !query output +2011-11-13 11:11:11 2009-09-11 11:11:11 2011-11-11 11:11:09 2011-11-10 22:58:58.876544 2009-11-11 11:11:11 2011-11-12 23:11:11 + + +-- !query +select date '2012-01-01' - interval 3 hours, + date '2012-01-01' + interval '12:12:12' hour to second, + interval '2' minute + date '2012-01-01' +-- !query schema +struct +-- !query output +2011-12-31 21:00:00 2012-01-01 12:12:12 2012-01-01 00:02:00 + + +-- !query +select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'yyyy-MM-dd GGGGG'" + } +} + + +-- !query +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd MM yyyy EEEEEE'" + } +} + + +-- !query +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd MM yyyy EEEEE'" + } +} + + +-- !query +select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd MM yyyy EEEEE'" + } +} + + +-- !query +select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd/MMMMM/yyyy'" + } +} + + +-- !query +select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd/MMMMM/yyyy'" + } +} + + +-- !query +select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03') +-- !query schema +struct +-- !query output +2022-01-14 01:02:03 + + +-- !query +select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03') +-- !query schema +struct +-- !query output +2022-02-14 02:00:03 + + +-- !query +select timestampadd(YEAR, 1, date'2022-02-15') +-- !query schema +struct +-- !query output +2023-02-15 00:00:00 + + +-- !query +select timestampadd(SECOND, -1, date'2022-02-15') +-- !query schema +struct +-- !query output +2022-02-14 23:59:59 + + +-- !query +select timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampadd`", + "invalidValue" : "'MONTH'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03')" + } ] +} + + +-- !query +select timestampadd('SECOND', -1, date'2022-02-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampadd`", + "invalidValue" : "'SECOND'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "timestampadd('SECOND', -1, date'2022-02-15')" + } ] +} + + +-- !query +select timestampdiff(MONTH, timestamp'2022-02-14 01:02:03', timestamp'2022-01-14 01:02:03') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select timestampdiff(MINUTE, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03') +-- !query schema +struct +-- !query output +58 + + +-- !query +select timestampdiff(YEAR, date'2022-02-15', date'2023-02-15') +-- !query schema +struct +-- !query output +1 + + +-- !query +select timestampdiff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampdiff`", + "invalidValue" : "'MINUTE'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 94, + "fragment" : "timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03')" + } ] +} + + +-- !query +select timestampdiff('YEAR', date'2022-02-15', date'2023-02-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampdiff`", + "invalidValue" : "'YEAR'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "timestampdiff('YEAR', date'2022-02-15', date'2023-02-15')" + } ] +} + + +-- !query +select timediff(QUARTER, timestamp'2023-08-10 01:02:03', timestamp'2022-01-14 01:02:03') +-- !query schema +struct +-- !query output +-6 + + +-- !query +select timediff(HOUR, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 12:00:03') +-- !query schema +struct +-- !query output +10 + + +-- !query +select timediff(DAY, date'2022-02-15', date'2023-02-15') +-- !query schema +struct +-- !query output +365 + + +-- !query +select timediff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timediff`", + "invalidValue" : "'MINUTE'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 89, + "fragment" : "timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03')" + } ] +} + + +-- !query +select timediff('YEAR', date'2020-02-15', date'2023-02-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timediff`", + "invalidValue" : "'YEAR'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "timediff('YEAR', date'2020-02-15', date'2023-02-15')" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/try_aggregates.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/try_aggregates.sql.out new file mode 100644 index 000000000000..df1fe996781a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/try_aggregates.sql.out @@ -0,0 +1,299 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_sum(col) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +30 + + +-- !query +SELECT try_sum(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct +-- !query output +30.0 + + +-- !query +SELECT try_sum(col) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +25 + + +-- !query +SELECT try_sum(col) FROM VALUES (NULL), (NULL) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (98765432109876543210987654321098765432BD), (98765432109876543210987654321098765432BD) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +0-2 + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct +-- !query output +0 00:00:02.000000000 + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct +-- !query output +-9223372036854775806 + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_avg(col) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +10.0 + + +-- !query +SELECT try_avg(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct +-- !query output +10.00000 + + +-- !query +SELECT try_avg(col) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +12.5 + + +-- !query +SELECT try_avg(col) FROM VALUES (NULL), (NULL) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct +-- !query output +4.6116860184273879E18 + + +-- !query +SELECT try_avg(col) FROM VALUES (98765432109876543210987654321098765432BD), (98765432109876543210987654321098765432BD) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +0-1 + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct +-- !query output +0 00:00:01.000000000 + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct +-- !query output +-4.6116860184273879E18 + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/try_aggregates.sql.out.java21 b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/try_aggregates.sql.out.java21 new file mode 100644 index 000000000000..7affe568234f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/try_aggregates.sql.out.java21 @@ -0,0 +1,299 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_sum(col) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +30 + + +-- !query +SELECT try_sum(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct +-- !query output +30.0 + + +-- !query +SELECT try_sum(col) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +25 + + +-- !query +SELECT try_sum(col) FROM VALUES (NULL), (NULL) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (98765432109876543210987654321098765432BD), (98765432109876543210987654321098765432BD) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +0-2 + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct +-- !query output +0 00:00:02.000000000 + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct +-- !query output +-9223372036854775806 + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_avg(col) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +10.0 + + +-- !query +SELECT try_avg(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct +-- !query output +10.00000 + + +-- !query +SELECT try_avg(col) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +12.5 + + +-- !query +SELECT try_avg(col) FROM VALUES (NULL), (NULL) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct +-- !query output +4.611686018427388E18 + + +-- !query +SELECT try_avg(col) FROM VALUES (98765432109876543210987654321098765432BD), (98765432109876543210987654321098765432BD) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +0-1 + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct +-- !query output +0 00:00:01.000000000 + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct +-- !query output +-4.611686018427388E18 + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/try_arithmetic.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/try_arithmetic.sql.out new file mode 100644 index 000000000000..b12680c2a675 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/try_arithmetic.sql.out @@ -0,0 +1,569 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_add(1, 1) +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT try_add(2147483647, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(2147483647, decimal(1)) +-- !query schema +struct +-- !query output +2147483648 + + +-- !query +SELECT try_add(2147483647, "1") +-- !query schema +struct +-- !query output +2.147483648E9 + + +-- !query +SELECT try_add(-2147483648, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(9223372036854775807L, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(-9223372036854775808L, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(1, (2147483647 + 1)) +-- !query schema +struct +-- !query output +-2147483647 + + +-- !query +SELECT try_add(1L, (9223372036854775807L + 1L)) +-- !query schema +struct +-- !query output +-9223372036854775807 + + +-- !query +SELECT try_add(1, 1.0 / 0.0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(date'2021-01-01', 1) +-- !query schema +struct +-- !query output +2021-01-02 + + +-- !query +SELECT try_add(1, date'2021-01-01') +-- !query schema +struct +-- !query output +2021-01-02 + + +-- !query +SELECT try_add(date'2021-01-01', interval 2 year) +-- !query schema +struct +-- !query output +2023-01-01 + + +-- !query +SELECT try_add(date'2021-01-01', interval 2 second) +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(interval 2 year, date'2021-01-01') +-- !query schema +struct +-- !query output +2023-01-01 + + +-- !query +SELECT try_add(interval 2 second, date'2021-01-01') +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(timestamp_ltz'2021-01-01 00:00:00', interval 2 year) +-- !query schema +struct +-- !query output +2023-01-01 00:00:00 + + +-- !query +SELECT try_add(timestamp_ntz'2021-01-01 00:00:00', interval 2 second) +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(interval 2 year, timestamp_ltz'2021-01-01 00:00:00') +-- !query schema +struct +-- !query output +2023-01-01 00:00:00 + + +-- !query +SELECT try_add(interval 2 second, timestamp_ntz'2021-01-01 00:00:00') +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(interval 2 year, interval 2 year) +-- !query schema +struct +-- !query output +4-0 + + +-- !query +SELECT try_add(interval 2 second, interval 2 second) +-- !query schema +struct +-- !query output +0 00:00:04.000000000 + + +-- !query +SELECT try_add(interval 2 year, interval 2 second) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2' YEAR\"", + "inputType" : "\"INTERVAL YEAR\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2' YEAR + INTERVAL '02' SECOND\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "try_add(interval 2 year, interval 2 second)" + } ] +} + + +-- !query +SELECT try_add(interval 2147483647 month, interval 2 month) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(interval 106751991 day, interval 3 day) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, 0.5) +-- !query schema +struct +-- !query output +2.000000 + + +-- !query +SELECT try_divide(1, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(0, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, (2147483647 + 1)) +-- !query schema +struct +-- !query output +-4.6566128730773926E-10 + + +-- !query +SELECT try_divide(1L, (9223372036854775807L + 1L)) +-- !query schema +struct +-- !query output +-1.0842021724855044E-19 + + +-- !query +SELECT try_divide(1, 1.0 / 0.0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, decimal(0)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, "0") +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 2 year, 2) +-- !query schema +struct +-- !query output +1-0 + + +-- !query +SELECT try_divide(interval 2 second, 2) +-- !query schema +struct +-- !query output +0 00:00:01.000000000 + + +-- !query +SELECT try_divide(interval 2 year, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 2 second, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 2147483647 month, 0.5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 106751991 day, 0.5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(1, 1) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT try_subtract(2147483647, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(2147483647, decimal(-1)) +-- !query schema +struct +-- !query output +2147483648 + + +-- !query +SELECT try_subtract(2147483647, "-1") +-- !query schema +struct +-- !query output +2.147483648E9 + + +-- !query +SELECT try_subtract(-2147483648, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(9223372036854775807L, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(-9223372036854775808L, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(1, (2147483647 + 1)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(1L, (9223372036854775807L + 1L)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(1, 1.0 / 0.0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(interval 2 year, interval 3 year) +-- !query schema +struct +-- !query output +-1-0 + + +-- !query +SELECT try_subtract(interval 3 second, interval 2 second) +-- !query schema +struct +-- !query output +0 00:00:01.000000000 + + +-- !query +SELECT try_subtract(interval 2147483647 month, interval -2 month) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(interval 106751991 day, interval -3 day) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(2, 3) +-- !query schema +struct +-- !query output +6 + + +-- !query +SELECT try_multiply(2147483647, -2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(2147483647, decimal(-2)) +-- !query schema +struct +-- !query output +-4294967294 + + +-- !query +SELECT try_multiply(2147483647, "-2") +-- !query schema +struct +-- !query output +-4.294967294E9 + + +-- !query +SELECT try_multiply(-2147483648, 2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(9223372036854775807L, 2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(-9223372036854775808L, -2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(1, (2147483647 + 1)) +-- !query schema +struct +-- !query output +-2147483648 + + +-- !query +SELECT try_multiply(1L, (9223372036854775807L + 1L)) +-- !query schema +struct +-- !query output +-9223372036854775808 + + +-- !query +SELECT try_multiply(1, 1.0 / 0.0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(interval 2 year, 2) +-- !query schema +struct +-- !query output +4-0 + + +-- !query +SELECT try_multiply(interval 2 second, 2) +-- !query schema +struct +-- !query output +0 00:00:04.000000000 + + +-- !query +SELECT try_multiply(interval 2 year, 0) +-- !query schema +struct +-- !query output +0-0 + + +-- !query +SELECT try_multiply(interval 2 second, 0) +-- !query schema +struct +-- !query output +0 00:00:00.000000000 + + +-- !query +SELECT try_multiply(interval 2147483647 month, 2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(interval 106751991 day, 2) +-- !query schema +struct +-- !query output +NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/try_arithmetic.sql.out.java21 b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/try_arithmetic.sql.out.java21 new file mode 100644 index 000000000000..002a0dfcf37e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/try_arithmetic.sql.out.java21 @@ -0,0 +1,569 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_add(1, 1) +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT try_add(2147483647, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(2147483647, decimal(1)) +-- !query schema +struct +-- !query output +2147483648 + + +-- !query +SELECT try_add(2147483647, "1") +-- !query schema +struct +-- !query output +2.147483648E9 + + +-- !query +SELECT try_add(-2147483648, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(9223372036854775807L, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(-9223372036854775808L, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(1, (2147483647 + 1)) +-- !query schema +struct +-- !query output +-2147483647 + + +-- !query +SELECT try_add(1L, (9223372036854775807L + 1L)) +-- !query schema +struct +-- !query output +-9223372036854775807 + + +-- !query +SELECT try_add(1, 1.0 / 0.0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(date'2021-01-01', 1) +-- !query schema +struct +-- !query output +2021-01-02 + + +-- !query +SELECT try_add(1, date'2021-01-01') +-- !query schema +struct +-- !query output +2021-01-02 + + +-- !query +SELECT try_add(date'2021-01-01', interval 2 year) +-- !query schema +struct +-- !query output +2023-01-01 + + +-- !query +SELECT try_add(date'2021-01-01', interval 2 second) +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(interval 2 year, date'2021-01-01') +-- !query schema +struct +-- !query output +2023-01-01 + + +-- !query +SELECT try_add(interval 2 second, date'2021-01-01') +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(timestamp_ltz'2021-01-01 00:00:00', interval 2 year) +-- !query schema +struct +-- !query output +2023-01-01 00:00:00 + + +-- !query +SELECT try_add(timestamp_ntz'2021-01-01 00:00:00', interval 2 second) +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(interval 2 year, timestamp_ltz'2021-01-01 00:00:00') +-- !query schema +struct +-- !query output +2023-01-01 00:00:00 + + +-- !query +SELECT try_add(interval 2 second, timestamp_ntz'2021-01-01 00:00:00') +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(interval 2 year, interval 2 year) +-- !query schema +struct +-- !query output +4-0 + + +-- !query +SELECT try_add(interval 2 second, interval 2 second) +-- !query schema +struct +-- !query output +0 00:00:04.000000000 + + +-- !query +SELECT try_add(interval 2 year, interval 2 second) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2' YEAR\"", + "inputType" : "\"INTERVAL YEAR\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2' YEAR + INTERVAL '02' SECOND\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "try_add(interval 2 year, interval 2 second)" + } ] +} + + +-- !query +SELECT try_add(interval 2147483647 month, interval 2 month) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(interval 106751991 day, interval 3 day) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, 0.5) +-- !query schema +struct +-- !query output +2.000000 + + +-- !query +SELECT try_divide(1, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(0, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, (2147483647 + 1)) +-- !query schema +struct +-- !query output +-4.656612873077393E-10 + + +-- !query +SELECT try_divide(1L, (9223372036854775807L + 1L)) +-- !query schema +struct +-- !query output +-1.0842021724855044E-19 + + +-- !query +SELECT try_divide(1, 1.0 / 0.0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, decimal(0)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, "0") +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 2 year, 2) +-- !query schema +struct +-- !query output +1-0 + + +-- !query +SELECT try_divide(interval 2 second, 2) +-- !query schema +struct +-- !query output +0 00:00:01.000000000 + + +-- !query +SELECT try_divide(interval 2 year, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 2 second, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 2147483647 month, 0.5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 106751991 day, 0.5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(1, 1) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT try_subtract(2147483647, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(2147483647, decimal(-1)) +-- !query schema +struct +-- !query output +2147483648 + + +-- !query +SELECT try_subtract(2147483647, "-1") +-- !query schema +struct +-- !query output +2.147483648E9 + + +-- !query +SELECT try_subtract(-2147483648, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(9223372036854775807L, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(-9223372036854775808L, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(1, (2147483647 + 1)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(1L, (9223372036854775807L + 1L)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(1, 1.0 / 0.0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(interval 2 year, interval 3 year) +-- !query schema +struct +-- !query output +-1-0 + + +-- !query +SELECT try_subtract(interval 3 second, interval 2 second) +-- !query schema +struct +-- !query output +0 00:00:01.000000000 + + +-- !query +SELECT try_subtract(interval 2147483647 month, interval -2 month) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(interval 106751991 day, interval -3 day) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(2, 3) +-- !query schema +struct +-- !query output +6 + + +-- !query +SELECT try_multiply(2147483647, -2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(2147483647, decimal(-2)) +-- !query schema +struct +-- !query output +-4294967294 + + +-- !query +SELECT try_multiply(2147483647, "-2") +-- !query schema +struct +-- !query output +-4.294967294E9 + + +-- !query +SELECT try_multiply(-2147483648, 2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(9223372036854775807L, 2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(-9223372036854775808L, -2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(1, (2147483647 + 1)) +-- !query schema +struct +-- !query output +-2147483648 + + +-- !query +SELECT try_multiply(1L, (9223372036854775807L + 1L)) +-- !query schema +struct +-- !query output +-9223372036854775808 + + +-- !query +SELECT try_multiply(1, 1.0 / 0.0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(interval 2 year, 2) +-- !query schema +struct +-- !query output +4-0 + + +-- !query +SELECT try_multiply(interval 2 second, 2) +-- !query schema +struct +-- !query output +0 00:00:04.000000000 + + +-- !query +SELECT try_multiply(interval 2 year, 0) +-- !query schema +struct +-- !query output +0-0 + + +-- !query +SELECT try_multiply(interval 2 second, 0) +-- !query schema +struct +-- !query output +0 00:00:00.000000000 + + +-- !query +SELECT try_multiply(interval 2147483647 month, 2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(interval 106751991 day, 2) +-- !query schema +struct +-- !query output +NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/try_datetime_functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/try_datetime_functions.sql.out new file mode 100644 index 000000000000..75a6f15bd363 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/try_datetime_functions.sql.out @@ -0,0 +1,56 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select try_to_timestamp(null), try_to_timestamp('2016-12-31 00:12:00'), try_to_timestamp('2016-12-31', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 + + +-- !query +select try_to_timestamp(1) +-- !query schema +struct +-- !query output +1969-12-31 16:00:01 + + +-- !query +select try_to_timestamp('2016-12-31 abc') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_timestamp("02-29", "MM-dd") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd MM yyyy EEEEEE'" + } +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/try_element_at.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/try_element_at.sql.out new file mode 100644 index 000000000000..0437f9d6dd9e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/nonansi/try_element_at.sql.out @@ -0,0 +1,67 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_element_at(array(1, 2, 3), 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_INDEX_OF_ZERO", + "sqlState" : "22003" +} + + +-- !query +SELECT try_element_at(array(1, 2, 3), 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT try_element_at(array(1, 2, 3), 3) +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT try_element_at(array(1, 2, 3), 4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_element_at(array(1, 2, 3), -1) +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT try_element_at(array(1, 2, 3), -4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_element_at(map('a','b'), 'a') +-- !query schema +struct +-- !query output +b + + +-- !query +SELECT try_element_at(map('a','b'), 'abc') +-- !query schema +struct +-- !query output +NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/null-handling.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/null-handling.sql.out new file mode 100644 index 000000000000..fb96be8317a5 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/null-handling.sql.out @@ -0,0 +1,332 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create table t1(a int, b int, c int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t1 values(1,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t1 values(2,0,1) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t1 values(3,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t1 values(4,1,1) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t1 values(5,null,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t1 values(6,null,1) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t1 values(7,null,null) +-- !query schema +struct<> +-- !query output + + + +-- !query +select a, b+c from t1 +-- !query schema +struct +-- !query output +1 0 +2 1 +3 1 +4 2 +5 NULL +6 NULL +7 NULL + + +-- !query +select b + 0 from t1 where a = 5 +-- !query schema +struct<(b + 0):int> +-- !query output +NULL + + +-- !query +select -100 + b + 100 from t1 where a = 5 +-- !query schema +struct<((-100 + b) + 100):int> +-- !query output +NULL + + +-- !query +select a+10, b*0 from t1 +-- !query schema +struct<(a + 10):int,(b * 0):int> +-- !query output +11 0 +12 0 +13 0 +14 0 +15 NULL +16 NULL +17 NULL + + +-- !query +select distinct b from t1 +-- !query schema +struct +-- !query output +0 +1 +NULL + + +-- !query +select b from t1 union select b from t1 +-- !query schema +struct +-- !query output +0 +1 +NULL + + +-- !query +select a+20, case b when c then 1 else 0 end from t1 +-- !query schema +struct<(a + 20):int,CASE WHEN (b = c) THEN 1 ELSE 0 END:int> +-- !query output +21 1 +22 0 +23 0 +24 1 +25 0 +26 0 +27 0 + + +-- !query +select a+30, case c when b then 1 else 0 end from t1 +-- !query schema +struct<(a + 30):int,CASE WHEN (c = b) THEN 1 ELSE 0 END:int> +-- !query output +31 1 +32 0 +33 0 +34 1 +35 0 +36 0 +37 0 + + +-- !query +select a+40, case when b<>0 then 1 else 0 end from t1 +-- !query schema +struct<(a + 40):int,CASE WHEN (NOT (b = 0)) THEN 1 ELSE 0 END:int> +-- !query output +41 0 +42 0 +43 1 +44 1 +45 0 +46 0 +47 0 + + +-- !query +select a+50, case when not b<>0 then 1 else 0 end from t1 +-- !query schema +struct<(a + 50):int,CASE WHEN (NOT (NOT (b = 0))) THEN 1 ELSE 0 END:int> +-- !query output +51 1 +52 1 +53 0 +54 0 +55 0 +56 0 +57 0 + + +-- !query +select a+60, case when b<>0 and c<>0 then 1 else 0 end from t1 +-- !query schema +struct<(a + 60):int,CASE WHEN ((NOT (b = 0)) AND (NOT (c = 0))) THEN 1 ELSE 0 END:int> +-- !query output +61 0 +62 0 +63 0 +64 1 +65 0 +66 0 +67 0 + + +-- !query +select a+70, case when not (b<>0 and c<>0) then 1 else 0 end from t1 +-- !query schema +struct<(a + 70):int,CASE WHEN (NOT ((NOT (b = 0)) AND (NOT (c = 0)))) THEN 1 ELSE 0 END:int> +-- !query output +71 1 +72 1 +73 1 +74 0 +75 1 +76 0 +77 0 + + +-- !query +select a+80, case when b<>0 or c<>0 then 1 else 0 end from t1 +-- !query schema +struct<(a + 80):int,CASE WHEN ((NOT (b = 0)) OR (NOT (c = 0))) THEN 1 ELSE 0 END:int> +-- !query output +81 0 +82 1 +83 1 +84 1 +85 0 +86 1 +87 0 + + +-- !query +select a+90, case when not (b<>0 or c<>0) then 1 else 0 end from t1 +-- !query schema +struct<(a + 90):int,CASE WHEN (NOT ((NOT (b = 0)) OR (NOT (c = 0)))) THEN 1 ELSE 0 END:int> +-- !query output +91 1 +92 0 +93 0 +94 0 +95 0 +96 0 +97 0 + + +-- !query +select count(*), count(b), sum(b), avg(b), min(b), max(b) from t1 +-- !query schema +struct +-- !query output +7 4 2 0.5 0 1 + + +-- !query +select a+100 from t1 where b<10 +-- !query schema +struct<(a + 100):int> +-- !query output +101 +102 +103 +104 + + +-- !query +select a+110 from t1 where not b>10 +-- !query schema +struct<(a + 110):int> +-- !query output +111 +112 +113 +114 + + +-- !query +select a+120 from t1 where b<10 OR c=1 +-- !query schema +struct<(a + 120):int> +-- !query output +121 +122 +123 +124 +126 + + +-- !query +select a+130 from t1 where b<10 AND c=1 +-- !query schema +struct<(a + 130):int> +-- !query output +132 +134 + + +-- !query +select a+140 from t1 where not (b<10 AND c=1) +-- !query schema +struct<(a + 140):int> +-- !query output +141 +143 +145 + + +-- !query +select a+150 from t1 where not (c=1 AND b<10) +-- !query schema +struct<(a + 150):int> +-- !query output +151 +153 +155 + + +-- !query +select b, c, equal_null(b, c), equal_null(c, b) from t1 +-- !query schema +struct +-- !query output +0 0 true true +0 1 false false +1 0 false false +1 1 true true +NULL 0 false false +NULL 1 false false +NULL NULL true true + + +-- !query +drop table t1 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/null-propagation.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/null-propagation.sql.out new file mode 100644 index 000000000000..ef16981ff1d8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/null-propagation.sql.out @@ -0,0 +1,35 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT COUNT(NULL) FROM VALUES 1, 2, 3 +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT COUNT(1 + NULL) FROM VALUES 1, 2, 3 +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT COUNT(NULL) OVER () FROM VALUES 1, 2, 3 +-- !query schema +struct +-- !query output +0 +0 +0 + + +-- !query +SELECT COUNT(1 + NULL) OVER () FROM VALUES 1, 2, 3 +-- !query schema +struct +-- !query output +0 +0 +0 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/operators.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/operators.sql.out new file mode 100644 index 000000000000..356e5eca5feb --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/operators.sql.out @@ -0,0 +1,699 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select -100 +-- !query schema +struct<-100:int> +-- !query output +-100 + + +-- !query +select +230 +-- !query schema +struct<(+ 230):int> +-- !query output +230 + + +-- !query +select -5.2 +-- !query schema +struct<-5.2:decimal(2,1)> +-- !query output +-5.2 + + +-- !query +select +6.8e0 +-- !query schema +struct<(+ 6.8):double> +-- !query output +6.8 + + +-- !query +select -key, +key from testdata where key = 2 +-- !query schema +struct<(- key):int,(+ key):int> +-- !query output +-2 2 + + +-- !query +select -(key + 1), - key + 1, +(key + 5) from testdata where key = 1 +-- !query schema +struct<(- (key + 1)):int,((- key) + 1):int,(+ (key + 5)):int> +-- !query output +-2 0 6 + + +-- !query +select -max(key), +max(key) from testdata +-- !query schema +struct<(- max(key)):int,(+ max(key)):int> +-- !query output +-100 100 + + +-- !query +select - (-10) +-- !query schema +struct<(- -10):int> +-- !query output +10 + + +-- !query +select + (-key) from testdata where key = 32 +-- !query schema +struct<(+ (- key)):int> +-- !query output +-32 + + +-- !query +select - (+max(key)) from testdata +-- !query schema +struct<(- (+ max(key))):int> +-- !query output +-100 + + +-- !query +select - - 3 +-- !query schema +struct<(- -3):int> +-- !query output +3 + + +-- !query +select - + 20 +-- !query schema +struct<(- (+ 20)):int> +-- !query output +-20 + + +-- !query +select + + 100 +-- !query schema +struct<(+ (+ 100)):int> +-- !query output +100 + + +-- !query +select - - max(key) from testdata +-- !query schema +struct<(- (- max(key))):int> +-- !query output +100 + + +-- !query +select + - key from testdata where key = 33 +-- !query schema +struct<(+ (- key)):int> +-- !query output +-33 + + +-- !query +select 5 / 2 +-- !query schema +struct<(5 / 2):double> +-- !query output +2.5 + + +-- !query +select 5 / 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "5 / 0" + } ] +} + + +-- !query +select 5 / null +-- !query schema +struct<(5 / NULL):double> +-- !query output +NULL + + +-- !query +select null / 5 +-- !query schema +struct<(NULL / 5):double> +-- !query output +NULL + + +-- !query +select 5 div 2 +-- !query schema +struct<(5 div 2):bigint> +-- !query output +2 + + +-- !query +select 5 div 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 14, + "fragment" : "5 div 0" + } ] +} + + +-- !query +select 5 div null +-- !query schema +struct<(5 div NULL):bigint> +-- !query output +NULL + + +-- !query +select null div 5 +-- !query schema +struct<(NULL div 5):bigint> +-- !query output +NULL + + +-- !query +select cast(51 as decimal(10, 0)) div cast(2 as decimal(2, 0)) +-- !query schema +struct<(CAST(51 AS DECIMAL(10,0)) div CAST(2 AS DECIMAL(2,0))):bigint> +-- !query output +25 + + +-- !query +select cast(5 as decimal(1, 0)) div cast(0 as decimal(2, 0)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 60, + "fragment" : "cast(5 as decimal(1, 0)) div cast(0 as decimal(2, 0))" + } ] +} + + +-- !query +select cast(5 as decimal(1, 0)) div cast(null as decimal(2, 0)) +-- !query schema +struct<(CAST(5 AS DECIMAL(1,0)) div CAST(NULL AS DECIMAL(2,0))):bigint> +-- !query output +NULL + + +-- !query +select cast(null as decimal(1, 0)) div cast(5 as decimal(2, 0)) +-- !query schema +struct<(CAST(NULL AS DECIMAL(1,0)) div CAST(5 AS DECIMAL(2,0))):bigint> +-- !query output +NULL + + +-- !query +select 1 + 2 +-- !query schema +struct<(1 + 2):int> +-- !query output +3 + + +-- !query +select 1 - 2 +-- !query schema +struct<(1 - 2):int> +-- !query output +-1 + + +-- !query +select 2 * 5 +-- !query schema +struct<(2 * 5):int> +-- !query output +10 + + +-- !query +select 5 % 3 +-- !query schema +struct<(5 % 3):int> +-- !query output +2 + + +-- !query +select pmod(-7, 3) +-- !query schema +struct +-- !query output +2 + + +-- !query +select sec(1) +-- !query schema +struct +-- !query output +1.8508157176809255 + + +-- !query +select sec(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select sec(0) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +select sec(-1) +-- !query schema +struct +-- !query output +1.8508157176809255 + + +-- !query +select csc(1) +-- !query schema +struct +-- !query output +1.1883951057781212 + + +-- !query +select csc(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select csc(0) +-- !query schema +struct +-- !query output +Infinity + + +-- !query +select csc(-1) +-- !query schema +struct +-- !query output +-1.1883951057781212 + + +-- !query +select cot(1) +-- !query schema +struct +-- !query output +0.6420926159343306 + + +-- !query +select cot(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select cot(0) +-- !query schema +struct +-- !query output +Infinity + + +-- !query +select cot(-1) +-- !query schema +struct +-- !query output +-0.6420926159343306 + + +-- !query +select ceiling(0) +-- !query schema +struct +-- !query output +0 + + +-- !query +select ceiling(1) +-- !query schema +struct +-- !query output +1 + + +-- !query +select ceil(1234567890123456) +-- !query schema +struct +-- !query output +1234567890123456 + + +-- !query +select ceiling(1234567890123456) +-- !query schema +struct +-- !query output +1234567890123456 + + +-- !query +select ceil(0.01) +-- !query schema +struct +-- !query output +1 + + +-- !query +select ceiling(-0.10) +-- !query schema +struct +-- !query output +0 + + +-- !query +select floor(0) +-- !query schema +struct +-- !query output +0 + + +-- !query +select floor(1) +-- !query schema +struct +-- !query output +1 + + +-- !query +select floor(1234567890123456) +-- !query schema +struct +-- !query output +1234567890123456 + + +-- !query +select floor(0.01) +-- !query schema +struct +-- !query output +0 + + +-- !query +select floor(-0.10) +-- !query schema +struct +-- !query output +-1 + + +-- !query +select 1 > 0.00001 +-- !query schema +struct<(1 > 0):boolean> +-- !query output +true + + +-- !query +select mod(7, 2), mod(7, 0), mod(0, 2), mod(7, null), mod(null, 2), mod(null, null) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 27, + "fragment" : "mod(7, 0)" + } ] +} + + +-- !query +select BIT_LENGTH('abc') +-- !query schema +struct +-- !query output +24 + + +-- !query +select CHAR_LENGTH('abc') +-- !query schema +struct +-- !query output +3 + + +-- !query +select CHARACTER_LENGTH('abc') +-- !query schema +struct +-- !query output +3 + + +-- !query +select OCTET_LENGTH('abc') +-- !query schema +struct +-- !query output +3 + + +-- !query +select abs(-3.13), abs('-2.19') +-- !query schema +struct +-- !query output +3.13 2.19 + + +-- !query +select positive('-1.11'), positive(-1.11), negative('-1.11'), negative(-1.11) +-- !query schema +struct<(+ -1.11):double,(+ -1.11):decimal(3,2),negative(-1.11):double,negative(-1.11):decimal(3,2)> +-- !query output +-1.11 -1.11 1.11 1.11 + + +-- !query +select pmod(-7, 2), pmod(0, 2), pmod(7, 0), pmod(7, null), pmod(null, 2), pmod(null, null) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 33, + "stopIndex" : 42, + "fragment" : "pmod(7, 0)" + } ] +} + + +-- !query +select pmod(cast(3.13 as decimal), cast(0 as decimal)), pmod(cast(2 as smallint), cast(0 as smallint)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "pmod(cast(3.13 as decimal), cast(0 as decimal))" + } ] +} + + +-- !query +select width_bucket(5.35, 0.024, 10.06, 5) +-- !query schema +struct +-- !query output +3 + + +-- !query +select width_bucket(5.35, 0.024, 10.06, 3 + 2) +-- !query schema +struct +-- !query output +3 + + +-- !query +select width_bucket('5.35', '0.024', '10.06', '5') +-- !query schema +struct +-- !query output +3 + + +-- !query +select width_bucket(5.35, 0.024, 10.06, 2.5) +-- !query schema +struct +-- !query output +2 + + +-- !query +select width_bucket(5.35, 0.024, 10.06, 0.5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select width_bucket(null, 0.024, 10.06, 5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select width_bucket(5.35, null, 10.06, 5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select width_bucket(5.35, 0.024, null, -5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select width_bucket(5.35, 0.024, 10.06, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select width_bucket(5.35, 0.024, 10.06, -5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select width_bucket(5.35, 0.024, 10.06, 9223372036854775807L) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select width_bucket(5.35, 0.024, 10.06, 9223372036854775807L - 1) +-- !query schema +struct +-- !query output +4894746858139549697 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/order-by-all.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/order-by-all.sql.out new file mode 100644 index 000000000000..5ea0c2aa17c8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/order-by-all.sql.out @@ -0,0 +1,210 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view data as select * from values + (0, 1), + (0, 2), + (1, 3), + (1, NULL) + as data(g, i) +-- !query schema +struct<> +-- !query output + + + +-- !query +select g from data order by all +-- !query schema +struct +-- !query output +0 +0 +1 +1 + + +-- !query +select * from data order by all +-- !query schema +struct +-- !query output +0 1 +0 2 +1 NULL +1 3 + + +-- !query +select * from data order by aLl +-- !query schema +struct +-- !query output +0 1 +0 2 +1 NULL +1 3 + + +-- !query +select * from data order by all asc +-- !query schema +struct +-- !query output +0 1 +0 2 +1 NULL +1 3 + + +-- !query +select * from data order by all desc +-- !query schema +struct +-- !query output +1 3 +1 NULL +0 2 +0 1 + + +-- !query +select * from data order by all nulls first +-- !query schema +struct +-- !query output +0 1 +0 2 +1 NULL +1 3 + + +-- !query +select * from data order by all nulls last +-- !query schema +struct +-- !query output +0 1 +0 2 +1 3 +1 NULL + + +-- !query +select * from data order by all asc nulls first +-- !query schema +struct +-- !query output +0 1 +0 2 +1 NULL +1 3 + + +-- !query +select * from data order by all desc nulls first +-- !query schema +struct +-- !query output +1 NULL +1 3 +0 2 +0 1 + + +-- !query +select * from data order by all asc nulls last +-- !query schema +struct +-- !query output +0 1 +0 2 +1 3 +1 NULL + + +-- !query +select * from data order by all desc nulls last +-- !query schema +struct +-- !query output +1 3 +1 NULL +0 2 +0 1 + + +-- !query +select * from data union all select * from data order by all +-- !query schema +struct +-- !query output +0 1 +0 1 +0 2 +0 2 +1 NULL +1 NULL +1 3 +1 3 + + +-- !query +select * from data union select * from data order by all +-- !query schema +struct +-- !query output +0 1 +0 2 +1 NULL +1 3 + + +-- !query +select * from data order by all limit 2 +-- !query schema +struct +-- !query output +0 1 +0 2 + + +-- !query +select * from values("z", 1), ("y", 2), ("x", 3) AS T(col1, all) order by all +-- !query schema +struct +-- !query output +z 1 +y 2 +x 3 + + +-- !query +select name, dept, rank() over (partition by dept order by all) as rank +from values('Lisa', 'Sales', 10000, 35) as T(name, dept, salary, age) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`all`", + "proposal" : "`age`, `name`, `dept`, `salary`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 60, + "stopIndex" : 62, + "fragment" : "all" + } ] +} + + +-- !query +SELECT all, (SELECT col2 FROM VALUES (2, 2) ORDER BY ALL) FROM VALUES (1) AS t1 (all) +-- !query schema +struct +-- !query output +1 2 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/order-by-nulls-ordering.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/order-by-nulls-ordering.sql.out new file mode 100644 index 000000000000..4523c0d6dae9 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/order-by-nulls-ordering.sql.out @@ -0,0 +1,251 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create table spark_10747(col1 int, col2 int, col3 int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO spark_10747 VALUES (6, 12, 10), (6, 11, 4), (6, 9, 10), (6, 15, 8), +(6, 15, 8), (6, 7, 4), (6, 7, 8), (6, 13, null), (6, 10, null) +-- !query schema +struct<> +-- !query output + + + +-- !query +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 desc nulls last, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2 +-- !query schema +struct +-- !query output +6 9 10 28 +6 13 NULL 34 +6 10 NULL 41 +6 12 10 43 +6 15 8 55 +6 15 8 56 +6 11 4 56 +6 7 8 58 +6 7 4 58 + + +-- !query +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 desc nulls first, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2 +-- !query schema +struct +-- !query output +6 10 NULL 32 +6 11 4 33 +6 13 NULL 44 +6 7 4 48 +6 9 10 51 +6 15 8 55 +6 12 10 56 +6 15 8 56 +6 7 8 58 + + +-- !query +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 asc nulls last, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2 +-- !query schema +struct +-- !query output +6 7 4 25 +6 13 NULL 35 +6 11 4 40 +6 10 NULL 44 +6 7 8 55 +6 15 8 57 +6 15 8 58 +6 12 10 59 +6 9 10 61 + + +-- !query +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 asc nulls first, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2 +-- !query schema +struct +-- !query output +6 10 NULL 30 +6 12 10 36 +6 13 NULL 41 +6 7 4 48 +6 9 10 51 +6 11 4 53 +6 7 8 55 +6 15 8 57 +6 15 8 58 + + +-- !query +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 ASC NULLS FIRST, COL2 +-- !query schema +struct +-- !query output +6 10 NULL +6 13 NULL +6 7 4 +6 11 4 +6 7 8 +6 15 8 +6 15 8 +6 9 10 +6 12 10 + + +-- !query +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 NULLS LAST, COL2 +-- !query schema +struct +-- !query output +6 7 4 +6 11 4 +6 7 8 +6 15 8 +6 15 8 +6 9 10 +6 12 10 +6 10 NULL +6 13 NULL + + +-- !query +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS FIRST, COL2 +-- !query schema +struct +-- !query output +6 10 NULL +6 13 NULL +6 9 10 +6 12 10 +6 7 8 +6 15 8 +6 15 8 +6 7 4 +6 11 4 + + +-- !query +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS LAST, COL2 +-- !query schema +struct +-- !query output +6 9 10 +6 12 10 +6 7 8 +6 15 8 +6 15 8 +6 7 4 +6 11 4 +6 10 NULL +6 13 NULL + + +-- !query +drop table spark_10747 +-- !query schema +struct<> +-- !query output + + + +-- !query +create table spark_10747_mix( +col1 string, +col2 int, +col3 double, +col4 decimal(10,2), +col5 decimal(20,1)) +using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO spark_10747_mix VALUES +('b', 2, 1.0, 1.00, 10.0), +('d', 3, 2.0, 3.00, 0.0), +('c', 3, 2.0, 2.00, 15.1), +('d', 3, 0.0, 3.00, 1.0), +(null, 3, 0.0, 3.00, 1.0), +('d', 3, null, 4.00, 1.0), +('a', 1, 1.0, 1.00, null), +('c', 3, 2.0, 2.00, null) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from spark_10747_mix order by col1 nulls last, col5 nulls last +-- !query schema +struct +-- !query output +a 1 1.0 1.00 NULL +b 2 1.0 1.00 10.0 +c 3 2.0 2.00 15.1 +c 3 2.0 2.00 NULL +d 3 2.0 3.00 0.0 +d 3 0.0 3.00 1.0 +d 3 NULL 4.00 1.0 +NULL 3 0.0 3.00 1.0 + + +-- !query +select * from spark_10747_mix order by col1 desc nulls first, col5 desc nulls first +-- !query schema +struct +-- !query output +NULL 3 0.0 3.00 1.0 +d 3 0.0 3.00 1.0 +d 3 NULL 4.00 1.0 +d 3 2.0 3.00 0.0 +c 3 2.0 2.00 NULL +c 3 2.0 2.00 15.1 +b 2 1.0 1.00 10.0 +a 1 1.0 1.00 NULL + + +-- !query +select * from spark_10747_mix order by col5 desc nulls first, col3 desc nulls last +-- !query schema +struct +-- !query output +c 3 2.0 2.00 NULL +a 1 1.0 1.00 NULL +c 3 2.0 2.00 15.1 +b 2 1.0 1.00 10.0 +d 3 0.0 3.00 1.0 +NULL 3 0.0 3.00 1.0 +d 3 NULL 4.00 1.0 +d 3 2.0 3.00 0.0 + + +-- !query +drop table spark_10747_mix +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/order-by-ordinal.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/order-by-ordinal.sql.out new file mode 100644 index 000000000000..f975715c12bc --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/order-by-ordinal.sql.out @@ -0,0 +1,182 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from data order by 1 desc +-- !query schema +struct +-- !query output +3 1 +3 2 +2 1 +2 2 +1 1 +1 2 + + +-- !query +select * from data order by 1 desc, b desc +-- !query schema +struct +-- !query output +3 2 +3 1 +2 2 +2 1 +1 2 +1 1 + + +-- !query +select * from data order by 1 desc, 2 desc +-- !query schema +struct +-- !query output +3 2 +3 1 +2 2 +2 1 +1 2 +1 1 + + +-- !query +select * from data order by 1 + 0 desc, b desc +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +1 1 +2 1 +3 1 + + +-- !query +select * from data order by 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "ORDER_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "0", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 29, + "fragment" : "0" + } ] +} + + +-- !query +select * from data order by -1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "ORDER_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 30, + "fragment" : "-1" + } ] +} + + +-- !query +select * from data order by 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "ORDER_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "3", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 29, + "fragment" : "3" + } ] +} + + +-- !query +select * from data sort by 1 desc +-- !query schema +struct +-- !query output +1 1 +1 2 +2 1 +2 2 +3 1 +3 2 + + +-- !query +set spark.sql.orderByOrdinal=false +-- !query schema +struct +-- !query output +spark.sql.orderByOrdinal false + + +-- !query +select * from data order by 0 +-- !query schema +struct +-- !query output +1 1 +1 2 +2 1 +2 2 +3 1 +3 2 + + +-- !query +select * from data sort by 0 +-- !query schema +struct +-- !query output +1 1 +1 2 +2 1 +2 2 +3 1 +3 2 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/order-by.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/order-by.sql.out new file mode 100644 index 000000000000..786564fbc3d6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/order-by.sql.out @@ -0,0 +1,194 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT a FROM testData ORDER BY a +-- !query schema +struct +-- !query output +NULL +NULL +1 +1 +2 +2 +3 +3 +3 + + +-- !query +SELECT a FROM testData GROUP BY a, b ORDER BY a +-- !query schema +struct +-- !query output +NULL +NULL +1 +1 +2 +2 +3 +3 +3 + + +-- !query +SELECT b FROM testData WHERE a > 1 ORDER BY a +-- !query schema +struct +-- !query output +1 +2 +1 +2 +NULL + + +-- !query +SELECT b FROM testData GROUP BY a, b ORDER BY a +-- !query schema +struct +-- !query output +NULL +1 +2 +1 +1 +2 +1 +2 +NULL + + +-- !query +SELECT col1 FROM VALUES (1, named_struct('f1', 1)) ORDER BY col2.f1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT col1 FROM VALUES (1, named_struct('f1', named_struct('f2', 1))) ORDER BY col2.f1.f2 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT a, (SELECT b FROM testData GROUP BY b HAVING b > 1 ORDER BY a) FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"a ASC NULLS FIRST\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 68, + "fragment" : "ORDER BY a" + } ] +} + + +-- !query +SELECT a, (SELECT b FROM VALUES (1, 2) AS innerTestData (all, b) ORDER BY ALL) FROM testData +-- !query schema +struct +-- !query output +1 2 +1 2 +2 2 +2 2 +3 2 +3 2 +3 2 +NULL 2 +NULL 2 + + +-- !query +SELECT * FROM testData ORDER BY (SELECT a FROM testData ORDER BY b) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +{ + "errorClass" : "SCALAR_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 33, + "stopIndex" : 67, + "fragment" : "(SELECT a FROM testData ORDER BY b)" + } ] +} + + +-- !query +SELECT * FROM testData ORDER BY (SELECT * FROM testData ORDER BY (SELECT a FROM testData ORDER BY b)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_SUBQUERY_EXPRESSION.SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", + "sqlState" : "42823", + "messageParameters" : { + "number" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 33, + "stopIndex" : 101, + "fragment" : "(SELECT * FROM testData ORDER BY (SELECT a FROM testData ORDER BY b))" + } ] +} + + +-- !query +SELECT * FROM testData ORDER BY (SELECT a FROM VALUES (1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_SCALAR_SUBQUERY", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Sort [scalar-subquery#x [a#x] ASC NULLS FIRST], true\n: +- Project [outer(a#x)]\n: +- LocalRelation [col1#x]\n+- Project [a#x, b#x]\n +- SubqueryAlias testdata\n +- View (`testData`, [a#x, b#x])\n +- Project [cast(a#x as int) AS a#x, cast(b#x as int) AS b#x]\n +- Project [a#x, b#x]\n +- SubqueryAlias testData\n +- LocalRelation [a#x, b#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 58, + "fragment" : "ORDER BY (SELECT a FROM VALUES (1))" + } ] +} + + +-- !query +DROP VIEW IF EXISTS testData +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/outer-join.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/outer-join.sql.out new file mode 100644 index 000000000000..83983a68e638 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/outer-join.sql.out @@ -0,0 +1,69 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(-234), (145), (367), (975), (298) +as t1(int_col1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES +(-769, -244), (-800, -409), (940, 86), (-507, 304), (-367, 158) +as t2(int_col0, int_col1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + (SUM(COALESCE(t1.int_col1, t2.int_col0))), + ((COALESCE(t1.int_col1, t2.int_col0)) * 2) +FROM t1 +RIGHT JOIN t2 + ON (t2.int_col0) = (t1.int_col1) +GROUP BY GREATEST(COALESCE(t2.int_col1, 109), COALESCE(t1.int_col1, -449)), + COALESCE(t1.int_col1, t2.int_col0) +HAVING (SUM(COALESCE(t1.int_col1, t2.int_col0))) + > ((COALESCE(t1.int_col1, t2.int_col0)) * 2) +-- !query schema +struct +-- !query output +-367 -734 +-507 -1014 +-769 -1538 +-800 -1600 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (97) as t1(int_col1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (0) as t2(int_col1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * +FROM ( +SELECT + COALESCE(t2.int_col1, t1.int_col1) AS int_col + FROM t1 + LEFT JOIN t2 ON false +) t where (t.int_col) is not null +-- !query schema +struct +-- !query output +97 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/parse-query-correctness-old-behavior.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/parse-query-correctness-old-behavior.sql.out new file mode 100644 index 000000000000..6c9751864324 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/parse-query-correctness-old-behavior.sql.out @@ -0,0 +1,777 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num + UNION + SELECT 2 + UNION + SELECT 3 + UNION + SELECT 4 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 + UNION + SELECT 2 + UNION + SELECT 3 + UNION + SELECT 4 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "1", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE TEMPORARY VIEW", + "viewName" : "`v1`" + } +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1, 'a' + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "a", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE TEMPORARY VIEW", + "viewName" : "`v1`" + } +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num, 'a' + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "a", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE TEMPORARY VIEW", + "viewName" : "`v1`" + } +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1, 'a' AS a + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<1:int,a:string> +-- !query output +1 a +2 b + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num, 'a' AS a + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num + EXCEPT + SELECT 2 + EXCEPT + SELECT 1 + EXCEPT + SELECT 2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 + EXCEPT + SELECT 2 + EXCEPT + SELECT 1 + EXCEPT + SELECT 2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "1", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE TEMPORARY VIEW", + "viewName" : "`v1`" + } +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1, 'a' + EXCEPT + SELECT 2, 'b' + EXCEPT + SELECT 1, 'a' + EXCEPT + SELECT 2, 'b' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "a", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE TEMPORARY VIEW", + "viewName" : "`v1`" + } +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num, 'a' + EXCEPT + SELECT 2, 'b' + EXCEPT + SELECT 1, 'a' + EXCEPT + SELECT 2, 'b' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "a", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE TEMPORARY VIEW", + "viewName" : "`v1`" + } +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1, 'a' AS a + EXCEPT + SELECT 2, 'b' + EXCEPT + SELECT 1, 'a' + EXCEPT + SELECT 2, 'b' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<1:int,a:string> +-- !query output +1 a + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num, 'a' AS a + EXCEPT + SELECT 2, 'b' + EXCEPT + SELECT 1, 'a' + EXCEPT + SELECT 2, 'b' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output +1 a + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num + INTERSECT + SELECT 1 + INTERSECT + SELECT 2 + INTERSECT + SELECT 2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 + INTERSECT + SELECT 1 + INTERSECT + SELECT 2 + INTERSECT + SELECT 2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "1", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE TEMPORARY VIEW", + "viewName" : "`v1`" + } +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1, 'a' + INTERSECT + SELECT 1, 'a' + INTERSECT + SELECT 2, 'b' + INTERSECT + SELECT 2, 'b' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "a", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE TEMPORARY VIEW", + "viewName" : "`v1`" + } +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num, 'a' + INTERSECT + SELECT 1, 'a' + INTERSECT + SELECT 2, 'b' + INTERSECT + SELECT 2, 'b' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "a", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE TEMPORARY VIEW", + "viewName" : "`v1`" + } +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1, 'a' AS a + INTERSECT + SELECT 1, 'a' + INTERSECT + SELECT 2, 'b' + INTERSECT + SELECT 2, 'b' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<1:int,a:string> +-- !query output +1 a + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num, 'a' AS a + INTERSECT + SELECT 1, 'a' + INTERSECT + SELECT 2, 'b' + INTERSECT + SELECT 2, 'b' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output +1 a + + +-- !query +DROP VIEW v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v1 WITH SCHEMA EVOLUTION AS + SELECT 1, 'a' + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<1:int,UNION:string> +-- !query output +1 a + + +-- !query +CREATE OR REPLACE VIEW v1 WITH SCHEMA EVOLUTION AS + SELECT 1 AS num, 'a' + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output +1 a + + +-- !query +CREATE OR REPLACE VIEW v1 WITH SCHEMA EVOLUTION AS + SELECT 1 AS num, 'a' AS a + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +DROP VIEW v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE v1 INT +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 UNION SELECT 2 UNION SELECT 3' INTO v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 UNION SELECT 1 UNION SELECT 1' INTO v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 EXCEPT SELECT 2 EXCEPT SELECT 3' INTO v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 EXCEPT SELECT 1 EXCEPT SELECT 1' INTO v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 INTERSECT SELECT 2 INTERSECT SELECT 3' INTO v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 INTERSECT SELECT 1 INTERSECT SELECT 1' INTO v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 JOIN SELECT 2' INTO v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 VALUES (1)' INTO v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 alias garbage garbage garbage' INTO v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 WITH abc' INTO v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +DROP TEMPORARY VARIABLE v1 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/parse-query-correctness.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/parse-query-correctness.sql.out new file mode 100644 index 000000000000..1a16dd7564c0 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/parse-query-correctness.sql.out @@ -0,0 +1,765 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num + UNION + SELECT 2 + UNION + SELECT 3 + UNION + SELECT 4 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output +1 +2 +3 +4 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 + UNION + SELECT 2 + UNION + SELECT 3 + UNION + SELECT 4 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<1:int> +-- !query output +1 +2 +3 +4 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1, 'a' + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<1:int,a:string> +-- !query output +1 a +2 b +3 c +4 d + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num, 'a' + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output +1 a +2 b +3 c +4 d + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1, 'a' AS a + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<1:int,a:string> +-- !query output +1 a +2 b +3 c +4 d + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num, 'a' AS a + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output +1 a +2 b +3 c +4 d + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num + EXCEPT + SELECT 2 + EXCEPT + SELECT 1 + EXCEPT + SELECT 2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 + EXCEPT + SELECT 2 + EXCEPT + SELECT 1 + EXCEPT + SELECT 2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<1:int> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1, 'a' + EXCEPT + SELECT 2, 'b' + EXCEPT + SELECT 1, 'a' + EXCEPT + SELECT 2, 'b' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<1:int,a:string> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num, 'a' + EXCEPT + SELECT 2, 'b' + EXCEPT + SELECT 1, 'a' + EXCEPT + SELECT 2, 'b' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1, 'a' AS a + EXCEPT + SELECT 2, 'b' + EXCEPT + SELECT 1, 'a' + EXCEPT + SELECT 2, 'b' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<1:int,a:string> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num, 'a' AS a + EXCEPT + SELECT 2, 'b' + EXCEPT + SELECT 1, 'a' + EXCEPT + SELECT 2, 'b' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num + INTERSECT + SELECT 1 + INTERSECT + SELECT 2 + INTERSECT + SELECT 2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 + INTERSECT + SELECT 1 + INTERSECT + SELECT 2 + INTERSECT + SELECT 2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<1:int> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1, 'a' + INTERSECT + SELECT 1, 'a' + INTERSECT + SELECT 2, 'b' + INTERSECT + SELECT 2, 'b' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<1:int,a:string> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num, 'a' + INTERSECT + SELECT 1, 'a' + INTERSECT + SELECT 2, 'b' + INTERSECT + SELECT 2, 'b' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1, 'a' AS a + INTERSECT + SELECT 1, 'a' + INTERSECT + SELECT 2, 'b' + INTERSECT + SELECT 2, 'b' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<1:int,a:string> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v1 AS + SELECT 1 AS num, 'a' AS a + INTERSECT + SELECT 1, 'a' + INTERSECT + SELECT 2, 'b' + INTERSECT + SELECT 2, 'b' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output + + + +-- !query +DROP VIEW v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v1 WITH SCHEMA EVOLUTION AS + SELECT 1, 'a' + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct<1:int,a:string> +-- !query output +1 a +2 b +3 c +4 d + + +-- !query +CREATE OR REPLACE VIEW v1 WITH SCHEMA EVOLUTION AS + SELECT 1 AS num, 'a' + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output +1 a +2 b +3 c +4 d + + +-- !query +CREATE OR REPLACE VIEW v1 WITH SCHEMA EVOLUTION AS + SELECT 1 AS num, 'a' AS a + UNION + SELECT 2, 'b' + UNION + SELECT 3, 'c' + UNION + SELECT 4, 'd' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v1 +-- !query schema +struct +-- !query output +1 a +2 b +3 c +4 d + + +-- !query +DROP VIEW v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE v1 INT +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 UNION SELECT 2 UNION SELECT 3' INTO v1 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +{ + "errorClass" : "ROW_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000" +} + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +NULL + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 UNION SELECT 1 UNION SELECT 1' INTO v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 EXCEPT SELECT 2 EXCEPT SELECT 3' INTO v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 EXCEPT SELECT 1 EXCEPT SELECT 1' INTO v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +NULL + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 INTERSECT SELECT 2 INTERSECT SELECT 3' INTO v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +NULL + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 INTERSECT SELECT 1 INTERSECT SELECT 1' INTO v1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 JOIN SELECT 2' INTO v1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'SELECT'", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 50, + "fragment" : "EXECUTE IMMEDIATE 'SELECT 1 JOIN SELECT 2' INTO v1" + } ] +} + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 VALUES (1)' INTO v1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 47, + "fragment" : "EXECUTE IMMEDIATE 'SELECT 1 VALUES (1)' INTO v1" + } ] +} + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 alias garbage garbage garbage' INTO v1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'garbage'", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 66, + "fragment" : "EXECUTE IMMEDIATE 'SELECT 1 alias garbage garbage garbage' INTO v1" + } ] +} + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +EXECUTE IMMEDIATE 'SELECT 1 WITH abc' INTO v1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'abc'", + "hint" : ": extra input 'abc'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 45, + "fragment" : "EXECUTE IMMEDIATE 'SELECT 1 WITH abc' INTO v1" + } ] +} + + +-- !query +SELECT v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +DROP TEMPORARY VARIABLE v1 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/parse-schema-string.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/parse-schema-string.sql.out new file mode 100644 index 000000000000..793b545b3241 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/parse-schema-string.sql.out @@ -0,0 +1,31 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select from_csv('1', 'create INT') +-- !query schema +struct> +-- !query output +{"create":1} + + +-- !query +select from_csv('1', 'cube INT') +-- !query schema +struct> +-- !query output +{"cube":1} + + +-- !query +select from_json('{"create":1}', 'create INT') +-- !query schema +struct> +-- !query output +{"create":1} + + +-- !query +select from_json('{"cube":1}', 'cube INT') +-- !query schema +struct> +-- !query output +{"cube":1} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/percentiles.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/percentiles.sql.out new file mode 100644 index 000000000000..55aaa8ee7378 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/percentiles.sql.out @@ -0,0 +1,1190 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW aggr AS SELECT * FROM VALUES +(0, 0), (0, 10), (0, 20), (0, 30), (0, 40), (1, 10), (1, 20), (2, 10), (2, 20), (2, 25), (2, 30), (3, 60), (4, null) +AS aggr(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW basic_pays AS SELECT * FROM VALUES +('Diane Murphy','Accounting',8435), +('Mary Patterson','Accounting',9998), +('Jeff Firrelli','Accounting',8992), +('William Patterson','Accounting',8870), +('Gerard Bondur','Accounting',11472), +('Anthony Bow','Accounting',6627), +('Leslie Jennings','IT',8113), +('Leslie Thompson','IT',5186), +('Julie Firrelli','Sales',9181), +('Steve Patterson','Sales',9441), +('Foon Yue Tseng','Sales',6660), +('George Vanauf','Sales',10563), +('Loui Bondur','SCM',10449), +('Gerard Hernandez','SCM',6949), +('Pamela Castillo','SCM',11303), +('Larry Bott','SCM',11798), +('Barry Jones','SCM',10586) +AS basic_pays(employee_name, department, salary) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + percentile_cont(0.25) WITHIN GROUP (ORDER BY v), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr +-- !query schema +struct 0)):double,percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC):double,percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE (k > 0)):double> +-- !query output +10.0 15.0 30.0 27.5 + + +-- !query +SELECT + k, + percentile_cont(0.25) WITHIN GROUP (ORDER BY v), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr +GROUP BY k +ORDER BY k +-- !query schema +struct 0)):double,percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC):double,percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE (k > 0)):double> +-- !query output +0 10.0 NULL 30.0 NULL +1 12.5 12.5 17.5 17.5 +2 17.5 17.5 26.25 26.25 +3 60.0 60.0 60.0 60.0 +4 NULL NULL NULL NULL + + +-- !query +SELECT + percentile_disc(0.25) WITHIN GROUP (ORDER BY v), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr +-- !query schema +struct 0)):double,percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC):double,percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE (k > 0)):double> +-- !query output +10.0 10.0 30.0 30.0 + + +-- !query +SELECT + k, + percentile_disc(0.25) WITHIN GROUP (ORDER BY v), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC), + percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0) +FROM aggr +GROUP BY k +ORDER BY k +-- !query schema +struct 0)):double,percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC):double,percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE (k > 0)):double> +-- !query output +0 10.0 NULL 30.0 NULL +1 10.0 10.0 20.0 20.0 +2 10.0 10.0 30.0 30.0 +3 60.0 60.0 60.0 60.0 +4 NULL NULL NULL NULL + + +-- !query +SELECT + median(v), + percentile(v, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY v) +FROM aggr +-- !query schema +struct +-- !query output +20.0 20.0 20.0 + + +-- !query +SELECT + round(v, 0) WITHIN GROUP (ORDER BY v) +FROM aggr +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", + "sqlState" : "42000", + "messageParameters" : { + "prettyName" : "`round`", + "syntax" : "WITHIN GROUP (ORDER BY ...)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 46, + "fragment" : "round(v, 0) WITHIN GROUP (ORDER BY v)" + } ] +} + + +-- !query +SELECT + round(v, 0) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY k) +FROM aggr +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", + "sqlState" : "42000", + "messageParameters" : { + "prettyName" : "`round`", + "syntax" : "WITHIN GROUP (ORDER BY ...)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 68, + "fragment" : "round(v, 0) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY k)" + } ] +} + + +-- !query +SELECT + percentile(v, 0.5) WITHIN GROUP (ORDER BY v) +FROM aggr +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", + "sqlState" : "42000", + "messageParameters" : { + "prettyName" : "`percentile`", + "syntax" : "WITHIN GROUP (ORDER BY ...)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 53, + "fragment" : "percentile(v, 0.5) WITHIN GROUP (ORDER BY v)" + } ] +} + + +-- !query +SELECT + percentile(v, 0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY k) +FROM aggr +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", + "sqlState" : "42000", + "messageParameters" : { + "prettyName" : "`percentile`", + "syntax" : "WITHIN GROUP (ORDER BY ...)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 75, + "fragment" : "percentile(v, 0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY k)" + } ] +} + + +-- !query +SELECT + percentile_cont(DISTINCT 0.5) WITHIN GROUP (ORDER BY v) +FROM aggr +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.DISTINCT_UNSUPPORTED", + "sqlState" : "42K0K", + "messageParameters" : { + "funcName" : "`percentile_cont`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 64, + "fragment" : "percentile_cont(DISTINCT 0.5) WITHIN GROUP (ORDER BY v)" + } ] +} + + +-- !query +SELECT + percentile_cont(DISTINCT 0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY k) +FROM aggr +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.DISTINCT_UNSUPPORTED", + "sqlState" : "42K0K", + "messageParameters" : { + "funcName" : "`percentile_cont`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 86, + "fragment" : "percentile_cont(DISTINCT 0.5) WITHIN GROUP (ORDER BY v) OVER (PARTITION BY k)" + } ] +} + + +-- !query +SELECT + percentile_cont() WITHIN GROUP (ORDER BY v) +FROM aggr +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`percentile_cont`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 52, + "fragment" : "percentile_cont() WITHIN GROUP (ORDER BY v)" + } ] +} + + +-- !query +SELECT + percentile_cont() WITHIN GROUP (ORDER BY v) OVER (PARTITION BY k) +FROM aggr +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`percentile_cont`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 74, + "fragment" : "percentile_cont() WITHIN GROUP (ORDER BY v) OVER (PARTITION BY k)" + } ] +} + + +-- !query +SELECT + percentile_cont(0.5) +FROM aggr +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.WITHIN_GROUP_MISSING", + "sqlState" : "42K0K", + "messageParameters" : { + "funcName" : "`percentile_cont`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 29, + "fragment" : "percentile_cont(0.5)" + } ] +} + + +-- !query +SELECT + percentile_cont(0.5) OVER (PARTITION BY k) +FROM aggr +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.WITHIN_GROUP_MISSING", + "sqlState" : "42K0K", + "messageParameters" : { + "funcName" : "`percentile_cont`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 51, + "fragment" : "percentile_cont(0.5) OVER (PARTITION BY k)" + } ] +} + + +-- !query +SELECT + percentile_cont(0.5) WITHIN GROUP (ORDER BY k, v) +FROM aggr +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_WITHIN_GROUP_EXPRESSION.WRONG_NUM_ORDERINGS", + "sqlState" : "42K0K", + "messageParameters" : { + "actualNum" : "2", + "expectedNum" : "1", + "funcName" : "`percentile_cont`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 58, + "fragment" : "percentile_cont(0.5) WITHIN GROUP (ORDER BY k, v)" + } ] +} + + +-- !query +SELECT + percentile_cont(k, 0.5) WITHIN GROUP (ORDER BY v) +FROM aggr +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`percentile_cont`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 58, + "fragment" : "percentile_cont(k, 0.5) WITHIN GROUP (ORDER BY v)" + } ] +} + + +-- !query +SELECT + k, + median(v), + percentile(v, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY v) +FROM aggr +GROUP BY k +ORDER BY k +-- !query schema +struct +-- !query output +0 20.0 20.0 20.0 +1 15.0 15.0 15.0 +2 22.5 22.5 22.5 +3 60.0 60.0 60.0 +4 NULL NULL NULL + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department), + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department) +FROM basic_pays +ORDER BY salary +-- !query schema +struct +-- !query output +Leslie Thompson IT 5186 5917.75 5186.0 7381.25 8113.0 +Anthony Bow Accounting 6627 8543.75 8435.0 9746.5 9998.0 +Foon Yue Tseng Sales 6660 8550.75 6660.0 9721.5 10563.0 +Gerard Hernandez SCM 6949 10449.0 10449.0 11303.0 11303.0 +Leslie Jennings IT 8113 5917.75 5186.0 7381.25 8113.0 +Diane Murphy Accounting 8435 8543.75 8435.0 9746.5 9998.0 +William Patterson Accounting 8870 8543.75 8435.0 9746.5 9998.0 +Jeff Firrelli Accounting 8992 8543.75 8435.0 9746.5 9998.0 +Julie Firrelli Sales 9181 8550.75 6660.0 9721.5 10563.0 +Steve Patterson Sales 9441 8550.75 6660.0 9721.5 10563.0 +Mary Patterson Accounting 9998 8543.75 8435.0 9746.5 9998.0 +Loui Bondur SCM 10449 10449.0 10449.0 11303.0 11303.0 +George Vanauf Sales 10563 8550.75 6660.0 9721.5 10563.0 +Barry Jones SCM 10586 10449.0 10449.0 11303.0 11303.0 +Pamela Castillo SCM 11303 10449.0 10449.0 11303.0 11303.0 +Gerard Bondur Accounting 11472 8543.75 8435.0 9746.5 9998.0 +Larry Bott SCM 11798 10449.0 10449.0 11303.0 11303.0 + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary), + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ORDER BY salary) +FROM basic_pays +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "sqlState" : "42601", + "messageParameters" : { + "aggFunc" : "\"percentile_cont(salary, 0.25)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 157, + "fragment" : "percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary)" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ORDER BY salary) +FROM basic_pays +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "sqlState" : "42601", + "messageParameters" : { + "aggFunc" : "\"percentile_disc(salary, 0.25)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 157, + "fragment" : "percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary)" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + median(salary) OVER (PARTITION BY department ORDER BY salary) +FROM basic_pays +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "sqlState" : "42601", + "messageParameters" : { + "aggFunc" : "\"median(salary)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 119, + "fragment" : "median(salary) OVER (PARTITION BY department ORDER BY salary)" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING), + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +FROM basic_pays +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "sqlState" : "42601", + "messageParameters" : { + "aggFunc" : "\"percentile_cont(salary, 0.25)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 190, + "fragment" : "percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING), + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +FROM basic_pays +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "sqlState" : "42601", + "messageParameters" : { + "aggFunc" : "\"percentile_disc(salary, 0.25)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 190, + "fragment" : "percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +FROM basic_pays +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "sqlState" : "42601", + "messageParameters" : { + "aggFunc" : "\"median(salary)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 152, + "fragment" : "median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department) +ORDER BY salary +-- !query schema +struct +-- !query output +Leslie Thompson IT 5186 5917.75 5186.0 7381.25 8113.0 +Anthony Bow Accounting 6627 8543.75 8435.0 9746.5 9998.0 +Foon Yue Tseng Sales 6660 8550.75 6660.0 9721.5 10563.0 +Gerard Hernandez SCM 6949 10449.0 10449.0 11303.0 11303.0 +Leslie Jennings IT 8113 5917.75 5186.0 7381.25 8113.0 +Diane Murphy Accounting 8435 8543.75 8435.0 9746.5 9998.0 +William Patterson Accounting 8870 8543.75 8435.0 9746.5 9998.0 +Jeff Firrelli Accounting 8992 8543.75 8435.0 9746.5 9998.0 +Julie Firrelli Sales 9181 8550.75 6660.0 9721.5 10563.0 +Steve Patterson Sales 9441 8550.75 6660.0 9721.5 10563.0 +Mary Patterson Accounting 9998 8543.75 8435.0 9746.5 9998.0 +Loui Bondur SCM 10449 10449.0 10449.0 11303.0 11303.0 +George Vanauf Sales 10563 8550.75 6660.0 9721.5 10563.0 +Barry Jones SCM 10586 10449.0 10449.0 11303.0 11303.0 +Pamela Castillo SCM 11303 10449.0 10449.0 11303.0 11303.0 +Gerard Bondur Accounting 11472 8543.75 8435.0 9746.5 9998.0 +Larry Bott SCM 11798 10449.0 10449.0 11303.0 11303.0 + + +-- !query +SELECT + employee_name, + department, + salary, + median(salary) OVER w, + percentile_cont(0.5) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.5) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.5) WITHIN GROUP (ORDER BY salary DESC) OVER w, + percentile_disc(0.5) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WHERE salary > 8900 +WINDOW w AS (PARTITION BY department) +ORDER BY salary +-- !query schema +struct +-- !query output +Jeff Firrelli Accounting 8992 9998.0 9998.0 9998.0 9998.0 9998.0 +Julie Firrelli Sales 9181 9441.0 9441.0 9441.0 9441.0 9441.0 +Steve Patterson Sales 9441 9441.0 9441.0 9441.0 9441.0 9441.0 +Mary Patterson Accounting 9998 9998.0 9998.0 9998.0 9998.0 9998.0 +Loui Bondur SCM 10449 10944.5 10944.5 10586.0 10944.5 11303.0 +George Vanauf Sales 10563 9441.0 9441.0 9441.0 9441.0 9441.0 +Barry Jones SCM 10586 10944.5 10944.5 10586.0 10944.5 11303.0 +Pamela Castillo SCM 11303 10944.5 10944.5 10586.0 10944.5 11303.0 +Gerard Bondur Accounting 11472 9998.0 9998.0 9998.0 9998.0 9998.0 +Larry Bott SCM 11798 10944.5 10944.5 10586.0 10944.5 11303.0 + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ORDER BY salary) +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "sqlState" : "42601", + "messageParameters" : { + "aggFunc" : "\"percentile_cont(salary, 0.25)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 117, + "fragment" : "percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ORDER BY salary) +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "sqlState" : "42601", + "messageParameters" : { + "aggFunc" : "\"percentile_disc(salary, 0.25)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 117, + "fragment" : "percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + median(salary) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ORDER BY salary) +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "sqlState" : "42601", + "messageParameters" : { + "aggFunc" : "\"median(salary)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 79, + "fragment" : "median(salary) OVER w" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "sqlState" : "42601", + "messageParameters" : { + "aggFunc" : "\"percentile_cont(salary, 0.25)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 117, + "fragment" : "percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w, + percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "sqlState" : "42601", + "messageParameters" : { + "aggFunc" : "\"percentile_disc(salary, 0.25)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 117, + "fragment" : "percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w" + } ] +} + + +-- !query +SELECT + employee_name, + department, + salary, + median(salary) OVER w +FROM basic_pays +WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WINDOW_SPEC_FOR_AGGREGATION_FUNC", + "sqlState" : "42601", + "messageParameters" : { + "aggFunc" : "\"median(salary)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 59, + "stopIndex" : 79, + "fragment" : "median(salary) OVER w" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW intervals AS SELECT * FROM VALUES +(0, INTERVAL '0' MONTH, INTERVAL '0' SECOND, INTERVAL '0' MINUTE), +(0, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(0, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(0, INTERVAL '30' MONTH, INTERVAL '30' SECOND, INTERVAL '30' MINUTE), +(0, INTERVAL '40' MONTH, INTERVAL '40' SECOND, INTERVAL '40' MINUTE), +(1, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(1, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(2, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE), +(2, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE), +(2, INTERVAL '25' MONTH, INTERVAL '25' SECOND, INTERVAL '25' MINUTE), +(2, INTERVAL '30' MONTH, INTERVAL '30' SECOND, INTERVAL '30' MINUTE), +(3, INTERVAL '60' MONTH, INTERVAL '60' SECOND, INTERVAL '60' MINUTE), +(4, null, null, null) +AS intervals(k, dt, ym, dt2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt), + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt DESC) +FROM intervals +-- !query schema +struct +-- !query output +0-10 2-6 + + +-- !query +SELECT + k, + percentile_cont(0.25) WITHIN GROUP (ORDER BY ym), + percentile_cont(0.25) WITHIN GROUP (ORDER BY ym DESC) +FROM intervals +GROUP BY k +ORDER BY k +-- !query schema +struct +-- !query output +0 0 00:00:10.000000000 0 00:00:30.000000000 +1 0 00:00:12.500000000 0 00:00:17.500000000 +2 0 00:00:17.500000000 0 00:00:26.250000000 +3 0 00:01:00.000000000 0 00:01:00.000000000 +4 NULL NULL + + +-- !query +SELECT + k, + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt2), + percentile_cont(0.25) WITHIN GROUP (ORDER BY dt2 DESC) +FROM intervals +GROUP BY k +ORDER BY k +-- !query schema +struct +-- !query output +0 0 00:10:00.000000000 0 00:30:00.000000000 +1 0 00:12:30.000000000 0 00:17:30.000000000 +2 0 00:17:30.000000000 0 00:26:15.000000000 +3 0 01:00:00.000000000 0 01:00:00.000000000 +4 NULL NULL + + +-- !query +SELECT + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt), + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt DESC) +FROM intervals +-- !query schema +struct +-- !query output +0-10 2-6 + + +-- !query +SELECT + k, + percentile_disc(0.25) WITHIN GROUP (ORDER BY ym), + percentile_disc(0.25) WITHIN GROUP (ORDER BY ym DESC) +FROM intervals +GROUP BY k +ORDER BY k +-- !query schema +struct +-- !query output +0 0 00:00:10.000000000 0 00:00:30.000000000 +1 0 00:00:10.000000000 0 00:00:20.000000000 +2 0 00:00:10.000000000 0 00:00:30.000000000 +3 0 00:01:00.000000000 0 00:01:00.000000000 +4 NULL NULL + + +-- !query +SELECT + k, + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt2), + percentile_disc(0.25) WITHIN GROUP (ORDER BY dt2 DESC) +FROM intervals +GROUP BY k +ORDER BY k +-- !query schema +struct +-- !query output +0 0 00:10:00.000000000 0 00:30:00.000000000 +1 0 00:10:00.000000000 0 00:20:00.000000000 +2 0 00:10:00.000000000 0 00:30:00.000000000 +3 0 01:00:00.000000000 0 01:00:00.000000000 +4 NULL NULL + + +-- !query +SELECT + median(dt), + percentile(dt, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY dt) +FROM intervals +-- !query schema +struct +-- !query output +1-8 1-8 1-8 + + +-- !query +SELECT + k, + median(ym), + percentile(ym, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY ym) +FROM intervals +GROUP BY k +ORDER BY k +-- !query schema +struct +-- !query output +0 0 00:00:20.000000000 0 00:00:20.000000000 0 00:00:20.000000000 +1 0 00:00:15.000000000 0 00:00:15.000000000 0 00:00:15.000000000 +2 0 00:00:22.500000000 0 00:00:22.500000000 0 00:00:22.500000000 +3 0 00:01:00.000000000 0 00:01:00.000000000 0 00:01:00.000000000 +4 NULL NULL NULL + + +-- !query +SELECT + k, + median(dt2), + percentile(dt2, 0.5), + percentile_cont(0.5) WITHIN GROUP (ORDER BY dt2) +FROM intervals +GROUP BY k +ORDER BY k +-- !query schema +struct +-- !query output +0 0 00:20:00.000000000 0 00:20:00.000000000 0 00:20:00.000000000 +1 0 00:15:00.000000000 0 00:15:00.000000000 0 00:15:00.000000000 +2 0 00:22:30.000000000 0 00:22:30.000000000 0 00:22:30.000000000 +3 0 01:00:00.000000000 0 01:00:00.000000000 0 01:00:00.000000000 +4 NULL NULL NULL + + +-- !query +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0) AS v(a) +-- !query schema +struct +-- !query output +0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 + + +-- !query +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0), (1) AS v(a) +-- !query schema +struct +-- !query output +0.0 0.0 0.0 0.0 0.0 0.0 1.0 1.0 1.0 1.0 1.0 + + +-- !query +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0), (1), (2) AS v(a) +-- !query schema +struct +-- !query output +0.0 0.0 0.0 0.0 1.0 1.0 1.0 2.0 2.0 2.0 2.0 + + +-- !query +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0), (1), (2), (3), (4) AS v(a) +-- !query schema +struct +-- !query output +0.0 0.0 0.0 1.0 1.0 2.0 2.0 3.0 3.0 4.0 4.0 + + +-- !query +SET spark.sql.legacy.percentileDiscCalculation = true +-- !query schema +struct +-- !query output +spark.sql.legacy.percentileDiscCalculation true + + +-- !query +SELECT + percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0, + percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1, + percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2, + percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3, + percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4, + percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5, + percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6, + percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7, + percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8, + percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9, + percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10 +FROM VALUES (0), (1), (2), (3), (4) AS v(a) +-- !query schema +struct +-- !query output +0.0 0.0 0.0 1.0 1.0 2.0 2.0 2.0 3.0 3.0 4.0 + + +-- !query +SELECT + percentile_cont(b) WITHIN GROUP (ORDER BY a DESC) as p0 +FROM values (12, 0.25), (13, 0.25), (22, 0.25) as v(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"b\"", + "inputName" : "`percentage`", + "inputType" : "\"DOUBLE\"", + "sqlExpr" : "\"percentile_cont(a, b)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 58, + "fragment" : "percentile_cont(b) WITHIN GROUP (ORDER BY a DESC)" + } ] +} + + +-- !query +SET spark.sql.legacy.percentileDiscCalculation = false +-- !query schema +struct +-- !query output +spark.sql.legacy.percentileDiscCalculation false diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/pipe-operators.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/pipe-operators.sql.out new file mode 100644 index 000000000000..bf7453361276 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/pipe-operators.sql.out @@ -0,0 +1,4276 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +drop table if exists t +-- !query schema +struct<> +-- !query output + + + +-- !query +create table t(x int, y string) using csv +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t values (0, 'abc'), (1, 'def') +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table if exists other +-- !query schema +struct<> +-- !query output + + + +-- !query +create table other(a int, b int) using json +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into other values (1, 1), (1, 2), (2, 4) +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table if exists st +-- !query schema +struct<> +-- !query output + + + +-- !query +create table st(x int, col struct) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into st values (1, (2, 3)) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view courseSales as select * from values + ("dotNET", 2012, 10000), + ("Java", 2012, 20000), + ("dotNET", 2012, 5000), + ("dotNET", 2013, 48000), + ("Java", 2013, 30000) + as courseSales(course, year, earnings) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view courseEarnings as select * from values + ("dotNET", 15000, 48000, 22500), + ("Java", 20000, 30000, NULL) + as courseEarnings(course, `2012`, `2013`, `2014`) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view courseEarningsAndSales as select * from values + ("dotNET", 15000, NULL, 48000, 1, 22500, 1), + ("Java", 20000, 1, 30000, 2, NULL, NULL) + as courseEarningsAndSales( + course, earnings2012, sales2012, earnings2013, sales2013, earnings2014, sales2014) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view yearsWithComplexTypes as select * from values + (2012, array(1, 1), map('1', 1), struct(1, 'a')), + (2013, array(2, 2), map('2', 2), struct(2, 'b')) + as yearsWithComplexTypes(y, a, m, s) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view join_test_t1 as select * from values (1) as grouping(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view join_test_t2 as select * from values (1) as grouping(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view join_test_t3 as select * from values (1) as grouping(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view join_test_empty_table as select a from join_test_t2 where false +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view lateral_test_t1(c1, c2) + as values (0, 1), (1, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view lateral_test_t2(c1, c2) + as values (0, 2), (0, 3) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view lateral_test_t3(c1, c2) + as values (0, array(0, 1)), (1, array(2)), (2, array()), (null, array(4)) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view lateral_test_t4(c1, c2) + as values (0, 1), (0, 2), (1, 1), (1, 3) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view natural_join_test_t1 as select * from values + ("one", 1), ("two", 2), ("three", 3) as natural_join_test_t1(k, v1) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view natural_join_test_t2 as select * from values + ("one", 1), ("two", 22), ("one", 5) as natural_join_test_t2(k, v2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view natural_join_test_t3 as select * from values + ("one", 4), ("two", 5), ("one", 6) as natural_join_test_t3(k, v3) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view windowTestData as select * from values + (null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), + (1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), + (1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), + (2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), + (1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), + (2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), + (3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), + (null, null, null, null, null, null), + (3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) + AS testData(val, val_long, val_double, val_date, val_timestamp, cate) +-- !query schema +struct<> +-- !query output + + + +-- !query +from t +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +from t +|> select 1 as x +-- !query schema +struct +-- !query output +1 +1 + + +-- !query +from t as t_alias +|> select t_alias.x +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +from t as t_alias +|> select t_alias.x as tx, t_alias.y as ty +|> where ty = 'def' +|> select tx +-- !query schema +struct +-- !query output +1 + + +-- !query +from t, other +|> select t.x + other.a as z +-- !query schema +struct +-- !query output +1 +1 +2 +2 +2 +3 + + +-- !query +from t join other on (t.x = other.a) +|> select t.x + other.a as z +-- !query schema +struct +-- !query output +2 +2 + + +-- !query +from t lateral view explode(array(100, 101)) as ly +|> select t.x + ly as z +-- !query schema +struct +-- !query output +100 +101 +101 +102 + + +-- !query +from st +|> select col.i1 +-- !query schema +struct +-- !query output +2 + + +-- !query +from st as st_alias +|> select st_alias.col.i1 +-- !query schema +struct +-- !query output +2 + + +-- !query +from values (0), (1) tab(col) +|> select col as x +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +from t +|> from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'from'", + "hint" : "" + } +} + + +-- !query +table t +|> select 1 as x +-- !query schema +struct +-- !query output +1 +1 + + +-- !query +table t +|> select x, y +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> select x, y +|> select x + length(y) as z +-- !query schema +struct +-- !query output +3 +4 + + +-- !query +values (0), (1) tab(col) +|> select col * 2 as result +-- !query schema +struct +-- !query output +0 +2 + + +-- !query +(select * from t union all select * from t) +|> select x + length(y) as result +-- !query schema +struct +-- !query output +3 +3 +4 +4 + + +-- !query +(table t + |> select x, y + |> select x) +union all +select x from t where x < 1 +-- !query schema +struct +-- !query output +0 +0 +1 + + +-- !query +(select col from st) +|> select col.i1 +-- !query schema +struct +-- !query output +2 + + +-- !query +table st +|> select st.col.i1 +-- !query schema +struct +-- !query output +2 + + +-- !query +table t +|> select (select a from other where x = a limit 1) as result +-- !query schema +struct +-- !query output +1 +NULL + + +-- !query +select (values (0) tab(col) |> select col) as result +-- !query schema +struct +-- !query output +0 + + +-- !query +table t +|> select (select any_value(a) from other where x = a limit 1) as result +-- !query schema +struct +-- !query output +1 +NULL + + +-- !query +table t +|> select x + length(x) as z, z + 1 as plus_one +-- !query schema +struct +-- !query output +1 2 +2 3 + + +-- !query +table t +|> select first_value(x) over (partition by y) as result +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +select 1 x, 2 y, 3 z +|> select 1 + sum(x) over (), + avg(y) over (), + x, + avg(x+1) over (partition by y order by z) AS a2 +|> select a2 +-- !query schema +struct +-- !query output +2.0 + + +-- !query +table t +|> select x, count(*) over () +|> select x +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +table t +|> select distinct x, y +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> select * +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> select * except (y) +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +table t +|> select /*+ repartition(3) */ * +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> select /*+ repartition(3) */ distinct x +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +table t +|> select /*+ repartition(3) */ all x +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +table t +|> select sum(x) as result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PIPE_OPERATOR_CONTAINS_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "clause" : "SELECT", + "expr" : "sum(x#x)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 24, + "fragment" : "sum(x)" + } ] +} + + +-- !query +table t +|> select y, length(y) + sum(x) as result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PIPE_OPERATOR_CONTAINS_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "clause" : "SELECT", + "expr" : "sum(x#x)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 39, + "fragment" : "sum(x)" + } ] +} + + +-- !query +from t +|> select sum(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PIPE_OPERATOR_CONTAINS_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "clause" : "SELECT", + "expr" : "sum(x#x)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 18, + "stopIndex" : 23, + "fragment" : "sum(x)" + } ] +} + + +-- !query +from t as t_alias +|> select y, sum(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PIPE_OPERATOR_CONTAINS_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "clause" : "SELECT", + "expr" : "sum(x#x)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 37, + "fragment" : "sum(x)" + } ] +} + + +-- !query +from t as t_alias +|> select y, sum(x) group by y +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'group'", + "hint" : "" + } +} + + +-- !query +table t +|> extend 1 as z +-- !query schema +struct +-- !query output +0 abc 1 +1 def 1 + + +-- !query +table t +|> extend 1 +-- !query schema +struct +-- !query output +0 abc 1 +1 def 1 + + +-- !query +table t +|> extend x as z +-- !query schema +struct +-- !query output +0 abc 0 +1 def 1 + + +-- !query +table t +|> extend x + length(y) as z +-- !query schema +struct +-- !query output +0 abc 3 +1 def 4 + + +-- !query +table t +|> extend x + length(y) as z, x + 1 as zz +-- !query schema +struct +-- !query output +0 abc 3 1 +1 def 4 2 + + +-- !query +table t +|> extend x + length(y) as z +|> extend z + 1 as zz +-- !query schema +struct +-- !query output +0 abc 3 4 +1 def 4 5 + + +-- !query +select col from st +|> extend col.i1 as z +-- !query schema +struct,z:int> +-- !query output +{"i1":2,"i2":3} 2 + + +-- !query +table t +|> extend (select a from other where x = a limit 1) as z +-- !query schema +struct +-- !query output +0 abc NULL +1 def 1 + + +-- !query +table t +|> where exists ( + table other + |> extend t.x + |> select * except (a, b)) +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> extend 1 as x +-- !query schema +struct +-- !query output +0 abc 1 +1 def 1 + + +-- !query +table t +|> extend first_value(x) over (partition by y) as result +-- !query schema +struct +-- !query output +0 abc 0 +1 def 1 + + +-- !query +table t +|> extend x + length(y) as z, z + 1 as plus_one +-- !query schema +struct +-- !query output +0 abc 3 4 +1 def 4 5 + + +-- !query +table t +|> extend sum(x) as z +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PIPE_OPERATOR_CONTAINS_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "clause" : "EXTEND", + "expr" : "sum(x#x)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 24, + "fragment" : "sum(x)" + } ] +} + + +-- !query +table t +|> extend distinct x as z +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'as'", + "hint" : "" + } +} + + +-- !query +table t +|> extend * +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_USAGE_OF_STAR_OR_REGEX", + "sqlState" : "42000", + "messageParameters" : { + "elem" : "'*'", + "prettyName" : "expression `pipeexpression`" + } +} + + +-- !query +table t +|> set x = 1 +-- !query schema +struct +-- !query output +1 abc +1 def + + +-- !query +table t +|> set y = x +-- !query schema +struct +-- !query output +0 0 +1 1 + + +-- !query +table t +|> extend 1 as z +|> set z = x + length(y) +-- !query schema +struct +-- !query output +0 abc 3 +1 def 4 + + +-- !query +table t +|> extend 1 as z +|> extend 2 as zz +|> set z = x + length(y), zz = x + 1 +-- !query schema +struct +-- !query output +0 abc 3 1 +1 def 4 2 + + +-- !query +table other +|> extend 3 as c +|> set a = b, b = c +-- !query schema +struct +-- !query output +1 3 3 +2 3 3 +4 3 3 + + +-- !query +table t +|> extend 1 as z +|> extend 2 as zz +|> set z = x + length(y), zz = z + 1 +-- !query schema +struct +-- !query output +0 abc 3 4 +1 def 4 5 + + +-- !query +table t +|> extend 1 as z +|> set z = x + length(y) +|> set z = z + 1 +-- !query schema +struct +-- !query output +0 abc 4 +1 def 5 + + +-- !query +table t +|> extend 1 as z +|> set z = x + length(y), z = z + 1 +-- !query schema +struct +-- !query output +0 abc 4 +1 def 5 + + +-- !query +select col from st +|> extend 1 as z +|> set z = col.i1 +-- !query schema +struct,z:int> +-- !query output +{"i1":2,"i2":3} 2 + + +-- !query +table t +|> set y = (select a from other where x = a limit 1) +-- !query schema +struct +-- !query output +0 NULL +1 1 + + +-- !query +table t +|> extend 1 as `x.y.z` +|> set `x.y.z` = x + length(y) +-- !query schema +struct +-- !query output +0 abc 3 +1 def 4 + + +-- !query +table t +|> extend 1 as z +|> set z = first_value(x) over (partition by y) +-- !query schema +struct +-- !query output +0 abc 0 +1 def 1 + + +-- !query +values (0), (1) lhs(a) +|> inner join values (1), (2) rhs(a) using (a) +|> extend lhs.a + rhs.a as z1 +|> extend lhs.a - rhs.a as z2 +|> drop z1 +|> where z2 = 0 +|> order by lhs.a, rhs.a, z2 +|> set z2 = 4 +|> limit 2 +|> select lhs.a, rhs.a, z2 +-- !query schema +struct +-- !query output +1 1 4 + + +-- !query +table t +|> set z = 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`z`", + "proposal" : "`x`, `y`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 20, + "fragment" : "table t\n|> set z = 1" + } ] +} + + +-- !query +table t +|> set x = 1 as z +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'as'", + "hint" : "" + } +} + + +-- !query +select col from st +|> set col.i1 = 42 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0035", + "messageParameters" : { + "message" : "SQL pipe syntax |> SET operator with multi-part assignment key (only single-part keys are allowed)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 37, + "fragment" : "col.i1 = 42" + } ] +} + + +-- !query +table t +|> drop y +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +select 1 as x, 2 as y, 3 as z +|> drop z, y +-- !query schema +struct +-- !query output +1 + + +-- !query +select 1 as x, 2 as y, 3 as z +|> drop z +|> drop y +-- !query schema +struct +-- !query output +1 + + +-- !query +select x from t +|> drop x +-- !query schema +struct<> +-- !query output + + + +-- !query +table t +|> extend 1 as `x.y.z` +|> drop `x.y.z` +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> drop z +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`z`", + "proposal" : "`x`, `y`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 17, + "fragment" : "table t\n|> drop z" + } ] +} + + +-- !query +table st +|> drop col.i1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'.'", + "hint" : "" + } +} + + +-- !query +table st +|> drop `col.i1` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`col.i1`", + "proposal" : "`col`, `x`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 25, + "fragment" : "table st\n|> drop `col.i1`" + } ] +} + + +-- !query +select 1 as x, 2 as y, 3 as z +|> drop z, y, z +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "EXCEPT_OVERLAPPING_COLUMNS", + "sqlState" : "42702", + "messageParameters" : { + "columns" : "z, y, z" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 45, + "fragment" : "select 1 as x, 2 as y, 3 as z\n|> drop z, y, z" + } ] +} + + +-- !query +table t +|> as u +|> select u.x, u.y +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +select 1 as x, 2 as y +|> as u +|> select u.x, u.y +-- !query schema +struct +-- !query output +1 2 + + +-- !query +table t +|> as `u.v` +|> select `u.v`.x, `u.v`.y +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> as u +|> as v +|> select v.x, v.y +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> as u +|> where u.x = 1 +-- !query schema +struct +-- !query output +1 def + + +-- !query +table t +|> as u, v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "','", + "hint" : "" + } +} + + +-- !query +table t +|> as 1 + 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'1'", + "hint" : "" + } +} + + +-- !query +table t +|> as u-v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_IDENTIFIER", + "sqlState" : "42602", + "messageParameters" : { + "ident" : "u-v" + } +} + + +-- !query +table t +|> as u@v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'@'", + "hint" : "" + } +} + + +-- !query +table t +|> as u#######v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'#'", + "hint" : "" + } +} + + +-- !query +table t +|> where true +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> where x + length(y) < 4 +-- !query schema +struct +-- !query output +0 abc + + +-- !query +table t +|> where x + length(y) < 4 +|> where x + length(y) < 3 +-- !query schema +struct +-- !query output + + + +-- !query +(select x, sum(length(y)) as sum_len from t group by x) +|> where x = 1 +-- !query schema +struct +-- !query output +1 3 + + +-- !query +table t +|> where t.x = 1 +-- !query schema +struct +-- !query output +1 def + + +-- !query +table t +|> where spark_catalog.default.t.x = 1 +-- !query schema +struct +-- !query output +1 def + + +-- !query +(select col from st) +|> where col.i1 = 1 +-- !query schema +struct> +-- !query output + + + +-- !query +table st +|> where st.col.i1 = 2 +-- !query schema +struct> +-- !query output +1 {"i1":2,"i2":3} + + +-- !query +table t +|> where exists (select a from other where x = a limit 1) +-- !query schema +struct +-- !query output +1 def + + +-- !query +table t +|> where (select any_value(a) from other where x = a limit 1) = 1 +-- !query schema +struct +-- !query output +1 def + + +-- !query +table t +|> where sum(x) = 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(sum(x) = 1)\"", + "expressionList" : "sum(spark_catalog.default.t.x)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 27, + "fragment" : "table t\n|> where sum(x) = 1" + } ] +} + + +-- !query +table t +|> where y = 'abc' or length(y) + sum(x) = 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"((y = abc) OR ((length(y) + sum(x)) = 1))\"", + "expressionList" : "sum(spark_catalog.default.t.x)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 52, + "fragment" : "table t\n|> where y = 'abc' or length(y) + sum(x) = 1" + } ] +} + + +-- !query +table t +|> where sum(x) over (partition by y) = 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1034", + "messageParameters" : { + "clauseName" : "WHERE" + } +} + + +-- !query +table t +|> where sum(x) over w = 1 + window w as (partition by y) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "NOT_ALLOWED_IN_PIPE_OPERATOR_WHERE.WINDOW_CLAUSE", + "sqlState" : "42601", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 66, + "fragment" : "table t\n|> where sum(x) over w = 1\n window w as (partition by y)" + } ] +} + + +-- !query +select * from t where sum(x) over (partition by y) = 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1034", + "messageParameters" : { + "clauseName" : "WHERE" + } +} + + +-- !query +table t +|> select x, length(y) as z +|> where x + length(y) < 4 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`y`", + "proposal" : "`z`, `spark_catalog`.`default`.`t`.`x`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 57, + "stopIndex" : 57, + "fragment" : "y" + } ] +} + + +-- !query +table t +|> select x, length(y) as z +|> limit 1000 +|> where x + length(y) < 4 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`y`", + "proposal" : "`z`, `spark_catalog`.`default`.`t`.`x`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 71, + "stopIndex" : 71, + "fragment" : "y" + } ] +} + + +-- !query +table t +|> select x, length(y) as z +|> limit 1000 offset 1 +|> where x + length(y) < 4 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`y`", + "proposal" : "`z`, `spark_catalog`.`default`.`t`.`x`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 80, + "stopIndex" : 80, + "fragment" : "y" + } ] +} + + +-- !query +table t +|> select x, length(y) as z +|> order by x, y +|> where x + length(y) < 4 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`y`", + "proposal" : "`z`, `spark_catalog`.`default`.`t`.`x`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 52, + "stopIndex" : 52, + "fragment" : "y" + } ] +} + + +-- !query +(select x, sum(length(y)) as sum_len from t group by x) +|> where sum(length(y)) = 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`y`", + "proposal" : "`sum_len`, `spark_catalog`.`default`.`t`.`x`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 77, + "stopIndex" : 77, + "fragment" : "y" + } ] +} + + +-- !query +table courseSales +|> select `year`, course, earnings +|> pivot ( + sum(earnings) + for course in ('dotNET', 'Java') + ) +-- !query schema +struct +-- !query output +2012 15000 20000 +2013 48000 30000 + + +-- !query +table courseSales +|> select `year` as y, course as c, earnings as e +|> pivot ( + sum(e) as s, avg(e) as a + for y in (2012 as firstYear, 2013 as secondYear) + ) +-- !query schema +struct +-- !query output +Java 20000 20000.0 30000 30000.0 +dotNET 15000 7500.0 48000 48000.0 + + +-- !query +select course, `year`, y, a +from courseSales +join yearsWithComplexTypes on `year` = y +|> pivot ( + max(a) + for (y, course) in ((2012, 'dotNET'), (2013, 'Java')) + ) +-- !query schema +struct,{2013, Java}:array> +-- !query output +2012 [1,1] NULL +2013 NULL [2,2] + + +-- !query +select earnings, `year`, s +from courseSales +join yearsWithComplexTypes on `year` = y +|> pivot ( + sum(earnings) + for s in ((1, 'a'), (2, 'b')) + ) +-- !query schema +struct +-- !query output +2012 35000 NULL +2013 NULL 78000 + + +-- !query +table courseEarnings +|> unpivot ( + earningsYear for `year` in (`2012`, `2013`, `2014`) + ) +-- !query schema +struct +-- !query output +Java 2012 20000 +Java 2013 30000 +dotNET 2012 15000 +dotNET 2013 48000 +dotNET 2014 22500 + + +-- !query +table courseEarnings +|> unpivot include nulls ( + earningsYear for `year` in (`2012`, `2013`, `2014`) + ) +-- !query schema +struct +-- !query output +Java 2012 20000 +Java 2013 30000 +Java 2014 NULL +dotNET 2012 15000 +dotNET 2013 48000 +dotNET 2014 22500 + + +-- !query +table courseEarningsAndSales +|> unpivot include nulls ( + (earnings, sales) for `year` in ( + (earnings2012, sales2012) as `2012`, + (earnings2013, sales2013) as `2013`, + (earnings2014, sales2014) as `2014`) + ) +-- !query schema +struct +-- !query output +Java 2012 20000 1 +Java 2013 30000 2 +Java 2014 NULL NULL +dotNET 2012 15000 NULL +dotNET 2013 48000 1 +dotNET 2014 22500 1 + + +-- !query +table courseSales +|> select course, earnings +|> pivot ( + sum(earnings) + for `year` in (2012, 2013) + ) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`year`", + "proposal" : "`course`, `earnings`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 49, + "stopIndex" : 111, + "fragment" : "pivot (\n sum(earnings)\n for `year` in (2012, 2013)\n )" + } ] +} + + +-- !query +table courseSales +|> pivot ( + sum(earnings) + for `year` in (course, 2013) + ) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NON_LITERAL_PIVOT_VALUES", + "sqlState" : "42K08", + "messageParameters" : { + "expression" : "\"course\"" + } +} + + +-- !query +table courseSales +|> select course, earnings +|> pivot ( + sum(earnings) + for `year` in (2012, 2013) + ) + unpivot ( + earningsYear for `year` in (`2012`, `2013`, `2014`) + ) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "NOT_ALLOWED_IN_FROM.UNPIVOT_WITH_PIVOT", + "sqlState" : "42601", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 186, + "fragment" : "table courseSales\n|> select course, earnings\n|> pivot (\n sum(earnings)\n for `year` in (2012, 2013)\n )\n unpivot (\n earningsYear for `year` in (`2012`, `2013`, `2014`)\n )" + } ] +} + + +-- !query +table courseSales +|> select course, earnings +|> unpivot ( + earningsYear for `year` in (`2012`, `2013`, `2014`) + ) + pivot ( + sum(earnings) + for `year` in (2012, 2013) + ) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "NOT_ALLOWED_IN_FROM.UNPIVOT_WITH_PIVOT", + "sqlState" : "42601", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 186, + "fragment" : "table courseSales\n|> select course, earnings\n|> unpivot (\n earningsYear for `year` in (`2012`, `2013`, `2014`)\n )\n pivot (\n sum(earnings)\n for `year` in (2012, 2013)\n )" + } ] +} + + +-- !query +table courseSales +|> select course, earnings +|> pivot ( + sum(earnings) + for `year` in (2012, 2013) + ) + pivot ( + sum(earnings) + for `year` in (2012, 2013) + ) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'pivot'", + "hint" : "" + } +} + + +-- !query +table courseSales +|> select course, earnings +|> unpivot ( + earningsYear for `year` in (`2012`, `2013`, `2014`) + ) + unpivot ( + earningsYear for `year` in (`2012`, `2013`, `2014`) + ) + pivot ( + sum(earnings) + for `year` in (2012, 2013) + ) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'unpivot'", + "hint" : "" + } +} + + +-- !query +table t +|> tablesample (100 percent) repeatable (0) +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> tablesample (2 rows) repeatable (0) +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> tablesample (bucket 1 out of 1) repeatable (0) +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> tablesample (100 percent) repeatable (0) +|> tablesample (5 rows) repeatable (0) +|> tablesample (bucket 1 out of 1) repeatable (0) +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> tablesample () +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0014", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 25, + "fragment" : "tablesample ()" + } ] +} + + +-- !query +table t +|> tablesample (-100 percent) repeatable (0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Sampling fraction (-1.0) must be on interval [0, 1]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 52, + "fragment" : "tablesample (-100 percent) repeatable (0)" + } ] +} + + +-- !query +table t +|> tablesample (-5 rows) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_NEGATIVE", + "sqlState" : "42K0E", + "messageParameters" : { + "expr" : "\"-5\"", + "name" : "limit", + "v" : "-5" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 26, + "fragment" : "-5" + } ] +} + + +-- !query +table t +|> tablesample (x rows) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE", + "sqlState" : "42K0E", + "messageParameters" : { + "expr" : "\"x\"", + "name" : "limit" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 25, + "fragment" : "x" + } ] +} + + +-- !query +table t +|> tablesample (bucket 2 out of 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Sampling fraction (2.0) must be on interval [0, 1]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 42, + "fragment" : "tablesample (bucket 2 out of 1)" + } ] +} + + +-- !query +table t +|> tablesample (200b) repeatable (0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0015", + "messageParameters" : { + "msg" : "byteLengthLiteral" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 44, + "fragment" : "tablesample (200b) repeatable (0)" + } ] +} + + +-- !query +table t +|> tablesample (200) repeatable (0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0016", + "messageParameters" : { + "bytesStr" : "200" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 43, + "fragment" : "tablesample (200) repeatable (0)" + } ] +} + + +-- !query +table join_test_t1 +|> inner join join_test_empty_table +-- !query schema +struct +-- !query output + + + +-- !query +table join_test_t1 +|> cross join join_test_empty_table +-- !query schema +struct +-- !query output + + + +-- !query +table join_test_t1 +|> left outer join join_test_empty_table +-- !query schema +struct +-- !query output +1 NULL + + +-- !query +table join_test_t1 +|> right outer join join_test_empty_table +-- !query schema +struct +-- !query output + + + +-- !query +table join_test_t1 +|> full outer join join_test_empty_table using (a) +-- !query schema +struct +-- !query output +1 + + +-- !query +table join_test_t1 +|> full outer join join_test_empty_table on (join_test_t1.a = join_test_empty_table.a) +-- !query schema +struct +-- !query output +1 NULL + + +-- !query +table join_test_t1 +|> left semi join join_test_empty_table +-- !query schema +struct +-- !query output + + + +-- !query +table join_test_t1 +|> left anti join join_test_empty_table +-- !query schema +struct +-- !query output +1 + + +-- !query +select * from join_test_t1 where true +|> inner join join_test_empty_table +-- !query schema +struct +-- !query output + + + +-- !query +select 1 as x, 2 as y +|> inner join (select 1 as x, 4 as y) using (x) +-- !query schema +struct +-- !query output +1 2 4 + + +-- !query +table join_test_t1 +|> inner join (join_test_t2 jt2 inner join join_test_t3 jt3 using (a)) using (a) +|> select a, join_test_t1.a, jt2.a, jt3.a +-- !query schema +struct +-- !query output +1 1 1 1 + + +-- !query +table join_test_t1 +|> inner join join_test_t2 tablesample (100 percent) repeatable (0) jt2 using (a) +-- !query schema +struct +-- !query output +1 + + +-- !query +table join_test_t1 +|> inner join (select 1 as a) tablesample (100 percent) repeatable (0) jt2 using (a) +-- !query schema +struct +-- !query output +1 + + +-- !query +table join_test_t1 +|> join join_test_t1 using (a) +-- !query schema +struct +-- !query output +1 + + +-- !query +table lateral_test_t1 +|> join lateral (select c1) +-- !query schema +struct +-- !query output +0 1 0 +1 2 1 + + +-- !query +table lateral_test_t1 +|> join lateral (select c1 from lateral_test_t2) +-- !query schema +struct +-- !query output +0 1 0 +0 1 0 +1 2 0 +1 2 0 + + +-- !query +table lateral_test_t1 +|> join lateral (select lateral_test_t1.c1 from lateral_test_t2) +-- !query schema +struct +-- !query output +0 1 0 +0 1 0 +1 2 1 +1 2 1 + + +-- !query +table lateral_test_t1 +|> join lateral (select lateral_test_t1.c1 + t2.c1 from lateral_test_t2 t2) +-- !query schema +struct +-- !query output +0 1 0 +0 1 0 +1 2 1 +1 2 1 + + +-- !query +table lateral_test_t1 +|> join lateral (select *) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +table lateral_test_t1 +|> join lateral (select * from lateral_test_t2) +-- !query schema +struct +-- !query output +0 1 0 2 +0 1 0 3 +1 2 0 2 +1 2 0 3 + + +-- !query +table lateral_test_t1 +|> join lateral (select lateral_test_t1.* from lateral_test_t2) +-- !query schema +struct +-- !query output +0 1 0 1 +0 1 0 1 +1 2 1 2 +1 2 1 2 + + +-- !query +table lateral_test_t1 +|> join lateral (select lateral_test_t1.*, t2.* from lateral_test_t2 t2) +-- !query schema +struct +-- !query output +0 1 0 1 0 2 +0 1 0 1 0 3 +1 2 1 2 0 2 +1 2 1 2 0 3 + + +-- !query +table lateral_test_t1 +|> join lateral_test_t2 +|> join lateral (select lateral_test_t1.c2 + lateral_test_t2.c2) +-- !query schema +struct +-- !query output +0 1 0 2 3 +0 1 0 3 4 +1 2 0 2 4 +1 2 0 3 5 + + +-- !query +table natural_join_test_t1 +|> natural join natural_join_test_t2 +|> where k = "one" +-- !query schema +struct +-- !query output +one 1 1 +one 1 5 + + +-- !query +table natural_join_test_t1 +|> natural join natural_join_test_t2 nt2 +|> select natural_join_test_t1.* +-- !query schema +struct +-- !query output +one 1 +one 1 +two 2 + + +-- !query +table natural_join_test_t1 +|> natural join natural_join_test_t2 nt2 +|> natural join natural_join_test_t3 nt3 +|> select natural_join_test_t1.*, nt2.*, nt3.* +-- !query schema +struct +-- !query output +one 1 one 1 one 4 +one 1 one 1 one 6 +one 1 one 5 one 4 +one 1 one 5 one 6 +two 2 two 22 two 5 + + +-- !query +table join_test_t1 +|> inner join join_test_empty_table + inner join join_test_empty_table +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'inner'", + "hint" : "" + } +} + + +-- !query +table join_test_t1 +|> select 1 + 2 as result +|> full outer join join_test_empty_table on (join_test_t1.a = join_test_empty_table.a) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`join_test_t1`.`a`", + "proposal" : "`result`, `join_test_empty_table`.`a`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 91, + "stopIndex" : 104, + "fragment" : "join_test_t1.a" + } ] +} + + +-- !query +table join_test_t1 jt +|> cross join (select * from jt) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'jt'", + "hint" : "" + } +} + + +-- !query +table t +|> union all table t +-- !query schema +struct +-- !query output +0 abc +0 abc +1 def +1 def + + +-- !query +table t +|> union table t +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +(select * from t) +|> union all table t +-- !query schema +struct +-- !query output +0 abc +0 abc +1 def +1 def + + +-- !query +(select * from t) +|> union table t +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +values (0, 'abc') tab(x, y) +|> union all table t +-- !query schema +struct +-- !query output +0 abc +0 abc +1 def + + +-- !query +values (2, 'xyz') tab(x, y) +|> union table t +|> where x = 0 +-- !query schema +struct +-- !query output +0 abc + + +-- !query +values (2, 'xyz') tab(x, y) +|> union table t +|> drop x +-- !query schema +struct +-- !query output +abc +def +xyz + + +-- !query +(select * from t) +|> union all (select * from t) +-- !query schema +struct +-- !query output +0 abc +0 abc +1 def +1 def + + +-- !query +table t +|> except all table t +-- !query schema +struct +-- !query output + + + +-- !query +table t +|> except table t +-- !query schema +struct +-- !query output + + + +-- !query +table t +|> intersect all table t +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> intersect table t +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> minus all table t +-- !query schema +struct +-- !query output + + + +-- !query +table t +|> minus table t +-- !query schema +struct +-- !query output + + + +-- !query +table t +|> select x +|> union all table t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NUM_COLUMNS_MISMATCH", + "sqlState" : "42826", + "messageParameters" : { + "firstNumColumns" : "1", + "invalidNumColumns" : "2", + "invalidOrdinalNum" : "second", + "operator" : "UNION" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 40, + "fragment" : "table t\n|> select x\n|> union all table t" + } ] +} + + +-- !query +table t +|> union all table st +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "second", + "dataType1" : "\"STRUCT\"", + "dataType2" : "\"STRING\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "table t\n|> union all table st" + } ] +} + + +-- !query +table t +|> order by x +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +(select * from t) +|> order by x +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +values (0, 'abc') tab(x, y) +|> order by x +-- !query schema +struct +-- !query output +0 abc + + +-- !query +table t +|> order by x +|> limit 1 +-- !query schema +struct +-- !query output +0 abc + + +-- !query +table t +|> where x = 1 +|> select y +|> limit 2 offset 1 +-- !query schema +struct +-- !query output + + + +-- !query +table t +|> where x = 1 +|> select y +|> offset 1 +-- !query schema +struct +-- !query output + + + +-- !query +table t +|> limit all offset 0 +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> distribute by x +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> cluster by x +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> sort by x distribute by x +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> order by x desc +order by y +-- !query schema +struct +-- !query output +0 abc +1 def + + +-- !query +table t +|> order by x desc order by x + y +order by y +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'order'", + "hint" : "" + } +} + + +-- !query +table t +|> select 1 + 2 as result +|> order by x +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`x`", + "proposal" : "`result`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 47, + "stopIndex" : 47, + "fragment" : "x" + } ] +} + + +-- !query +table t +|> select 1 + 2 as result +|> distribute by x +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`x`", + "proposal" : "`result`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 52, + "stopIndex" : 52, + "fragment" : "x" + } ] +} + + +-- !query +table t +|> order by x limit 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "MULTIPLE_QUERY_RESULT_CLAUSES_WITH_PIPE_OPERATORS", + "sqlState" : "42000", + "messageParameters" : { + "clause1" : "ORDER BY", + "clause2" : "LIMIT" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 29, + "fragment" : "order by x limit 1" + } ] +} + + +-- !query +table t +|> order by x sort by x +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_FEATURE.COMBINATION_QUERY_RESULT_CLAUSES", + "sqlState" : "0A000", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 31, + "fragment" : "order by x sort by x" + } ] +} + + +-- !query +table other +|> aggregate sum(b) as result group by a +-- !query schema +struct +-- !query output +1 3 +2 4 + + +-- !query +table other +|> aggregate sum(b) as result group by a +|> select result +-- !query schema +struct +-- !query output +3 +4 + + +-- !query +table other +|> aggregate sum(b) group by a + 1 as gkey +|> select gkey +-- !query schema +struct +-- !query output +2 +3 + + +-- !query +select 1 as x, 2 as y +|> aggregate group by x, y +-- !query schema +struct +-- !query output +1 2 + + +-- !query +select 3 as x, 4 as y +|> aggregate group by 1, 2 +-- !query schema +struct +-- !query output +3 4 + + +-- !query +values (3, 4) as tab(x, y) +|> aggregate sum(y) group by 1 +-- !query schema +struct +-- !query output +3 4 + + +-- !query +values (3, 4), (5, 4) as tab(x, y) +|> aggregate sum(y) group by 1 +-- !query schema +struct +-- !query output +3 4 +5 4 + + +-- !query +select 3 as x, 4 as y +|> aggregate sum(y) group by 1, 1 +-- !query schema +struct +-- !query output +3 3 4 + + +-- !query +select 1 as `1`, 2 as `2` +|> aggregate sum(`2`) group by `1` +-- !query schema +struct<1:int,sum(2):bigint> +-- !query output +1 2 + + +-- !query +select 3 as x, 4 as y +|> aggregate sum(y) group by 2 +-- !query schema +struct +-- !query output +4 4 + + +-- !query +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 2 +-- !query schema +struct +-- !query output +4 4 + + +-- !query +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 3 +-- !query schema +struct +-- !query output +5 4 + + +-- !query +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 2, 3 +-- !query schema +struct +-- !query output +4 5 4 + + +-- !query +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by 1, 2, 3 +-- !query schema +struct +-- !query output +3 4 5 4 + + +-- !query +select 3 as x, 4 as y, 5 as z +|> aggregate sum(y) group by x, 2, 3 +-- !query schema +struct +-- !query output +3 4 5 4 + + +-- !query +table t +|> aggregate sum(x) +-- !query schema +struct +-- !query output +1 + + +-- !query +table t +|> aggregate sum(x) + 1 as result_plus_one +-- !query schema +struct +-- !query output +2 + + +-- !query +table other +|> aggregate group by a +|> where a = 1 +-- !query schema +struct +-- !query output +1 + + +-- !query +select 1 as x, 2 as y, 3 as z +|> aggregate group by x, y, x + y as z +-- !query schema +struct +-- !query output +1 2 3 + + +-- !query +select 1 as x, 2 as y, 3 as z +|> aggregate group by x as z, x + y as z +-- !query schema +struct +-- !query output +1 3 + + +-- !query +select 1 as x, 2 as y, named_struct('z', 3) as st +|> aggregate group by x, y, x, x, st.z, (st).z, 1 + x, 2 + x +-- !query schema +struct +-- !query output +1 2 1 1 3 3 2 3 + + +-- !query +select 1 x, 2 y, 3 z +|> aggregate sum(z) z group by x, y +|> aggregate avg(z) z group by x +|> aggregate count(distinct z) c +-- !query schema +struct +-- !query output +1 + + +-- !query +select 1 x, 3 z +|> aggregate count(*) group by x, z, x +|> select x +-- !query schema +struct +-- !query output +1 + + +-- !query +table other +|> aggregate a + count(b) group by a +-- !query schema +struct +-- !query output +1 3 +2 3 + + +-- !query +table other +|> aggregate a group by a +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PIPE_OPERATOR_AGGREGATE_EXPRESSION_CONTAINS_NO_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "expr" : "a#x" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 37, + "stopIndex" : 37, + "fragment" : "a" + } ] +} + + +-- !query +select 3 as x, 4 as y +|> aggregate group by all +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_FEATURE.PIPE_OPERATOR_AGGREGATE_UNSUPPORTED_CASE", + "sqlState" : "0A000", + "messageParameters" : { + "case" : "GROUP BY ALL" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 47, + "fragment" : "select 3 as x, 4 as y\n|> aggregate group by all" + } ] +} + + +-- !query +table courseSales +|> aggregate sum(earnings) group by rollup(course, `year`) +|> where course = 'dotNET' and `year` = '2013' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_FEATURE.PIPE_OPERATOR_AGGREGATE_UNSUPPORTED_CASE", + "sqlState" : "0A000", + "messageParameters" : { + "case" : "GROUP BY ROLLUP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 123, + "fragment" : "table courseSales\n|> aggregate sum(earnings) group by rollup(course, `year`)\n|> where course = 'dotNET' and `year` = '2013'" + } ] +} + + +-- !query +table courseSales +|> aggregate sum(earnings) group by cube(course, `year`) +|> where course = 'dotNET' and `year` = '2013' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_FEATURE.PIPE_OPERATOR_AGGREGATE_UNSUPPORTED_CASE", + "sqlState" : "0A000", + "messageParameters" : { + "case" : "GROUP BY CUBE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 121, + "fragment" : "table courseSales\n|> aggregate sum(earnings) group by cube(course, `year`)\n|> where course = 'dotNET' and `year` = '2013'" + } ] +} + + +-- !query +table courseSales +|> aggregate sum(earnings) group by course, `year` grouping sets(course, `year`) +|> where course = 'dotNET' and `year` = '2013' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_FEATURE.PIPE_OPERATOR_AGGREGATE_UNSUPPORTED_CASE", + "sqlState" : "0A000", + "messageParameters" : { + "case" : "GROUPING SETS" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 145, + "fragment" : "table courseSales\n|> aggregate sum(earnings) group by course, `year` grouping sets(course, `year`)\n|> where course = 'dotNET' and `year` = '2013'" + } ] +} + + +-- !query +table courseSales +|> aggregate sum(earnings), grouping(course) + 1 + group by course +|> where course = 'dotNET' and `year` = '2013' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_FEATURE.PIPE_OPERATOR_AGGREGATE_UNSUPPORTED_CASE", + "sqlState" : "0A000", + "messageParameters" : { + "case" : "GROUPING" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 132, + "fragment" : "table courseSales\n|> aggregate sum(earnings), grouping(course) + 1\n group by course\n|> where course = 'dotNET' and `year` = '2013'" + } ] +} + + +-- !query +table courseSales +|> aggregate sum(earnings), grouping_id(course) + group by course +|> where course = 'dotNET' and `year` = '2013' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_FEATURE.PIPE_OPERATOR_AGGREGATE_UNSUPPORTED_CASE", + "sqlState" : "0A000", + "messageParameters" : { + "case" : "GROUPING_ID" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 131, + "fragment" : "table courseSales\n|> aggregate sum(earnings), grouping_id(course)\n group by course\n|> where course = 'dotNET' and `year` = '2013'" + } ] +} + + +-- !query +select 1 as x, 2 as y +|> aggregate group by () +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "')'", + "hint" : "" + } +} + + +-- !query +table other +|> aggregate a +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PIPE_OPERATOR_AGGREGATE_EXPRESSION_CONTAINS_NO_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "expr" : "a#x" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 26, + "fragment" : "a" + } ] +} + + +-- !query +table other +|> select sum(a) as result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PIPE_OPERATOR_CONTAINS_AGGREGATE_FUNCTION", + "sqlState" : "0A000", + "messageParameters" : { + "clause" : "SELECT", + "expr" : "sum(a#x)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 28, + "fragment" : "sum(a)" + } ] +} + + +-- !query +table other +|> aggregate +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0035", + "messageParameters" : { + "message" : "The AGGREGATE clause requires a list of aggregate expressions or a list of grouping expressions, or both" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 24, + "fragment" : "table other\n|> aggregate" + } ] +} + + +-- !query +table other +|> aggregate group by +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`group`", + "proposal" : "`a`, `b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 30, + "fragment" : "group" + } ] +} + + +-- !query +table other +|> group by a +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'group'", + "hint" : "" + } +} + + +-- !query +table other +|> aggregate sum(a) over () group by b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_FEATURE.PIPE_OPERATOR_AGGREGATE_UNSUPPORTED_CASE", + "sqlState" : "0A000", + "messageParameters" : { + "case" : "window functions; please update the query to move the window functions to a subsequent |> SELECT operator instead" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 50, + "fragment" : "table other\n|> aggregate sum(a) over () group by b" + } ] +} + + +-- !query +select 1 x, 2 y, 3 z +|> aggregate count(*) AS c, sum(x) AS x group by x +|> where c = 1 +|> where x = 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`x`", + "referenceNames" : "[`x`, `x`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 97, + "stopIndex" : 97, + "fragment" : "x" + } ] +} + + +-- !query +table windowTestData +|> select cate, sum(val) over w + window w as (partition by cate order by val) +-- !query schema +struct +-- !query output +NULL 3 +NULL NULL +a 2 +a 2 +a 4 +a NULL +b 1 +b 3 +b 6 + + +-- !query +table windowTestData +|> select cate, sum(val) over w + window w as (order by val_timestamp range between unbounded preceding and current row) +-- !query schema +struct +-- !query output +NULL 5 +NULL NULL +a 13 +a 5 +a 5 +a 6 +b 13 +b 5 +b 8 + + +-- !query +table windowTestData +|> select cate, val + window w as (partition by cate order by val) +-- !query schema +struct +-- !query output +NULL 3 +NULL NULL +a 1 +a 1 +a 2 +a NULL +b 1 +b 2 +b 3 + + +-- !query +table windowTestData +|> select cate, val, sum(val) over w as sum_val + window w as (partition by cate) +|> select cate, val, sum_val, first_value(cate) over w + window w as (order by val) +-- !query schema +struct +-- !query output +NULL 3 3 a +NULL NULL 3 a +a 1 4 a +a 1 4 a +a 2 4 a +a NULL 4 a +b 1 6 a +b 2 6 a +b 3 6 a + + +-- !query +table windowTestData +|> select cate, val, sum(val) over w1, first_value(cate) over w2 + window w1 as (partition by cate), w2 as (order by val) +-- !query schema +struct +-- !query output +NULL 3 3 a +NULL NULL 3 a +a 1 4 a +a 1 4 a +a 2 4 a +a NULL 4 a +b 1 6 a +b 2 6 a +b 3 6 a + + +-- !query +table windowTestData +|> select cate, val, sum(val) over w, first_value(val) over w + window w1 as (partition by cate order by val) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_WINDOW_SPECIFICATION", + "sqlState" : "42P20", + "messageParameters" : { + "docroot" : "https://spark.apache.org/docs/latest", + "windowName" : "w" + } +} + + +-- !query +(select col from st) +|> select col.i1, sum(col.i2) over w + window w as (partition by col.i1 order by col.i2) +-- !query schema +struct +-- !query output +2 3 + + +-- !query +table st +|> select st.col.i1, sum(st.col.i2) over w + window w as (partition by st.col.i1 order by st.col.i2) +-- !query schema +struct +-- !query output +2 3 + + +-- !query +table st +|> select spark_catalog.default.st.col.i1, sum(spark_catalog.default.st.col.i2) over w + window w as (partition by spark_catalog.default.st.col.i1 order by spark_catalog.default.st.col.i2) +-- !query schema +struct +-- !query output +2 3 + + +-- !query +table windowTestData +|> select cate, sum(val) over val + window val as (partition by cate order by val) +-- !query schema +struct +-- !query output +NULL 3 +NULL NULL +a 2 +a 2 +a 4 +a NULL +b 1 +b 3 +b 6 + + +-- !query +table windowTestData +|> select cate, sum(val) over w +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_WINDOW_SPECIFICATION", + "sqlState" : "42P20", + "messageParameters" : { + "docroot" : "https://spark.apache.org/docs/latest", + "windowName" : "w" + } +} + + +-- !query +table windowTestData +|> select cate, val, sum(val) over w1, first_value(cate) over w2 + window w1 as (partition by cate) + window w2 as (order by val) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "MISSING_WINDOW_SPECIFICATION", + "sqlState" : "42P20", + "messageParameters" : { + "docroot" : "https://spark.apache.org/docs/latest", + "windowName" : "w2" + } +} + + +-- !query +table windowTestData +|> select cate, val, sum(val) over w as sum_val + window w as (partition by cate order by val) +|> select cate, val, sum_val, first_value(cate) over w +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_WINDOW_SPECIFICATION", + "sqlState" : "42P20", + "messageParameters" : { + "docroot" : "https://spark.apache.org/docs/latest", + "windowName" : "w" + } +} + + +-- !query +table windowTestData +|> select cate, val, first_value(cate) over w as first_val +|> select cate, val, sum(val) over w as sum_val + window w as (order by val) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_WINDOW_SPECIFICATION", + "sqlState" : "42P20", + "messageParameters" : { + "docroot" : "https://spark.apache.org/docs/latest", + "windowName" : "w" + } +} + + +-- !query +with customer_total_return as +(select + sr_customer_sk as ctr_customer_sk, + sr_store_sk as ctr_store_sk, + sum(sr_return_amt) as ctr_total_return + from store_returns, date_dim + where sr_returned_date_sk = d_date_sk and d_year = 2000 + group by sr_customer_sk, sr_store_sk) +select c_customer_id +from customer_total_return ctr1, store, customer +where ctr1.ctr_total_return > + (select avg(ctr_total_return) * 1.2 + from customer_total_return ctr2 + where ctr1.ctr_store_sk = ctr2.ctr_store_sk) + and s_store_sk = ctr1.ctr_store_sk + and s_state = 'tn' + and ctr1.ctr_customer_sk = c_customer_sk +order by c_customer_id +limit 100 +-- !query schema +struct +-- !query output + + + +-- !query +with customer_total_return as + (from store_returns + |> join date_dim + |> where sr_returned_date_sk = d_date_sk and d_year = 2000 + |> aggregate sum(sr_return_amt) as ctr_total_return + group by sr_customer_sk as ctr_customer_sk, sr_store_sk as ctr_store_sk) +from customer_total_return ctr1 +|> join store +|> join customer +|> where ctr1.ctr_total_return > + (table customer_total_return + |> as ctr2 + |> where ctr1.ctr_store_sk = ctr2.ctr_store_sk + |> aggregate avg(ctr_total_return) * 1.2) + and s_store_sk = ctr1.ctr_store_sk + and s_state = 'tn' + and ctr1.ctr_customer_sk = c_customer_sk +|> order by c_customer_id +|> limit 100 +|> select c_customer_id +-- !query schema +struct +-- !query output + + + +-- !query +with wscs as +( select + sold_date_sk, + sales_price + from (select + ws_sold_date_sk sold_date_sk, + ws_ext_sales_price sales_price + from web_sales) x + union all + (select + cs_sold_date_sk sold_date_sk, + cs_ext_sales_price sales_price + from catalog_sales)), + wswscs as + ( select + d_week_seq, + sum(case when (d_day_name = 'sunday') + then sales_price + else null end) + sun_sales, + sum(case when (d_day_name = 'monday') + then sales_price + else null end) + mon_sales, + sum(case when (d_day_name = 'tuesday') + then sales_price + else null end) + tue_sales, + sum(case when (d_day_name = 'wednesday') + then sales_price + else null end) + wed_sales, + sum(case when (d_day_name = 'thursday') + then sales_price + else null end) + thu_sales, + sum(case when (d_day_name = 'friday') + then sales_price + else null end) + fri_sales, + sum(case when (d_day_name = 'saturday') + then sales_price + else null end) + sat_sales + from wscs, date_dim + where d_date_sk = sold_date_sk + group by d_week_seq) +select + d_week_seq1, + round(sun_sales1 / sun_sales2, 2), + round(mon_sales1 / mon_sales2, 2), + round(tue_sales1 / tue_sales2, 2), + round(wed_sales1 / wed_sales2, 2), + round(thu_sales1 / thu_sales2, 2), + round(fri_sales1 / fri_sales2, 2), + round(sat_sales1 / sat_sales2, 2) +from + (select + wswscs.d_week_seq d_week_seq1, + sun_sales sun_sales1, + mon_sales mon_sales1, + tue_sales tue_sales1, + wed_sales wed_sales1, + thu_sales thu_sales1, + fri_sales fri_sales1, + sat_sales sat_sales1 + from wswscs, date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and d_year = 2001) y, + (select + wswscs.d_week_seq d_week_seq2, + sun_sales sun_sales2, + mon_sales mon_sales2, + tue_sales tue_sales2, + wed_sales wed_sales2, + thu_sales thu_sales2, + fri_sales fri_sales2, + sat_sales sat_sales2 + from wswscs, date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and d_year = 2001 + 1) z +where d_week_seq1 = d_week_seq2 - 53 +order by d_week_seq1 +-- !query schema +struct +-- !query output + + + +-- !query +with wscs as + (table web_sales + |> select + ws_sold_date_sk sold_date_sk, + ws_ext_sales_price sales_price + |> as x + |> union all ( + table catalog_sales + |> select + cs_sold_date_sk sold_date_sk, + cs_ext_sales_price sales_price) + |> select + sold_date_sk, + sales_price), +wswscs as + (table wscs + |> join date_dim + |> where d_date_sk = sold_date_sk + |> aggregate + sum(case when (d_day_name = 'sunday') + then sales_price + else null end) + sun_sales, + sum(case when (d_day_name = 'monday') + then sales_price + else null end) + mon_sales, + sum(case when (d_day_name = 'tuesday') + then sales_price + else null end) + tue_sales, + sum(case when (d_day_name = 'wednesday') + then sales_price + else null end) + wed_sales, + sum(case when (d_day_name = 'thursday') + then sales_price + else null end) + thu_sales, + sum(case when (d_day_name = 'friday') + then sales_price + else null end) + fri_sales, + sum(case when (d_day_name = 'saturday') + then sales_price + else null end) + sat_sales + group by d_week_seq) +table wswscs +|> join date_dim +|> where date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001 +|> select + wswscs.d_week_seq d_week_seq1, + sun_sales sun_sales1, + mon_sales mon_sales1, + tue_sales tue_sales1, + wed_sales wed_sales1, + thu_sales thu_sales1, + fri_sales fri_sales1, + sat_sales sat_sales1 +|> as y +|> join ( + table wswscs + |> join date_dim + |> where date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001 + 1 + |> select + wswscs.d_week_seq d_week_seq2, + sun_sales sun_sales2, + mon_sales mon_sales2, + tue_sales tue_sales2, + wed_sales wed_sales2, + thu_sales thu_sales2, + fri_sales fri_sales2, + sat_sales sat_sales2 + |> as z) +|> where d_week_seq1 = d_week_seq2 - 53 +|> order by d_week_seq1 +|> select + d_week_seq1, + round(sun_sales1 / sun_sales2, 2), + round(mon_sales1 / mon_sales2, 2), + round(tue_sales1 / tue_sales2, 2), + round(wed_sales1 / wed_sales2, 2), + round(thu_sales1 / thu_sales2, 2), + round(fri_sales1 / fri_sales2, 2), + round(sat_sales1 / sat_sales2, 2) +-- !query schema +struct +-- !query output + + + +-- !query +select + dt.d_year, + item.i_brand_id brand_id, + item.i_brand brand, + sum(ss_ext_sales_price) sum_agg +from date_dim dt, store_sales, item +where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manufact_id = 128 + and dt.d_moy = 11 +group by dt.d_year, item.i_brand, item.i_brand_id +order by dt.d_year, sum_agg desc, brand_id +limit 100 +-- !query schema +struct +-- !query output + + + +-- !query +table date_dim +|> as dt +|> join store_sales +|> join item +|> where dt.d_date_sk = store_sales.ss_sold_date_sk + and store_sales.ss_item_sk = item.i_item_sk + and item.i_manufact_id = 128 + and dt.d_moy = 11 +|> aggregate sum(ss_ext_sales_price) sum_agg + group by dt.d_year d_year, item.i_brand_id brand_id, item.i_brand brand +|> order by d_year, sum_agg desc, brand_id +|> limit 100 +-- !query schema +struct +-- !query output + + + +-- !query +select + i_item_desc, + i_category, + i_class, + i_current_price, + sum(ws_ext_sales_price) as itemrevenue, + sum(ws_ext_sales_price) * 100 / sum(sum(ws_ext_sales_price)) + over + (partition by i_class) as revenueratio +from + web_sales, item, date_dim +where + ws_item_sk = i_item_sk + and i_category in ('sports', 'books', 'home') + and ws_sold_date_sk = d_date_sk + and d_date between cast('1999-02-22' as date) + and (cast('1999-02-22' as date) + interval 30 days) +group by + i_item_id, i_item_desc, i_category, i_class, i_current_price +order by + i_category, i_class, i_item_id, i_item_desc, revenueratio +limit 100 +-- !query schema +struct +-- !query output + + + +-- !query +table web_sales +|> join item +|> join date_dim +|> where ws_item_sk = i_item_sk + and i_category in ('sports', 'books', 'home') + and ws_sold_date_sk = d_date_sk + and d_date between cast('1999-02-22' as date) + and (cast('1999-02-22' as date) + interval 30 days) +|> aggregate sum(ws_ext_sales_price) AS itemrevenue + group by i_item_id, i_item_desc, i_category, i_class, i_current_price +|> extend + itemrevenue * 100 / sum(itemrevenue) + over (partition by i_class) as revenueratio +|> order by i_category, i_class, i_item_id, i_item_desc, revenueratio +|> select i_item_desc, i_category, i_class, i_current_price, itemrevenue, revenueratio +|> limit 100 +-- !query schema +struct +-- !query output + + + +-- !query +select + asceding.rnk, + i1.i_product_name best_performing, + i2.i_product_name worst_performing +from (select * +from (select + item_sk, + rank() + over ( + order by rank_col asc) rnk +from (select + ss_item_sk item_sk, + avg(ss_net_profit) rank_col +from store_sales ss1 +where ss_store_sk = 4 +group by ss_item_sk +having avg(ss_net_profit) > 0.9 * (select avg(ss_net_profit) rank_col +from store_sales +where ss_store_sk = 4 + and ss_addr_sk is null +group by ss_store_sk)) v1) v11 +where rnk < 11) asceding, + (select * + from (select + item_sk, + rank() + over ( + order by rank_col desc) rnk + from (select + ss_item_sk item_sk, + avg(ss_net_profit) rank_col + from store_sales ss1 + where ss_store_sk = 4 + group by ss_item_sk + having avg(ss_net_profit) > 0.9 * (select avg(ss_net_profit) rank_col + from store_sales + where ss_store_sk = 4 + and ss_addr_sk is null + group by ss_store_sk)) v2) v21 + where rnk < 11) descending, + item i1, item i2 +where asceding.rnk = descending.rnk + and i1.i_item_sk = asceding.item_sk + and i2.i_item_sk = descending.item_sk +order by asceding.rnk +limit 100 +-- !query schema +struct +-- !query output + + + +-- !query +from store_sales ss1 +|> where ss_store_sk = 4 +|> aggregate avg(ss_net_profit) rank_col + group by ss_item_sk as item_sk +|> where rank_col > 0.9 * ( + from store_sales + |> where ss_store_sk = 4 + and ss_addr_sk is null + |> aggregate avg(ss_net_profit) rank_col + group by ss_store_sk + |> select rank_col) +|> as v1 +|> select + item_sk, + rank() over ( + order by rank_col asc) rnk +|> as v11 +|> where rnk < 11 +|> as asceding +|> join ( + from store_sales ss1 + |> where ss_store_sk = 4 + |> aggregate avg(ss_net_profit) rank_col + group by ss_item_sk as item_sk + |> where rank_col > 0.9 * ( + table store_sales + |> where ss_store_sk = 4 + and ss_addr_sk is null + |> aggregate avg(ss_net_profit) rank_col + group by ss_store_sk + |> select rank_col) + |> as v2 + |> select + item_sk, + rank() over ( + order by rank_col asc) rnk + |> as v21 + |> where rnk < 11) descending +|> join item i1 +|> join item i2 +|> where asceding.rnk = descending.rnk + and i1.i_item_sk = asceding.item_sk + and i2.i_item_sk = descending.item_sk +|> order by asceding.rnk +|> select + asceding.rnk, + i1.i_product_name best_performing, + i2.i_product_name worst_performing +-- !query schema +struct +-- !query output + + + +-- !query +with web_v1 as ( + select + ws_item_sk item_sk, + d_date, + sum(sum(ws_sales_price)) + over (partition by ws_item_sk + order by d_date + rows between unbounded preceding and current row) cume_sales + from web_sales, date_dim + where ws_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ws_item_sk is not null + group by ws_item_sk, d_date), + store_v1 as ( + select + ss_item_sk item_sk, + d_date, + sum(sum(ss_sales_price)) + over (partition by ss_item_sk + order by d_date + rows between unbounded preceding and current row) cume_sales + from store_sales, date_dim + where ss_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ss_item_sk is not null + group by ss_item_sk, d_date) +select * +from (select + item_sk, + d_date, + web_sales, + store_sales, + max(web_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) web_cumulative, + max(store_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) store_cumulative +from (select + case when web.item_sk is not null + then web.item_sk + else store.item_sk end item_sk, + case when web.d_date is not null + then web.d_date + else store.d_date end d_date, + web.cume_sales web_sales, + store.cume_sales store_sales +from web_v1 web full outer join store_v1 store on (web.item_sk = store.item_sk + and web.d_date = store.d_date) + ) x) y +where web_cumulative > store_cumulative +order by item_sk, d_date +limit 100 +-- !query schema +struct +-- !query output + + + +-- !query +with web_v1 as ( + table web_sales + |> join date_dim + |> where ws_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ws_item_sk is not null + |> aggregate sum(ws_sales_price) as sum_ws_sales_price + group by ws_item_sk as item_sk, d_date + |> extend sum(sum_ws_sales_price) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) + as cume_sales), +store_v1 as ( + table store_sales + |> join date_dim + |> where ss_sold_date_sk = d_date_sk + and d_month_seq between 1200 and 1200 + 11 + and ss_item_sk is not null + |> aggregate sum(ss_sales_price) as sum_ss_sales_price + group by ss_item_sk as item_sk, d_date + |> extend sum(sum_ss_sales_price) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) + as cume_sales) +table web_v1 +|> as web +|> full outer join store_v1 store + on (web.item_sk = store.item_sk and web.d_date = store.d_date) +|> select + case when web.item_sk is not null + then web.item_sk + else store.item_sk end item_sk, + case when web.d_date is not null + then web.d_date + else store.d_date end d_date, + web.cume_sales web_sales, + store.cume_sales store_sales +|> as x +|> select + item_sk, + d_date, + web_sales, + store_sales, + max(web_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) web_cumulative, + max(store_sales) + over (partition by item_sk + order by d_date + rows between unbounded preceding and current row) store_cumulative +|> as y +|> where web_cumulative > store_cumulative +|> order by item_sk, d_date +|> limit 100 +-- !query schema +struct +-- !query output + + + +-- !query +drop table t +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table other +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table st +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/pivot.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/pivot.sql.out new file mode 100644 index 000000000000..aaafb19cb979 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/pivot.sql.out @@ -0,0 +1,564 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view courseSales as select * from values + ("dotNET", 2012, 10000), + ("Java", 2012, 20000), + ("dotNET", 2012, 5000), + ("dotNET", 2013, 48000), + ("Java", 2013, 30000) + as courseSales(course, year, earnings) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view years as select * from values + (2012, 1), + (2013, 2) + as years(y, s) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view yearsWithComplexTypes as select * from values + (2012, array(1, 1), map('1', 1), struct(1, 'a')), + (2013, array(2, 2), map('2', 2), struct(2, 'b')) + as yearsWithComplexTypes(y, a, m, s) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(earnings) + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct +-- !query output +2012 15000 20000 +2013 48000 30000 + + +-- !query +SELECT * FROM courseSales +PIVOT ( + sum(earnings) + FOR year IN (2012, 2013) +) +-- !query schema +struct +-- !query output +Java 20000 30000 +dotNET 15000 48000 + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(earnings), avg(earnings) + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct +-- !query output +2012 15000 7500.0 20000 20000.0 +2013 48000 48000.0 30000 30000.0 + + +-- !query +SELECT * FROM ( + SELECT course, earnings FROM courseSales +) +PIVOT ( + sum(earnings) + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct +-- !query output +63000 50000 + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(earnings), min(year) + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct +-- !query output +63000 2012 50000 2012 + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + sum(earnings) + FOR s IN (1, 2) +) +-- !query schema +struct +-- !query output +Java 2012 20000 NULL +Java 2013 NULL 30000 +dotNET 2012 15000 NULL +dotNET 2013 NULL 48000 + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + sum(earnings), min(s) + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct +-- !query output +2012 15000 1 20000 1 +2013 48000 2 30000 2 + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + sum(earnings * s) + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct +-- !query output +2012 15000 20000 +2013 96000 60000 + + +-- !query +SELECT 2012_s, 2013_s, 2012_a, 2013_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + sum(e) s, avg(e) a + FOR y IN (2012, 2013) +) +-- !query schema +struct<2012_s:bigint,2013_s:bigint,2012_a:double,2013_a:double,c:string> +-- !query output +15000 48000 7500.0 48000.0 dotNET +20000 30000 20000.0 30000.0 Java + + +-- !query +SELECT firstYear_s, secondYear_s, firstYear_a, secondYear_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + sum(e) s, avg(e) a + FOR y IN (2012 as firstYear, 2013 secondYear) +) +-- !query schema +struct +-- !query output +15000 48000 7500.0 48000.0 dotNET +20000 30000 20000.0 30000.0 Java + + +-- !query +SELECT * FROM courseSales +PIVOT ( + abs(earnings) + FOR year IN (2012, 2013) +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1006", + "messageParameters" : { + "sql" : "coursesales.earnings" + } +} + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(earnings), year + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1006", + "messageParameters" : { + "sql" : "__auto_generated_subquery_name.year" + } +} + + +-- !query +SELECT * FROM ( + SELECT course, earnings FROM courseSales +) +PIVOT ( + sum(earnings) + FOR year IN (2012, 2013) +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`year`", + "proposal" : "`course`, `earnings`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 62, + "stopIndex" : 113, + "fragment" : "PIVOT (\n sum(earnings)\n FOR year IN (2012, 2013)\n)" + } ] +} + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + ceil(sum(earnings)), avg(earnings) + 1 as a1 + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct +-- !query output +2012 15000 7501.0 20000 20001.0 +2013 48000 48001.0 30000 30001.0 + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(avg(earnings)) + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NESTED_AGGREGATE_FUNCTION", + "sqlState" : "42607", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 82, + "stopIndex" : 94, + "fragment" : "avg(earnings)" + } ] +} + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + sum(earnings) + FOR (course, year) IN (('dotNET', 2012), ('Java', 2013)) +) +-- !query schema +struct +-- !query output +1 15000 NULL +2 NULL 30000 + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + sum(earnings) + FOR (course, s) IN (('dotNET', 2) as c1, ('Java', 1) as c2) +) +-- !query schema +struct +-- !query output +2012 NULL 20000 +2013 48000 NULL + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + sum(earnings) + FOR (course, year) IN ('dotNET', 'Java') +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PIVOT_VALUE_DATA_TYPE_MISMATCH", + "sqlState" : "42K09", + "messageParameters" : { + "pivotType" : "struct", + "value" : "dotNET", + "valueType" : "string" + } +} + + +-- !query +SELECT * FROM courseSales +PIVOT ( + sum(earnings) + FOR year IN (s, 2013) +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`s`", + "proposal" : "`year`, `course`, `earnings`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 66, + "stopIndex" : 66, + "fragment" : "s" + } ] +} + + +-- !query +SELECT * FROM courseSales +PIVOT ( + sum(earnings) + FOR year IN (course, 2013) +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NON_LITERAL_PIVOT_VALUES", + "sqlState" : "42K08", + "messageParameters" : { + "expression" : "\"course\"" + } +} + + +-- !query +SELECT * FROM ( + SELECT course, year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + min(a) + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct,Java:array> +-- !query output +2012 [1,1] [1,1] +2013 [2,2] [2,2] + + +-- !query +SELECT * FROM ( + SELECT course, year, y, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + max(a) + FOR (y, course) IN ((2012, 'dotNET'), (2013, 'Java')) +) +-- !query schema +struct,{2013, Java}:array> +-- !query output +2012 [1,1] NULL +2013 NULL [2,2] + + +-- !query +SELECT * FROM ( + SELECT earnings, year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + sum(earnings) + FOR a IN (array(1, 1), array(2, 2)) +) +-- !query schema +struct +-- !query output +2012 35000 NULL +2013 NULL 78000 + + +-- !query +SELECT * FROM ( + SELECT course, earnings, year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + sum(earnings) + FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2))) +) +-- !query schema +struct +-- !query output +2012 15000 NULL +2013 NULL 30000 + + +-- !query +SELECT * FROM ( + SELECT earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + sum(earnings) + FOR s IN ((1, 'a'), (2, 'b')) +) +-- !query schema +struct +-- !query output +2012 35000 NULL +2013 NULL 78000 + + +-- !query +SELECT * FROM ( + SELECT course, earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + sum(earnings) + FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b'))) +) +-- !query schema +struct +-- !query output +2012 15000 NULL +2013 NULL 30000 + + +-- !query +SELECT * FROM ( + SELECT earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + sum(earnings) + FOR m IN (map('1', 1), map('2', 2)) +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPARABLE_PIVOT_COLUMN", + "sqlState" : "42818", + "messageParameters" : { + "columnName" : "`m`" + } +} + + +-- !query +SELECT * FROM ( + SELECT course, earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + sum(earnings) + FOR (course, m) IN (('dotNET', map('1', 1)), ('Java', map('2', 2))) +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPARABLE_PIVOT_COLUMN", + "sqlState" : "42818", + "messageParameters" : { + "columnName" : "`named_struct('course', __auto_generated_subquery_name`.`course, 'm', __auto_generated_subquery_name`.`m)`" + } +} + + +-- !query +SELECT * FROM ( + SELECT course, earnings, "a" as a, "z" as z, "b" as b, "y" as y, "c" as c, "x" as x, "d" as d, "w" as w + FROM courseSales +) +PIVOT ( + sum(Earnings) + FOR Course IN ('dotNET', 'Java') +) +-- !query schema +struct +-- !query output +a z b y c x d w 63000 50000 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/aggregates_part1.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/aggregates_part1.sql.out new file mode 100644 index 000000000000..42cf942add48 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/aggregates_part1.sql.out @@ -0,0 +1,525 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT avg(four) AS avg_1 FROM onek +-- !query schema +struct +-- !query output +1.5 + + +-- !query +SELECT avg(a) AS avg_32 FROM aggtest WHERE a < 100 +-- !query schema +struct +-- !query output +32.666666666666664 + + +-- !query +select CAST(avg(b) AS Decimal(10,3)) AS avg_107_943 FROM aggtest +-- !query schema +struct +-- !query output +107.943 + + +-- !query +SELECT sum(four) AS sum_1500 FROM onek +-- !query schema +struct +-- !query output +1500 + + +-- !query +SELECT sum(a) AS sum_198 FROM aggtest +-- !query schema +struct +-- !query output +198 + + +-- !query +SELECT sum(b) AS avg_431_773 FROM aggtest +-- !query schema +struct +-- !query output +431.77260909229517 + + +-- !query +SELECT max(four) AS max_3 FROM onek +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT max(a) AS max_100 FROM aggtest +-- !query schema +struct +-- !query output +100 + + +-- !query +SELECT max(aggtest.b) AS max_324_78 FROM aggtest +-- !query schema +struct +-- !query output +324.78 + + +-- !query +SELECT stddev_pop(b) FROM aggtest +-- !query schema +struct +-- !query output +131.10703231895047 + + +-- !query +SELECT stddev_samp(b) FROM aggtest +-- !query schema +struct +-- !query output +151.38936080399804 + + +-- !query +SELECT var_pop(b) FROM aggtest +-- !query schema +struct +-- !query output +17189.053923482323 + + +-- !query +SELECT var_samp(b) FROM aggtest +-- !query schema +struct +-- !query output +22918.738564643096 + + +-- !query +SELECT stddev_pop(CAST(b AS Decimal(38,0))) FROM aggtest +-- !query schema +struct +-- !query output +131.18117242958306 + + +-- !query +SELECT stddev_samp(CAST(b AS Decimal(38,0))) FROM aggtest +-- !query schema +struct +-- !query output +151.47497042966097 + + +-- !query +SELECT var_pop(CAST(b AS Decimal(38,0))) FROM aggtest +-- !query schema +struct +-- !query output +17208.5 + + +-- !query +SELECT var_samp(CAST(b AS Decimal(38,0))) FROM aggtest +-- !query schema +struct +-- !query output +22944.666666666668 + + +-- !query +SELECT var_pop(1.0), var_samp(2.0) +-- !query schema +struct +-- !query output +0.0 NULL + + +-- !query +SELECT stddev_pop(CAST(3.0 AS Decimal(38,0))), stddev_samp(CAST(4.0 AS Decimal(38,0))) +-- !query schema +struct +-- !query output +0.0 NULL + + +-- !query +select sum(CAST(null AS int)) from range(1,4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select sum(CAST(null AS long)) from range(1,4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select sum(CAST(null AS Decimal(38,0))) from range(1,4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select sum(CAST(null AS DOUBLE)) from range(1,4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select avg(CAST(null AS int)) from range(1,4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select avg(CAST(null AS long)) from range(1,4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select avg(CAST(null AS Decimal(38,0))) from range(1,4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select avg(CAST(null AS DOUBLE)) from range(1,4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select sum(CAST('NaN' AS DOUBLE)) from range(1,4) +-- !query schema +struct +-- !query output +NaN + + +-- !query +select avg(CAST('NaN' AS DOUBLE)) from range(1,4) +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES (CAST('1' AS DOUBLE)), (CAST('infinity' AS DOUBLE))) v(x) +-- !query schema +struct +-- !query output +Infinity NaN + + +-- !query +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES ('infinity'), ('1')) v(x) +-- !query schema +struct +-- !query output +Infinity NaN + + +-- !query +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES ('infinity'), ('infinity')) v(x) +-- !query schema +struct +-- !query output +Infinity NaN + + +-- !query +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES ('-infinity'), ('infinity')) v(x) +-- !query schema +struct +-- !query output +NaN NaN + + +-- !query +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x) +-- !query schema +struct +-- !query output +1.00000005E8 2.5 + + +-- !query +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES (7000000000005), (7000000000007)) v(x) +-- !query schema +struct +-- !query output +7.000000000006E12 1.0 + + +-- !query +SELECT regr_count(b, a) FROM aggtest +-- !query schema +struct +-- !query output +4 + + +-- !query +SELECT regr_sxx(b, a) FROM aggtest +-- !query schema +struct +-- !query output +5099.0 + + +-- !query +SELECT regr_syy(b, a) FROM aggtest +-- !query schema +struct +-- !query output +68756.21569392929 + + +-- !query +SELECT regr_sxy(b, a) FROM aggtest +-- !query schema +struct +-- !query output +2614.5158215500414 + + +-- !query +SELECT regr_avgx(b, a), regr_avgy(b, a) FROM aggtest +-- !query schema +struct +-- !query output +49.5 107.94315227307379 + + +-- !query +SELECT regr_r2(b, a) FROM aggtest +-- !query schema +struct +-- !query output +0.019497798203180258 + + +-- !query +SELECT regr_slope(b, a), regr_intercept(b, a) FROM aggtest +-- !query schema +struct +-- !query output +0.5127507004412711 82.56199260123087 + + +-- !query +SELECT covar_pop(b, a), covar_samp(b, a) FROM aggtest +-- !query schema +struct +-- !query output +653.6289553875104 871.5052738500139 + + +-- !query +SELECT corr(b, a) FROM aggtest +-- !query schema +struct +-- !query output +0.1396345165178734 + + +-- !query +CREATE TEMPORARY VIEW regr_test AS SELECT * FROM VALUES (10,150),(20,250),(30,350),(80,540),(100,200) AS regr_test (x, y) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test WHERE x IN (10,20,30,80) +-- !query schema +struct +-- !query output +4 140 2900.0 1290 83075.0 15050.0 + + +-- !query +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test +-- !query schema +struct +-- !query output +5 240 6280.0 1490 95080.0 8680.0 + + +-- !query +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test WHERE x IN (10,20,30) +-- !query schema +struct +-- !query output +3 60 200.0 750 20000.0 2000.0 + + +-- !query +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test WHERE x IN (80,100) +-- !query schema +struct +-- !query output +2 180 200.0 740 57800.0 -3400.0 + + +-- !query +DROP VIEW regr_test +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT count(four) AS cnt_1000 FROM onek +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT count(DISTINCT four) AS cnt_4 FROM onek +-- !query schema +struct +-- !query output +4 + + +-- !query +select ten, count(*), sum(four) from onek +group by ten order by ten +-- !query schema +struct +-- !query output +0 100 100 +1 100 200 +2 100 100 +3 100 200 +4 100 100 +5 100 200 +6 100 100 +7 100 200 +8 100 100 +9 100 200 + + +-- !query +select ten, count(four), sum(DISTINCT four) from onek +group by ten order by ten +-- !query schema +struct +-- !query output +0 100 2 +1 100 4 +2 100 2 +3 100 4 +4 100 2 +5 100 4 +6 100 2 +7 100 4 +8 100 2 +9 100 4 + + +-- !query +select ten, sum(distinct four) from onek a +group by ten +having exists (select 1 from onek b where sum(distinct a.four) = b.four) +-- !query schema +struct +-- !query output +0 2 +2 2 +4 2 +6 2 +8 2 + + +-- !query +select ten, sum(distinct four) from onek a +group by ten +having exists (select 1 from onek b + where sum(distinct a.four + b.four) = b.four) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "sqlState" : "0A000", + "messageParameters" : { + "function" : "sum(DISTINCT (outer(a.four) + b.four))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 114, + "stopIndex" : 142, + "fragment" : "sum(distinct a.four + b.four)" + } ] +} + + +-- !query +select + (select max((select i.unique2 from tenk1 i where i.unique1 = o.unique1))) +from tenk1 o +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`o`.`unique1`", + "proposal" : "`i`.`unique1`, `i`.`unique2`, `i`.`even`, `i`.`four`, `i`.`odd`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 71, + "stopIndex" : 79, + "fragment" : "o.unique1" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/aggregates_part2.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/aggregates_part2.sql.out new file mode 100644 index 000000000000..f9a816286ba1 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/aggregates_part2.sql.out @@ -0,0 +1,300 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (3, 3, 3, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT BIT_AND(b1) AS n1, BIT_OR(b2) AS n2 FROM bitwise_test where 1 = 0 +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +SELECT BIT_AND(b4) AS n1, BIT_OR(b4) AS n2 FROM bitwise_test where b4 is null +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +SELECT + BIT_AND(cast(b1 as tinyint)) AS a1, + BIT_AND(cast(b2 as smallint)) AS b1, + BIT_AND(b3) AS c1, + BIT_AND(b4) AS d1, + BIT_OR(cast(b1 as tinyint)) AS e7, + BIT_OR(cast(b2 as smallint)) AS f7, + BIT_OR(b3) AS g7, + BIT_OR(b4) AS h3 +FROM bitwise_test +-- !query schema +struct +-- !query output +1 1 1 1 7 7 7 3 + + +-- !query +SELECT b1 , bit_and(b2), bit_or(b4) FROM bitwise_test GROUP BY b1 +-- !query schema +struct +-- !query output +1 1 1 +3 3 NULL +7 7 3 + + +-- !query +SELECT b1, bit_and(b2) FROM bitwise_test GROUP BY b1 HAVING bit_and(b2) < 7 +-- !query schema +struct +-- !query output +1 1 +3 3 + + +-- !query +SELECT b1, b2, bit_and(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test +-- !query schema +struct +-- !query output +1 1 1 +3 3 3 +7 7 7 + + +-- !query +SELECT b1, b2, bit_or(b2) OVER (PARTITION BY b1 ORDER BY b2) FROM bitwise_test +-- !query schema +struct +-- !query output +1 1 1 +3 3 3 +7 7 7 + + +-- !query +SELECT + (NULL AND NULL) IS NULL AS `t`, + (TRUE AND NULL) IS NULL AS `t`, + (FALSE AND NULL) IS NULL AS `t`, + (NULL AND TRUE) IS NULL AS `t`, + (NULL AND FALSE) IS NULL AS `t`, + (TRUE AND TRUE) AS `t`, + NOT (TRUE AND FALSE) AS `t`, + NOT (FALSE AND TRUE) AS `t`, + NOT (FALSE AND FALSE) AS `t` +-- !query schema +struct +-- !query output +true true false true false true true true true + + +-- !query +SELECT + (NULL OR NULL) IS NULL AS `t`, + (TRUE OR NULL) IS NULL AS `t`, + (FALSE OR NULL) IS NULL AS `t`, + (NULL OR TRUE) IS NULL AS `t`, + (NULL OR FALSE) IS NULL AS `t`, + (TRUE OR TRUE) AS `t`, + (TRUE OR FALSE) AS `t`, + (FALSE OR TRUE) AS `t`, + NOT (FALSE OR FALSE) AS `t` +-- !query schema +struct +-- !query output +true false true false true true true true true + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW bool_test AS SELECT * FROM VALUES + (TRUE, null, FALSE, null), + (FALSE, TRUE, null, null), + (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT BOOL_AND(b1) AS n1, BOOL_OR(b3) AS n2 FROM bool_test WHERE 1 = 0 +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +SELECT + BOOL_AND(b1) AS f1, + BOOL_AND(b2) AS t2, + BOOL_AND(b3) AS f3, + BOOL_AND(b4) AS n4, + BOOL_AND(NOT b2) AS f5, + BOOL_AND(NOT b3) AS t6 +FROM bool_test +-- !query schema +struct +-- !query output +false true false NULL false true + + +-- !query +SELECT + EVERY(b1) AS f1, + EVERY(b2) AS t2, + EVERY(b3) AS f3, + EVERY(b4) AS n4, + EVERY(NOT b2) AS f5, + EVERY(NOT b3) AS t6 +FROM bool_test +-- !query schema +struct +-- !query output +false true false NULL false true + + +-- !query +SELECT + BOOL_OR(b1) AS t1, + BOOL_OR(b2) AS t2, + BOOL_OR(b3) AS f3, + BOOL_OR(b4) AS n4, + BOOL_OR(NOT b2) AS f5, + BOOL_OR(NOT b3) AS t6 +FROM bool_test +-- !query schema +struct +-- !query output +true true false NULL false true + + +-- !query +select min(unique1) from tenk1 +-- !query schema +struct +-- !query output +0 + + +-- !query +select max(unique1) from tenk1 +-- !query schema +struct +-- !query output +9999 + + +-- !query +select max(unique1) from tenk1 where unique1 < 42 +-- !query schema +struct +-- !query output +41 + + +-- !query +select max(unique1) from tenk1 where unique1 > 42 +-- !query schema +struct +-- !query output +9999 + + +-- !query +select max(unique1) from tenk1 where unique1 > 42000 +-- !query schema +struct +-- !query output +NULL + + +-- !query +select max(tenthous) from tenk1 where thousand = 33 +-- !query schema +struct +-- !query output +9033 + + +-- !query +select min(tenthous) from tenk1 where thousand = 33 +-- !query schema +struct +-- !query output +33 + + +-- !query +select distinct max(unique2) from tenk1 +-- !query schema +struct +-- !query output +9999 + + +-- !query +select max(unique2) from tenk1 order by 1 +-- !query schema +struct +-- !query output +9999 + + +-- !query +select max(unique2) from tenk1 order by max(unique2) +-- !query schema +struct +-- !query output +9999 + + +-- !query +select max(unique2) from tenk1 order by max(unique2)+1 +-- !query schema +struct +-- !query output +9999 + + +-- !query +select t1.max_unique2, g from (select max(unique2) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc +-- !query schema +struct +-- !query output +9999 3 +9999 2 +9999 1 + + +-- !query +select max(100) from tenk1 +-- !query schema +struct +-- !query output +100 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/aggregates_part3.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/aggregates_part3.sql.out new file mode 100644 index 000000000000..038bcc0283a6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/aggregates_part3.sql.out @@ -0,0 +1,59 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select max(min(unique1)) from tenk1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NESTED_AGGREGATE_FUNCTION", + "sqlState" : "42607", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 23, + "fragment" : "min(unique1)" + } ] +} + + +-- !query +select min(unique1) filter (where unique1 > 100) from tenk1 +-- !query schema +struct 100)):int> +-- !query output +101 + + +-- !query +select sum(1/ten) filter (where ten > 0) from tenk1 +-- !query schema +struct 0)):double> +-- !query output +2828.9682539682954 + + +-- !query +select ten, sum(distinct four) filter (where four > 10) from onek a +group by ten +having exists (select 1 from onek b where sum(distinct a.four) = b.four) +-- !query schema +struct 10)):bigint> +-- !query output +0 NULL +2 NULL +4 NULL +6 NULL +8 NULL + + +-- !query +select (select count(*) + from (values (1)) t0(inner_c)) +from (values (2),(3)) t1(outer_c) +-- !query schema +struct +-- !query output +1 +1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/aggregates_part4.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/aggregates_part4.sql.out new file mode 100644 index 000000000000..1aaa514eb139 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/aggregates_part4.sql.out @@ -0,0 +1,31 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select percentile_cont(0.5) within group (order by b) from aggtest +-- !query schema +struct +-- !query output +53.44850015640259 + + +-- !query +select percentile_cont(0.5) within group (order by b), sum(b) from aggtest +-- !query schema +struct +-- !query output +53.44850015640259 431.77260909229517 + + +-- !query +select percentile_cont(0.5) within group (order by thousand) from tenk1 +-- !query schema +struct +-- !query output +499.5 + + +-- !query +select percentile_disc(0.5) within group (order by thousand) from tenk1 +-- !query schema +struct +-- !query output +499.0 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/boolean.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/boolean.sql.out new file mode 100644 index 000000000000..12660768b95c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/boolean.sql.out @@ -0,0 +1,1108 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT 1 AS one +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT true AS true +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT false AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT boolean('t') AS true +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT boolean(' f ') AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT boolean('true') AS true +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT boolean('test') AS error +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'test'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "boolean('test')" + } ] +} + + +-- !query +SELECT boolean('false') AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT boolean('foo') AS error +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'foo'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "boolean('foo')" + } ] +} + + +-- !query +SELECT boolean('y') AS true +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT boolean('yes') AS true +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT boolean('yeah') AS error +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'yeah'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "boolean('yeah')" + } ] +} + + +-- !query +SELECT boolean('n') AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT boolean('no') AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT boolean('nay') AS error +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'nay'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "boolean('nay')" + } ] +} + + +-- !query +SELECT boolean('on') AS true +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'on'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "boolean('on')" + } ] +} + + +-- !query +SELECT boolean('off') AS `false` +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'off'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "boolean('off')" + } ] +} + + +-- !query +SELECT boolean('of') AS `false` +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'of'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "boolean('of')" + } ] +} + + +-- !query +SELECT boolean('o') AS error +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'o'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "boolean('o')" + } ] +} + + +-- !query +SELECT boolean('on_') AS error +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'on_'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "boolean('on_')" + } ] +} + + +-- !query +SELECT boolean('off_') AS error +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'off_'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "boolean('off_')" + } ] +} + + +-- !query +SELECT boolean('1') AS true +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT boolean('11') AS error +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'11'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "boolean('11')" + } ] +} + + +-- !query +SELECT boolean('0') AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT boolean('000') AS error +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'000'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "boolean('000')" + } ] +} + + +-- !query +SELECT boolean('') AS error +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "''", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "boolean('')" + } ] +} + + +-- !query +SELECT boolean('t') or boolean('f') AS true +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT boolean('t') and boolean('f') AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT not boolean('f') AS true +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT boolean('t') = boolean('f') AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT boolean('t') <> boolean('f') AS true +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT boolean('t') > boolean('f') AS true +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT boolean('t') >= boolean('f') AS true +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT boolean('f') < boolean('t') AS true +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT boolean('f') <= boolean('t') AS true +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT boolean(string('TrUe')) AS true, boolean(string('fAlse')) AS `false` +-- !query schema +struct +-- !query output +true false + + +-- !query +SELECT boolean(string(' true ')) AS true, + boolean(string(' FALSE')) AS `false` +-- !query schema +struct +-- !query output +true false + + +-- !query +SELECT string(boolean(true)) AS true, string(boolean(false)) AS `false` +-- !query schema +struct +-- !query output +true false + + +-- !query +SELECT boolean(string(' tru e ')) AS invalid +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "' tru e '", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "boolean(string(' tru e '))" + } ] +} + + +-- !query +SELECT boolean(string('')) AS invalid +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "''", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "boolean(string(''))" + } ] +} + + +-- !query +CREATE TABLE BOOLTBL1 (f1 boolean) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO BOOLTBL1 VALUES (cast('t' as boolean)) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO BOOLTBL1 VALUES (cast('True' as boolean)) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO BOOLTBL1 VALUES (cast('true' as boolean)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT '' AS t_3, BOOLTBL1.* FROM BOOLTBL1 +-- !query schema +struct +-- !query output + true + true + true + + +-- !query +SELECT '' AS t_3, BOOLTBL1.* + FROM BOOLTBL1 + WHERE f1 = boolean('true') +-- !query schema +struct +-- !query output + true + true + true + + +-- !query +SELECT '' AS t_3, BOOLTBL1.* + FROM BOOLTBL1 + WHERE f1 <> boolean('false') +-- !query schema +struct +-- !query output + true + true + true + + +-- !query +SELECT '' AS zero, BOOLTBL1.* + FROM BOOLTBL1 + WHERE booleq(boolean('false'), f1) +-- !query schema +struct +-- !query output + + + +-- !query +INSERT INTO BOOLTBL1 VALUES (boolean('f')) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT '' AS f_1, BOOLTBL1.* + FROM BOOLTBL1 + WHERE f1 = boolean('false') +-- !query schema +struct +-- !query output + false + + +-- !query +CREATE TABLE BOOLTBL2 (f1 boolean) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO BOOLTBL2 VALUES (boolean('f')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO BOOLTBL2 VALUES (boolean('false')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO BOOLTBL2 VALUES (boolean('False')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO BOOLTBL2 VALUES (boolean('FALSE')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO BOOLTBL2 + VALUES (boolean('XXX')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_INLINE_TABLE.FAILED_SQL_EXPRESSION_EVALUATION", + "sqlState" : "42000", + "messageParameters" : { + "sqlExpr" : "\"XXX\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 47, + "fragment" : "VALUES (boolean('XXX'))" + } ] +} + + +-- !query +SELECT '' AS f_4, BOOLTBL2.* FROM BOOLTBL2 +-- !query schema +struct +-- !query output + false + false + false + false + + +-- !query +SELECT '' AS tf_12, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE BOOLTBL2.f1 <> BOOLTBL1.f1 +-- !query schema +struct +-- !query output + true false + true false + true false + true false + true false + true false + true false + true false + true false + true false + true false + true false + + +-- !query +SELECT '' AS tf_12, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE boolne(BOOLTBL2.f1,BOOLTBL1.f1) +-- !query schema +struct +-- !query output + true false + true false + true false + true false + true false + true false + true false + true false + true false + true false + true false + true false + + +-- !query +SELECT '' AS ff_4, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE BOOLTBL2.f1 = BOOLTBL1.f1 and BOOLTBL1.f1 = boolean('false') +-- !query schema +struct +-- !query output + false false + false false + false false + false false + + +-- !query +SELECT '' AS tf_12_ff_4, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE BOOLTBL2.f1 = BOOLTBL1.f1 or BOOLTBL1.f1 = boolean('true') + ORDER BY BOOLTBL1.f1, BOOLTBL2.f1 +-- !query schema +struct +-- !query output + false false + false false + false false + false false + true false + true false + true false + true false + true false + true false + true false + true false + true false + true false + true false + true false + + +-- !query +SELECT '' AS True, f1 + FROM BOOLTBL1 + WHERE f1 IS TRUE +-- !query schema +struct +-- !query output + true + true + true + + +-- !query +SELECT '' AS `Not False`, f1 + FROM BOOLTBL1 + WHERE f1 IS NOT FALSE +-- !query schema +struct +-- !query output + true + true + true + + +-- !query +SELECT '' AS `False`, f1 + FROM BOOLTBL1 + WHERE f1 IS FALSE +-- !query schema +struct +-- !query output + false + + +-- !query +SELECT '' AS `Not True`, f1 + FROM BOOLTBL1 + WHERE f1 IS NOT TRUE +-- !query schema +struct +-- !query output + false + + +-- !query +SELECT '' AS `True`, f1 + FROM BOOLTBL2 + WHERE f1 IS TRUE +-- !query schema +struct +-- !query output + + + +-- !query +SELECT '' AS `Not False`, f1 + FROM BOOLTBL2 + WHERE f1 IS NOT FALSE +-- !query schema +struct +-- !query output + + + +-- !query +SELECT '' AS `False`, f1 + FROM BOOLTBL2 + WHERE f1 IS FALSE +-- !query schema +struct +-- !query output + false + false + false + false + + +-- !query +SELECT '' AS `Not True`, f1 + FROM BOOLTBL2 + WHERE f1 IS NOT TRUE +-- !query schema +struct +-- !query output + false + false + false + false + + +-- !query +CREATE TABLE BOOLTBL3 (d string, b boolean, o int) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO BOOLTBL3 VALUES ('true', true, 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO BOOLTBL3 VALUES ('false', false, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO BOOLTBL3 VALUES ('null', null, 3) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + d, + b IS TRUE AS istrue, + b IS NOT TRUE AS isnottrue, + b IS FALSE AS isfalse, + b IS NOT FALSE AS isnotfalse, + b IS UNKNOWN AS isunknown, + b IS NOT UNKNOWN AS isnotunknown +FROM booltbl3 ORDER BY o +-- !query schema +struct +-- !query output +true true false false true false true +false false true true false false true +null false true false true true false + + +-- !query +CREATE TABLE booltbl4(isfalse boolean, istrue boolean, isnul boolean) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO booltbl4 VALUES (false, true, null) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT istrue AND isnul AND istrue FROM booltbl4 +-- !query schema +struct<((istrue AND isnul) AND istrue):boolean> +-- !query output +NULL + + +-- !query +SELECT istrue AND istrue AND isnul FROM booltbl4 +-- !query schema +struct<((istrue AND istrue) AND isnul):boolean> +-- !query output +NULL + + +-- !query +SELECT isnul AND istrue AND istrue FROM booltbl4 +-- !query schema +struct<((isnul AND istrue) AND istrue):boolean> +-- !query output +NULL + + +-- !query +SELECT isfalse AND isnul AND istrue FROM booltbl4 +-- !query schema +struct<((isfalse AND isnul) AND istrue):boolean> +-- !query output +false + + +-- !query +SELECT istrue AND isfalse AND isnul FROM booltbl4 +-- !query schema +struct<((istrue AND isfalse) AND isnul):boolean> +-- !query output +false + + +-- !query +SELECT isnul AND istrue AND isfalse FROM booltbl4 +-- !query schema +struct<((isnul AND istrue) AND isfalse):boolean> +-- !query output +false + + +-- !query +SELECT isfalse OR isnul OR isfalse FROM booltbl4 +-- !query schema +struct<((isfalse OR isnul) OR isfalse):boolean> +-- !query output +NULL + + +-- !query +SELECT isfalse OR isfalse OR isnul FROM booltbl4 +-- !query schema +struct<((isfalse OR isfalse) OR isnul):boolean> +-- !query output +NULL + + +-- !query +SELECT isnul OR isfalse OR isfalse FROM booltbl4 +-- !query schema +struct<((isnul OR isfalse) OR isfalse):boolean> +-- !query output +NULL + + +-- !query +SELECT isfalse OR isnul OR istrue FROM booltbl4 +-- !query schema +struct<((isfalse OR isnul) OR istrue):boolean> +-- !query output +true + + +-- !query +SELECT istrue OR isfalse OR isnul FROM booltbl4 +-- !query schema +struct<((istrue OR isfalse) OR isnul):boolean> +-- !query output +true + + +-- !query +SELECT isnul OR istrue OR isfalse FROM booltbl4 +-- !query schema +struct<((isnul OR istrue) OR isfalse):boolean> +-- !query output +true + + +-- !query +DROP TABLE BOOLTBL1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE BOOLTBL2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE BOOLTBL3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE BOOLTBL4 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/case.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/case.sql.out new file mode 100644 index 000000000000..66b018b838c6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/case.sql.out @@ -0,0 +1,406 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE CASE_TBL ( + i integer, + f double +) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE CASE2_TBL ( + i integer, + j integer +) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE_TBL VALUES (1, 10.1) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE_TBL VALUES (2, 20.2) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE_TBL VALUES (3, -30.3) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE_TBL VALUES (4, NULL) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE2_TBL VALUES (1, -1) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE2_TBL VALUES (2, -2) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE2_TBL VALUES (3, -3) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE2_TBL VALUES (2, -4) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE2_TBL VALUES (1, NULL) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE2_TBL VALUES (NULL, -6) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT '3' AS `One`, + CASE + WHEN 1 < 2 THEN 3 + END AS `Simple WHEN` +-- !query schema +struct +-- !query output +3 3 + + +-- !query +SELECT '' AS `One`, + CASE + WHEN 1 > 2 THEN 3 + END AS `Simple default` +-- !query schema +struct +-- !query output + NULL + + +-- !query +SELECT '3' AS `One`, + CASE + WHEN 1 < 2 THEN 3 + ELSE 4 + END AS `Simple ELSE` +-- !query schema +struct +-- !query output +3 3 + + +-- !query +SELECT '4' AS `One`, + CASE + WHEN 1 > 2 THEN 3 + ELSE 4 + END AS `ELSE default` +-- !query schema +struct +-- !query output +4 4 + + +-- !query +SELECT '6' AS `One`, + CASE + WHEN 1 > 2 THEN 3 + WHEN 4 < 5 THEN 6 + ELSE 7 + END AS `Two WHEN with default` +-- !query schema +struct +-- !query output +6 6 + + +-- !query +SELECT '7' AS `None`, + CASE WHEN rand() < 0 THEN 1 + END AS `NULL on no matches` +-- !query schema +struct +-- !query output +7 NULL + + +-- !query +SELECT CASE WHEN 1=0 THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE 1 WHEN 0 THEN 1/0 WHEN 1 THEN 1 ELSE 2/0 END +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN i > 100 THEN 1/0 ELSE 0 END FROM case_tbl +-- !query schema +struct 100) THEN (1 / 0) ELSE 0 END:double> +-- !query output +0.0 +0.0 +0.0 +0.0 + + +-- !query +SELECT CASE 'a' WHEN 'a' THEN 1 ELSE 2 END +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT '' AS `Five`, + CASE + WHEN i >= 3 THEN i + END AS `>= 3 or Null` + FROM CASE_TBL +-- !query schema +struct= 3 or Null:int> +-- !query output + 3 + 4 + NULL + NULL + + +-- !query +SELECT '' AS `Five`, + CASE WHEN i >= 3 THEN (i + i) + ELSE i + END AS `Simplest Math` + FROM CASE_TBL +-- !query schema +struct +-- !query output + 1 + 2 + 6 + 8 + + +-- !query +SELECT '' AS `Five`, i AS `Value`, + CASE WHEN (i < 0) THEN 'small' + WHEN (i = 0) THEN 'zero' + WHEN (i = 1) THEN 'one' + WHEN (i = 2) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL +-- !query schema +struct +-- !query output + 1 one + 2 two + 3 big + 4 big + + +-- !query +SELECT '' AS `Five`, + CASE WHEN ((i < 0) or (i < 0)) THEN 'small' + WHEN ((i = 0) or (i = 0)) THEN 'zero' + WHEN ((i = 1) or (i = 1)) THEN 'one' + WHEN ((i = 2) or (i = 2)) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL +-- !query schema +struct +-- !query output + big + big + one + two + + +-- !query +SELECT * FROM CASE_TBL WHERE COALESCE(f,i) = 4 +-- !query schema +struct +-- !query output +4 NULL + + +-- !query +SELECT * FROM CASE_TBL WHERE NULLIF(f,i) = 2 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT COALESCE(a.f, b.i, b.j) + FROM CASE_TBL a, CASE2_TBL b +-- !query schema +struct +-- !query output +-30.3 +-30.3 +-30.3 +-30.3 +-30.3 +-30.3 +-6.0 +1.0 +1.0 +10.1 +10.1 +10.1 +10.1 +10.1 +10.1 +2.0 +2.0 +20.2 +20.2 +20.2 +20.2 +20.2 +20.2 +3.0 + + +-- !query +SELECT * + FROM CASE_TBL a, CASE2_TBL b + WHERE COALESCE(a.f, b.i, b.j) = 2 +-- !query schema +struct +-- !query output +4 NULL 2 -2 +4 NULL 2 -4 + + +-- !query +SELECT '' AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, + NULLIF(b.i, 4) AS `NULLIF(b.i,4)` + FROM CASE_TBL a, CASE2_TBL b +-- !query schema +struct +-- !query output + 1 2 + 1 2 + 1 3 + 1 NULL + 2 1 + 2 1 + 2 3 + 2 NULL + 3 1 + 3 1 + 3 2 + 3 2 + 3 NULL + 4 1 + 4 1 + 4 2 + 4 2 + 4 3 + 4 NULL + NULL 1 + NULL 1 + NULL 2 + NULL 2 + NULL 3 + + +-- !query +SELECT '' AS `Two`, * + FROM CASE_TBL a, CASE2_TBL b + WHERE COALESCE(f,b.i) = 2 +-- !query schema +struct +-- !query output + 4 NULL 2 -2 + 4 NULL 2 -4 + + +-- !query +SELECT CASE + (CASE vol('bar') + WHEN 'foo' THEN 'it was foo!' + WHEN vol(null) THEN 'null input' + WHEN 'bar' THEN 'it was bar!' END + ) + WHEN 'it was foo!' THEN 'foo recognized' + WHEN 'it was bar!' THEN 'bar recognized' + ELSE 'unrecognized' END +-- !query schema +struct +-- !query output +bar recognized + + +-- !query +DROP TABLE CASE_TBL +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE CASE2_TBL +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/comments.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/comments.sql.out new file mode 100644 index 000000000000..44db80e48887 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/comments.sql.out @@ -0,0 +1,71 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT 'trailing' AS first +-- !query schema +struct +-- !query output +trailing + + +-- !query +SELECT /* embedded single line */ 'embedded' AS `second` +-- !query schema +struct +-- !query output +embedded + + +-- !query +SELECT /* both embedded and trailing single line */ 'both' AS third +-- !query schema +struct +-- !query output +both + + +-- !query +SELECT 'before multi-line' AS fourth +-- !query schema +struct +-- !query output +before multi-line + + +-- !query +/* This is an example of SQL which should not execute: + * select 'multi-line'; + */ +SELECT 'after multi-line' AS fifth +-- !query schema +struct +-- !query output +after multi-line + + +-- !query +/* +SELECT 'trailing' as x1; -- inside block comment +*/ + +/* This block comment surrounds a query which itself has a block comment... +SELECT /* embedded single line */ 'embedded' AS x2; +*/ + +SELECT -- continued after the following block comments... +/* Deeply nested comment. + This includes a single apostrophe to make sure we aren't decoding this part as a string. +SELECT 'deep nest' AS n1; +/* Second level of nesting... +SELECT 'deeper nest' as n2; +/* Third level of nesting... +SELECT 'deepest nest' as n3; +*/ +Hoo boy. Still two deep... +*/ +Now just one deep... +*/ +'deeply nested example' AS sixth +-- !query schema +struct +-- !query output +deeply nested example diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/create_view.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/create_view.sql.out new file mode 100644 index 000000000000..90c26b858292 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/create_view.sql.out @@ -0,0 +1,2228 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE emp ( + name string, + age int, + salary int, + manager string +) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW toyemp AS + SELECT name, age, /* location ,*/ 12*salary AS annualsal + FROM emp +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW toyemp +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE emp +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE view_base_table (key int /* PRIMARY KEY */, data varchar(20)) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW key_dependent_view AS + SELECT * FROM view_base_table GROUP BY key +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"data\"", + "expressionAnyValue" : "\"any_value(data)\"" + } +} + + +-- !query +CREATE VIEW key_dependent_view_no_cols AS + SELECT FROM view_base_table GROUP BY key HAVING length(data) > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`FROM`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 53, + "stopIndex" : 56, + "fragment" : "FROM" + } ] +} + + +-- !query +CREATE TABLE viewtest_tbl (a int, b int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO viewtest_tbl VALUES (5, 10), (10, 15), (15, 20), (20, 25) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW viewtest AS + SELECT * FROM viewtest_tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW viewtest AS + SELECT * FROM viewtest_tbl WHERE a > 10 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM viewtest +-- !query schema +struct +-- !query output +15 20 +20 25 + + +-- !query +CREATE OR REPLACE VIEW viewtest AS + SELECT a, b FROM viewtest_tbl WHERE a > 5 ORDER BY b DESC +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM viewtest +-- !query schema +struct +-- !query output +20 25 +15 20 +10 15 + + +-- !query +CREATE OR REPLACE VIEW viewtest AS + SELECT a FROM viewtest_tbl WHERE a <> 20 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW viewtest AS + SELECT 1, * FROM viewtest_tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW viewtest AS + SELECT a, decimal(b) FROM viewtest_tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW viewtest AS + SELECT a, b, 0 AS c FROM viewtest_tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW viewtest +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE viewtest_tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE SCHEMA temp_view_test +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE temp_view_test.base_table (a int, id int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE temp_view_test.base_table2 (a int, id int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +USE temp_view_test +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW temp_table AS SELECT * FROM VALUES + (1, 1) as temp_table(a, id) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v1 AS SELECT * FROM base_table +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE EXTENDED v1 +-- !query schema +struct +-- !query output +a int +id int + +# Detailed Table Information +Catalog spark_catalog +Database temp_view_test +Table v1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM base_table +View Original Text SELECT * FROM base_table +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.temp_view_test +View Query Output Columns [`a`, `id`] + + +-- !query +CREATE VIEW v1_temp AS SELECT * FROM temp_table +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "sqlState" : "42K0F", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`temp_view_test`.`v1_temp`", + "tempObj" : "VIEW", + "tempObjName" : "`temp_table`" + } +} + + +-- !query +CREATE TEMP VIEW v2_temp AS SELECT * FROM base_table +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE EXTENDED v2_temp +-- !query schema +struct +-- !query output +a int +id int + + +-- !query +CREATE VIEW temp_view_test.v2 AS SELECT * FROM base_table +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE EXTENDED temp_view_test.v2 +-- !query schema +struct +-- !query output +a int +id int + +# Detailed Table Information +Catalog spark_catalog +Database temp_view_test +Table v2 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM base_table +View Original Text SELECT * FROM base_table +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.temp_view_test +View Query Output Columns [`a`, `id`] + + +-- !query +CREATE VIEW temp_view_test.v3_temp AS SELECT * FROM temp_table +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "sqlState" : "42K0F", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`temp_view_test`.`v3_temp`", + "tempObj" : "VIEW", + "tempObjName" : "`temp_table`" + } +} + + +-- !query +CREATE VIEW v3 AS + SELECT t1.a AS t1_a, t2.a AS t2_a + FROM base_table t1, base_table2 t2 + WHERE t1.id = t2.id +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE EXTENDED v3 +-- !query schema +struct +-- !query output +t1_a int +t2_a int + +# Detailed Table Information +Catalog spark_catalog +Database temp_view_test +Table v3 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT t1.a AS t1_a, t2.a AS t2_a + FROM base_table t1, base_table2 t2 + WHERE t1.id = t2.id +View Original Text SELECT t1.a AS t1_a, t2.a AS t2_a + FROM base_table t1, base_table2 t2 + WHERE t1.id = t2.id +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.temp_view_test +View Query Output Columns [`t1_a`, `t2_a`] + + +-- !query +CREATE VIEW v4_temp AS + SELECT t1.a AS t1_a, t2.a AS t2_a + FROM base_table t1, temp_table t2 + WHERE t1.id = t2.id +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "sqlState" : "42K0F", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`temp_view_test`.`v4_temp`", + "tempObj" : "VIEW", + "tempObjName" : "`temp_table`" + } +} + + +-- !query +CREATE VIEW v5_temp AS + SELECT t1.a AS t1_a, t2.a AS t2_a, t3.a AS t3_a + FROM base_table t1, base_table2 t2, temp_table t3 + WHERE t1.id = t2.id and t2.id = t3.id +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "sqlState" : "42K0F", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`temp_view_test`.`v5_temp`", + "tempObj" : "VIEW", + "tempObjName" : "`temp_table`" + } +} + + +-- !query +CREATE VIEW v4 AS SELECT * FROM base_table WHERE id IN (SELECT id FROM base_table2) +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE EXTENDED v4 +-- !query schema +struct +-- !query output +a int +id int + +# Detailed Table Information +Catalog spark_catalog +Database temp_view_test +Table v4 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM base_table WHERE id IN (SELECT id FROM base_table2) +View Original Text SELECT * FROM base_table WHERE id IN (SELECT id FROM base_table2) +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.temp_view_test +View Query Output Columns [`a`, `id`] + + +-- !query +CREATE VIEW v5 AS SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM base_table2) t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE EXTENDED v5 +-- !query schema +struct +-- !query output +id int +a int + +# Detailed Table Information +Catalog spark_catalog +Database temp_view_test +Table v5 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM base_table2) t2 +View Original Text SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM base_table2) t2 +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.temp_view_test +View Query Output Columns [`id`, `a`] + + +-- !query +CREATE VIEW v6 AS SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM base_table2) +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE EXTENDED v6 +-- !query schema +struct +-- !query output +a int +id int + +# Detailed Table Information +Catalog spark_catalog +Database temp_view_test +Table v6 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM base_table2) +View Original Text SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM base_table2) +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.temp_view_test +View Query Output Columns [`a`, `id`] + + +-- !query +CREATE VIEW v7 AS SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM base_table2) +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE EXTENDED v7 +-- !query schema +struct +-- !query output +a int +id int + +# Detailed Table Information +Catalog spark_catalog +Database temp_view_test +Table v7 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM base_table2) +View Original Text SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM base_table2) +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.temp_view_test +View Query Output Columns [`a`, `id`] + + +-- !query +CREATE VIEW v8 AS SELECT * FROM base_table WHERE EXISTS (SELECT 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE EXTENDED v8 +-- !query schema +struct +-- !query output +a int +id int + +# Detailed Table Information +Catalog spark_catalog +Database temp_view_test +Table v8 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM base_table WHERE EXISTS (SELECT 1) +View Original Text SELECT * FROM base_table WHERE EXISTS (SELECT 1) +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.temp_view_test +View Query Output Columns [`a`, `id`] + + +-- !query +CREATE VIEW v6_temp AS SELECT * FROM base_table WHERE id IN (SELECT id FROM temp_table) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "sqlState" : "42K0F", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`temp_view_test`.`v6_temp`", + "tempObj" : "VIEW", + "tempObjName" : "`temp_table`" + } +} + + +-- !query +CREATE VIEW v7_temp AS SELECT t1.id, t2.a FROM base_table t1, (SELECT * FROM temp_table) t2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "sqlState" : "42K0F", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`temp_view_test`.`v7_temp`", + "tempObj" : "VIEW", + "tempObjName" : "`temp_table`" + } +} + + +-- !query +CREATE VIEW v8_temp AS SELECT * FROM base_table WHERE EXISTS (SELECT 1 FROM temp_table) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "sqlState" : "42K0F", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`temp_view_test`.`v8_temp`", + "tempObj" : "VIEW", + "tempObjName" : "`temp_table`" + } +} + + +-- !query +CREATE VIEW v9_temp AS SELECT * FROM base_table WHERE NOT EXISTS (SELECT 1 FROM temp_table) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "sqlState" : "42K0F", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`temp_view_test`.`v9_temp`", + "tempObj" : "VIEW", + "tempObjName" : "`temp_table`" + } +} + + +-- !query +CREATE VIEW v10_temp AS SELECT * FROM v7_temp +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`v7_temp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 45, + "fragment" : "v7_temp" + } ] +} + + +-- !query +CREATE VIEW v11_temp AS SELECT t1.id, t2.a FROM base_table t1, v10_temp t2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`v10_temp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 64, + "stopIndex" : 71, + "fragment" : "v10_temp" + } ] +} + + +-- !query +CREATE VIEW v12_temp AS SELECT true FROM v11_temp +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`v11_temp`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 42, + "stopIndex" : 49, + "fragment" : "v11_temp" + } ] +} + + +-- !query +CREATE SCHEMA testviewschm2 +-- !query schema +struct<> +-- !query output + + + +-- !query +USE testviewschm2 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t1 (num int, name string) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t2 (num2 int, value string) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMP VIEW tt AS SELECT * FROM VALUES + (1, 'a') AS tt(num2, value) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW nontemp1 AS SELECT * FROM t1 CROSS JOIN t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE EXTENDED nontemp1 +-- !query schema +struct +-- !query output +num int +name string +num2 int +value string + +# Detailed Table Information +Catalog spark_catalog +Database testviewschm2 +Table nontemp1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t1 CROSS JOIN t2 +View Original Text SELECT * FROM t1 CROSS JOIN t2 +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.testviewschm2 +View Query Output Columns [`num`, `name`, `num2`, `value`] + + +-- !query +CREATE VIEW temporal1 AS SELECT * FROM t1 CROSS JOIN tt +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "sqlState" : "42K0F", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`testviewschm2`.`temporal1`", + "tempObj" : "VIEW", + "tempObjName" : "`tt`" + } +} + + +-- !query +CREATE VIEW nontemp2 AS SELECT * FROM t1 INNER JOIN t2 ON t1.num = t2.num2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE EXTENDED nontemp2 +-- !query schema +struct +-- !query output +num int +name string +num2 int +value string + +# Detailed Table Information +Catalog spark_catalog +Database testviewschm2 +Table nontemp2 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t1 INNER JOIN t2 ON t1.num = t2.num2 +View Original Text SELECT * FROM t1 INNER JOIN t2 ON t1.num = t2.num2 +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.testviewschm2 +View Query Output Columns [`num`, `name`, `num2`, `value`] + + +-- !query +CREATE VIEW temporal2 AS SELECT * FROM t1 INNER JOIN tt ON t1.num = tt.num2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "sqlState" : "42K0F", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`testviewschm2`.`temporal2`", + "tempObj" : "VIEW", + "tempObjName" : "`tt`" + } +} + + +-- !query +CREATE VIEW nontemp3 AS SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE EXTENDED nontemp3 +-- !query schema +struct +-- !query output +num int +name string +num2 int +value string + +# Detailed Table Information +Catalog spark_catalog +Database testviewschm2 +Table nontemp3 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 +View Original Text SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.testviewschm2 +View Query Output Columns [`num`, `name`, `num2`, `value`] + + +-- !query +CREATE VIEW temporal3 AS SELECT * FROM t1 LEFT JOIN tt ON t1.num = tt.num2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "sqlState" : "42K0F", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`testviewschm2`.`temporal3`", + "tempObj" : "VIEW", + "tempObjName" : "`tt`" + } +} + + +-- !query +CREATE VIEW nontemp4 AS SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 AND t2.value = 'xxx' +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE EXTENDED nontemp4 +-- !query schema +struct +-- !query output +num int +name string +num2 int +value string + +# Detailed Table Information +Catalog spark_catalog +Database testviewschm2 +Table nontemp4 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 AND t2.value = 'xxx' +View Original Text SELECT * FROM t1 LEFT JOIN t2 ON t1.num = t2.num2 AND t2.value = 'xxx' +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.testviewschm2 +View Query Output Columns [`num`, `name`, `num2`, `value`] + + +-- !query +CREATE VIEW temporal4 AS SELECT * FROM t1 LEFT JOIN tt ON t1.num = tt.num2 AND tt.value = 'xxx' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "sqlState" : "42K0F", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`testviewschm2`.`temporal4`", + "tempObj" : "VIEW", + "tempObjName" : "`tt`" + } +} + + +-- !query +CREATE VIEW temporal5 AS SELECT * FROM t1 WHERE num IN (SELECT num FROM t1 WHERE EXISTS (SELECT 1 FROM tt)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "sqlState" : "42K0F", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`testviewschm2`.`temporal5`", + "tempObj" : "VIEW", + "tempObjName" : "`tt`" + } +} + + +-- !query +CREATE TABLE tbl1 ( a int, b int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tbl2 (c int, d int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tbl3 (e int, f int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tbl4 (g int, h int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tmptbl (i int, j int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO tmptbl VALUES (1, 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW pubview AS SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE EXTENDED pubview +-- !query schema +struct +-- !query output +a int +b int + +# Detailed Table Information +Catalog spark_catalog +Database testviewschm2 +Table pubview +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) +View Original Text SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.testviewschm2 +View Query Output Columns [`a`, `b`] + + +-- !query +CREATE VIEW mytempview AS SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) +AND NOT EXISTS (SELECT g FROM tbl4 LEFT JOIN tmptbl ON tbl4.h = tmptbl.j) +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE EXTENDED mytempview +-- !query schema +struct +-- !query output +a int +b int + +# Detailed Table Information +Catalog spark_catalog +Database testviewschm2 +Table mytempview +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) +AND NOT EXISTS (SELECT g FROM tbl4 LEFT JOIN tmptbl ON tbl4.h = tmptbl.j) +View Original Text SELECT * FROM tbl1 WHERE tbl1.a +BETWEEN (SELECT d FROM tbl2 WHERE c = 1) AND (SELECT e FROM tbl3 WHERE f = 2) +AND EXISTS (SELECT g FROM tbl4 LEFT JOIN tbl3 ON tbl4.h = tbl3.f) +AND NOT EXISTS (SELECT g FROM tbl4 LEFT JOIN tmptbl ON tbl4.h = tmptbl.j) +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.testviewschm2 +View Query Output Columns [`a`, `b`] + + +-- !query +CREATE VIEW tt1 AS + SELECT * FROM ( + VALUES + ('abc', '0123456789', 42, 'abcd'), + ('0123456789', 'abc', 42.12, 'abc') + ) vv(a,b,c,d) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM tt1 +-- !query schema +struct +-- !query output +0123456789 abc 42.12 abc +abc 0123456789 42.00 abcd + + +-- !query +SELECT string(a) FROM tt1 +-- !query schema +struct +-- !query output +0123456789 +abc + + +-- !query +DROP VIEW tt1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tt1 (f1 int, f2 int, f3 string) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tx1 (x1 int, x2 int, x3 string) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE temp_view_test.tt1 (y1 int, f2 int, f3 string) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW aliased_view_1 AS + select * from tt1 + where exists (select 1 from tx1 where tt1.f1 = tx1.x1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW aliased_view_2 AS + select * from tt1 a1 + where exists (select 1 from tx1 where a1.f1 = tx1.x1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW aliased_view_3 AS + select * from tt1 + where exists (select 1 from tx1 a2 where tt1.f1 = a2.x1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW aliased_view_4 AS + select * from temp_view_test.tt1 + where exists (select 1 from tt1 where temp_view_test.tt1.y1 = tt1.f1) +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE aliased_view_1 +-- !query schema +struct +-- !query output +f1 int +f2 int +f3 string + + +-- !query +DESC TABLE aliased_view_2 +-- !query schema +struct +-- !query output +f1 int +f2 int +f3 string + + +-- !query +DESC TABLE aliased_view_3 +-- !query schema +struct +-- !query output +f1 int +f2 int +f3 string + + +-- !query +DESC TABLE aliased_view_4 +-- !query schema +struct +-- !query output +y1 int +f2 int +f3 string + + +-- !query +ALTER TABLE tx1 RENAME TO a1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE aliased_view_1 +-- !query schema +struct +-- !query output +f1 int +f2 int +f3 string + + +-- !query +DESC TABLE aliased_view_2 +-- !query schema +struct +-- !query output +f1 int +f2 int +f3 string + + +-- !query +DESC TABLE aliased_view_3 +-- !query schema +struct +-- !query output +f1 int +f2 int +f3 string + + +-- !query +DESC TABLE aliased_view_4 +-- !query schema +struct +-- !query output +y1 int +f2 int +f3 string + + +-- !query +ALTER TABLE tt1 RENAME TO a2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE aliased_view_1 +-- !query schema +struct +-- !query output +f1 int +f2 int +f3 string + + +-- !query +DESC TABLE aliased_view_2 +-- !query schema +struct +-- !query output +f1 int +f2 int +f3 string + + +-- !query +DESC TABLE aliased_view_3 +-- !query schema +struct +-- !query output +f1 int +f2 int +f3 string + + +-- !query +DESC TABLE aliased_view_4 +-- !query schema +struct +-- !query output +y1 int +f2 int +f3 string + + +-- !query +ALTER TABLE a1 RENAME TO tt1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE aliased_view_1 +-- !query schema +struct +-- !query output +f1 int +f2 int +f3 string + + +-- !query +DESC TABLE aliased_view_2 +-- !query schema +struct +-- !query output +f1 int +f2 int +f3 string + + +-- !query +DESC TABLE aliased_view_3 +-- !query schema +struct +-- !query output +f1 int +f2 int +f3 string + + +-- !query +DESC TABLE aliased_view_4 +-- !query schema +struct +-- !query output +y1 int +f2 int +f3 string + + +-- !query +ALTER TABLE a2 RENAME TO tx1 +-- !query schema +struct<> +-- !query output + + + +-- !query +create view view_of_joins as +select * from + (select * from (tbl1 cross join tbl2) same) ss, + (tbl3 cross join tbl4) same +-- !query schema +struct<> +-- !query output + + + +-- !query +create table tt2 (a int, b int, c int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +create table tt3 (ax bigint, b short, c decimal) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +create table tt4 (ay int, b int, q int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +create view v1 as select * from tt2 natural join tt3 +-- !query schema +struct<> +-- !query output + + + +-- !query +create view v1a as select * from (tt2 natural join tt3) j +-- !query schema +struct<> +-- !query output + + + +-- !query +create view v2 as select * from tt2 join tt3 using (b,c) join tt4 using (b) +-- !query schema +struct<> +-- !query output + + + +-- !query +create view v2a as select * from (tt2 join tt3 using (b,c) join tt4 using (b)) j +-- !query schema +struct<> +-- !query output + + + +-- !query +create view v3 as select * from tt2 join tt3 using (b,c) full join tt4 using (b) +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE v1 +-- !query schema +struct +-- !query output +b int +c int +a int +ax bigint + + +-- !query +DESC TABLE v1a +-- !query schema +struct +-- !query output +b int +c int +a int +ax bigint + + +-- !query +DESC TABLE v2 +-- !query schema +struct +-- !query output +b int +c int +a int +ax bigint +ay int +q int + + +-- !query +DESC TABLE v2a +-- !query schema +struct +-- !query output +b int +c int +a int +ax bigint +ay int +q int + + +-- !query +DESC TABLE v3 +-- !query schema +struct +-- !query output +b int +c int +a int +ax bigint +ay int +q int + + +-- !query +alter table tt2 add column d int +-- !query schema +struct<> +-- !query output + + + +-- !query +alter table tt2 add column e int +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE v1 +-- !query schema +struct +-- !query output +b int +c int +a int +ax bigint + + +-- !query +DESC TABLE v1a +-- !query schema +struct +-- !query output +b int +c int +a int +ax bigint + + +-- !query +DESC TABLE v2 +-- !query schema +struct +-- !query output +b int +c int +a int +ax bigint +ay int +q int + + +-- !query +DESC TABLE v2a +-- !query schema +struct +-- !query output +b int +c int +a int +ax bigint +ay int +q int + + +-- !query +DESC TABLE v3 +-- !query schema +struct +-- !query output +b int +c int +a int +ax bigint +ay int +q int + + +-- !query +drop table tt3 +-- !query schema +struct<> +-- !query output + + + +-- !query +create table tt3 (ax bigint, b short, d decimal) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +alter table tt3 add column c int +-- !query schema +struct<> +-- !query output + + + +-- !query +alter table tt3 add column e int +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE v1 +-- !query schema +struct +-- !query output +b int +c int +a int +ax bigint + + +-- !query +DESC TABLE v1a +-- !query schema +struct +-- !query output +b int +c int +a int +ax bigint + + +-- !query +DESC TABLE v2 +-- !query schema +struct +-- !query output +b int +c int +a int +ax bigint +ay int +q int + + +-- !query +DESC TABLE v2a +-- !query schema +struct +-- !query output +b int +c int +a int +ax bigint +ay int +q int + + +-- !query +DESC TABLE v3 +-- !query schema +struct +-- !query output +b int +c int +a int +ax bigint +ay int +q int + + +-- !query +create table tt5 (a int, b int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +create table tt6 (c int, d int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +create view vv1 as select * from (tt5 cross join tt6) j(aa,bb,cc,dd) +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE vv1 +-- !query schema +struct +-- !query output +aa int +bb int +cc int +dd int + + +-- !query +alter table tt5 add column c int +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE vv1 +-- !query schema +struct +-- !query output +aa int +bb int +cc int +dd int + + +-- !query +alter table tt5 add column cc int +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE vv1 +-- !query schema +struct +-- !query output +aa int +bb int +cc int +dd int + + +-- !query +create table tt7 (x int, /* xx int, */ y int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +create table tt8 (x int, z int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +create view vv2 as +select * from (values(1,2,3,4,5)) v(a,b,c,d,e) +union all +select * from tt7 full join tt8 using (x), tt8 tt8x +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE vv2 +-- !query schema +struct +-- !query output +a int +b int +c int +d int +e int + + +-- !query +create view vv3 as +select * from (values(1,2,3,4,5,6)) v(a,b,c,x,e,f) +union all +select * from + tt7 full join tt8 using (x), + tt7 tt7x full join tt8 tt8x using (x) +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE vv3 +-- !query schema +struct +-- !query output +a int +b int +c int +x int +e int +f int + + +-- !query +create view vv4 as +select * from (values(1,2,3,4,5,6,7)) v(a,b,c,x,e,f,g) +union all +select * from + tt7 full join tt8 using (x), + tt7 tt7x full join tt8 tt8x using (x) full join tt8 tt8y using (x) +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE vv4 +-- !query schema +struct +-- !query output +a int +b int +c int +x int +e int +f int +g int + + +-- !query +alter table tt7 add column zz int +-- !query schema +struct<> +-- !query output + + + +-- !query +alter table tt7 add column z int +-- !query schema +struct<> +-- !query output + + + +-- !query +alter table tt8 add column z2 int +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE vv2 +-- !query schema +struct +-- !query output +a int +b int +c int +d int +e int + + +-- !query +DESC TABLE vv3 +-- !query schema +struct +-- !query output +a int +b int +c int +x int +e int +f int + + +-- !query +DESC TABLE vv4 +-- !query schema +struct +-- !query output +a int +b int +c int +x int +e int +f int +g int + + +-- !query +create table tt7a (x date, /* xx int, */ y int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +create table tt8a (x timestamp, z int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +create view vv2a as +select * from (values(now(),2,3,now(),5)) v(a,b,c,d,e) +union all +select * from tt7a left join tt8a using (x), tt8a tt8ax +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE vv4 +-- !query schema +struct +-- !query output +a int +b int +c int +x int +e int +f int +g int + + +-- !query +DESC TABLE vv2a +-- !query schema +struct +-- !query output +a timestamp +b int +c int +d timestamp +e int + + +-- !query +create table tt9 (x int, xx int, y int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +create table tt10 (x int, z int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +create view vv5 as select x,y,z from tt9 join tt10 using(x) +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE vv5 +-- !query schema +struct +-- !query output +x int +y int +z int + + +-- !query +DESC TABLE vv5 +-- !query schema +struct +-- !query output +x int +y int +z int + + +-- !query +create table tt11 (x int, y int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +create table tt12 (x int, z int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +create table tt13 (z int, q int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +create view vv6 as select x,y,z,q from + (tt11 join tt12 using(x)) join tt13 using(z) +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE vv6 +-- !query schema +struct +-- !query output +x int +y int +z int +q int + + +-- !query +alter table tt11 add column z int +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE vv6 +-- !query schema +struct +-- !query output +x int +y int +z int +q int + + +-- !query +CREATE TABLE int8_tbl (q1 int, q2 int) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +create view tt18v as + select * from int8_tbl xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxy + union all + select * from int8_tbl xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxz +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE tt18v +-- !query schema +struct +-- !query output +q1 int +q2 int + + +-- !query +create view tt21v as +select * from tt5 natural inner join tt6 +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE tt21v +-- !query schema +struct +-- !query output +c int +a int +b int +cc int +d int + + +-- !query +create view tt22v as +select * from tt5 natural left join tt6 +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE tt22v +-- !query schema +struct +-- !query output +c int +a int +b int +cc int +d int + + +-- !query +create view tt23v (col_a, col_b) as +select q1 as other_name1, q2 as other_name2 from int8_tbl +union +select 42, 43 +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC TABLE tt23v +-- !query schema +struct +-- !query output +col_a int +col_b int + + +-- !query +DROP SCHEMA temp_view_test CASCADE +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP SCHEMA testviewschm2 CASCADE +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW temp_table +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW tt +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/date.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/date.sql.out new file mode 100755 index 000000000000..d9f4301dd0e8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/date.sql.out @@ -0,0 +1,736 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE DATE_TBL (f1 date) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO DATE_TBL VALUES (date('1957-04-09')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO DATE_TBL VALUES (date('1957-06-13')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO DATE_TBL VALUES (date('1996-02-28')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO DATE_TBL VALUES (date('1996-02-29')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO DATE_TBL VALUES (date('1996-03-01')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO DATE_TBL VALUES (date('1996-03-02')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO DATE_TBL VALUES (date('1997-02-28')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO DATE_TBL VALUES (date('1997-03-01')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO DATE_TBL VALUES (date('1997-03-02')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO DATE_TBL VALUES (date('2000-04-01')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO DATE_TBL VALUES (date('2000-04-02')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO DATE_TBL VALUES (date('2000-04-03')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO DATE_TBL VALUES (date('2038-04-08')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO DATE_TBL VALUES (date('2039-04-09')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO DATE_TBL VALUES (date('2040-04-10')) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT f1 AS `Fifteen` FROM DATE_TBL +-- !query schema +struct +-- !query output +1957-04-09 +1957-06-13 +1996-02-28 +1996-02-29 +1996-03-01 +1996-03-02 +1997-02-28 +1997-03-01 +1997-03-02 +2000-04-01 +2000-04-02 +2000-04-03 +2038-04-08 +2039-04-09 +2040-04-10 + + +-- !query +SELECT f1 AS `Nine` FROM DATE_TBL WHERE f1 < '2000-01-01' +-- !query schema +struct +-- !query output +1957-04-09 +1957-06-13 +1996-02-28 +1996-02-29 +1996-03-01 +1996-03-02 +1997-02-28 +1997-03-01 +1997-03-02 + + +-- !query +SELECT f1 AS `Three` FROM DATE_TBL + WHERE f1 BETWEEN '2000-01-01' AND '2001-01-01' +-- !query schema +struct +-- !query output +2000-04-01 +2000-04-02 +2000-04-03 + + +-- !query +SELECT date '1999-01-08' +-- !query schema +struct +-- !query output +1999-01-08 + + +-- !query +SELECT date '1999-01-18' +-- !query schema +struct +-- !query output +1999-01-18 + + +-- !query +SELECT date '1999 Jan 08' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 Jan 08'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date '1999 Jan 08'" + } ] +} + + +-- !query +SELECT date '1999 08 Jan' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 08 Jan'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date '1999 08 Jan'" + } ] +} + + +-- !query +SELECT date '1999-01-08' +-- !query schema +struct +-- !query output +1999-01-08 + + +-- !query +SELECT date '1999-08-01' +-- !query schema +struct +-- !query output +1999-08-01 + + +-- !query +SELECT date '1999 01 08' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 01 08'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "date '1999 01 08'" + } ] +} + + +-- !query +SELECT date '1999 08 01' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 08 01'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "date '1999 08 01'" + } ] +} + + +-- !query +SELECT date '1999-01-08' +-- !query schema +struct +-- !query output +1999-01-08 + + +-- !query +SELECT date '1999 Jan 08' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 Jan 08'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date '1999 Jan 08'" + } ] +} + + +-- !query +SELECT date '1999 08 Jan' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 08 Jan'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date '1999 08 Jan'" + } ] +} + + +-- !query +SELECT date '1999-01-08' +-- !query schema +struct +-- !query output +1999-01-08 + + +-- !query +SELECT date '1999-08-01' +-- !query schema +struct +-- !query output +1999-08-01 + + +-- !query +SELECT date '1999 01 08' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 01 08'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "date '1999 01 08'" + } ] +} + + +-- !query +SELECT date '1999 08 01' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 08 01'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "date '1999 08 01'" + } ] +} + + +-- !query +SELECT date '1999-01-08' +-- !query schema +struct +-- !query output +1999-01-08 + + +-- !query +SELECT date '1999-01-18' +-- !query schema +struct +-- !query output +1999-01-18 + + +-- !query +SELECT date '1999 Jan 08' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 Jan 08'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date '1999 Jan 08'" + } ] +} + + +-- !query +SELECT date '1999 08 Jan' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 08 Jan'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "date '1999 08 Jan'" + } ] +} + + +-- !query +SELECT date '1999-01-08' +-- !query schema +struct +-- !query output +1999-01-08 + + +-- !query +SELECT date '1999-08-01' +-- !query schema +struct +-- !query output +1999-08-01 + + +-- !query +SELECT date '1999 01 08' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 01 08'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "date '1999 01 08'" + } ] +} + + +-- !query +SELECT date '1999 08 01' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'1999 08 01'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "date '1999 08 01'" + } ] +} + + +-- !query +SELECT date '4714-11-24 BC' +-- !query schema +struct +-- !query output +4714-11-24 + + +-- !query +SELECT date '4714-11-23 BC' +-- !query schema +struct +-- !query output +4714-11-23 + + +-- !query +SELECT date '5874897-12-31' +-- !query schema +struct +-- !query output ++5874897-12-31 + + +-- !query +SELECT date '5874898-01-01' +-- !query schema +struct +-- !query output ++5874898-01-01 + + +-- !query +SELECT f1 - date '2000-01-01' AS `Days From 2K` FROM DATE_TBL +-- !query schema +struct +-- !query output +-2 years -10 months +-2 years -10 months -1 days +-2 years -9 months -30 days +-3 years -10 months +-3 years -10 months -1 days +-3 years -10 months -2 days +-3 years -9 months -30 days +-42 years -6 months -18 days +-42 years -8 months -22 days +3 months +3 months 1 days +3 months 2 days +38 years 3 months 7 days +39 years 3 months 8 days +40 years 3 months 9 days + + +-- !query +SELECT f1 - date 'epoch' AS `Days From Epoch` FROM DATE_TBL +-- !query schema +struct +-- !query output +-12 years -6 months -18 days +-12 years -8 months -22 days +26 years 1 months 27 days +26 years 1 months 28 days +26 years 2 months +26 years 2 months 1 days +27 years 1 months 27 days +27 years 2 months +27 years 2 months 1 days +30 years 3 months +30 years 3 months 1 days +30 years 3 months 2 days +68 years 3 months 7 days +69 years 3 months 8 days +70 years 3 months 9 days + + +-- !query +SELECT date 'yesterday' - date 'today' AS `One day` +-- !query schema +struct +-- !query output +-1 days + + +-- !query +SELECT date 'today' - date 'tomorrow' AS `One day` +-- !query schema +struct +-- !query output +-1 days + + +-- !query +SELECT date 'yesterday' - date 'tomorrow' AS `Two days` +-- !query schema +struct +-- !query output +-2 days + + +-- !query +SELECT date 'tomorrow' - date 'today' AS `One day` +-- !query schema +struct +-- !query output +1 days + + +-- !query +SELECT date 'today' - date 'yesterday' AS `One day` +-- !query schema +struct +-- !query output +1 days + + +-- !query +SELECT date 'tomorrow' - date 'yesterday' AS `Two days` +-- !query schema +struct +-- !query output +2 days + + +-- !query +select make_date(2013, 7, 15) +-- !query schema +struct +-- !query output +2013-07-15 + + +-- !query +select make_date(-44, 3, 15) +-- !query schema +struct +-- !query output +-0044-03-15 + + +-- !query +select make_date(2013, 2, 30) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid date 'FEBRUARY 30'" + } +} + + +-- !query +select make_date(2013, 13, 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for MonthOfYear (valid values 1 - 12): 13" + } +} + + +-- !query +select make_date(2013, 11, -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for DayOfMonth (valid values 1 - 28/31): -1" + } +} + + +-- !query +DROP TABLE DATE_TBL +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/float4.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/float4.sql.out new file mode 100644 index 000000000000..6b4b343d9cca --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/float4.sql.out @@ -0,0 +1,477 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE FLOAT4_TBL (f1 float) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT4_TBL VALUES (float(' 0.0')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT4_TBL VALUES (float('1004.30 ')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT4_TBL VALUES (float(' -34.84 ')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e+20')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e-20')) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT float('NaN') +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT float('nan') +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT float(' NAN ') +-- !query schema +struct< NAN :float> +-- !query output +NaN + + +-- !query +SELECT float('infinity') +-- !query schema +struct +-- !query output +Infinity + + +-- !query +SELECT float(' -INFINiTY ') +-- !query schema +struct< -INFINiTY :float> +-- !query output +-Infinity + + +-- !query +SELECT float('N A N') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'N A N'", + "sourceType" : "\"STRING\"", + "targetType" : "\"FLOAT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "float('N A N')" + } ] +} + + +-- !query +SELECT float('NaN x') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'NaN x'", + "sourceType" : "\"STRING\"", + "targetType" : "\"FLOAT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "float('NaN x')" + } ] +} + + +-- !query +SELECT float(' INFINITY x') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "' INFINITY x'", + "sourceType" : "\"STRING\"", + "targetType" : "\"FLOAT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "float(' INFINITY x')" + } ] +} + + +-- !query +SELECT float('Infinity') + 100.0 +-- !query schema +struct<(Infinity + 100.0):double> +-- !query output +Infinity + + +-- !query +SELECT float('Infinity') / float('Infinity') +-- !query schema +struct<(Infinity / Infinity):double> +-- !query output +NaN + + +-- !query +SELECT float('nan') / float('nan') +-- !query schema +struct<(nan / nan):double> +-- !query output +NaN + + +-- !query +SELECT float(decimal('nan')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'nan'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DECIMAL(10,0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 14, + "stopIndex" : 27, + "fragment" : "decimal('nan')" + } ] +} + + +-- !query +SELECT '' AS five, * FROM FLOAT4_TBL +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345679E-20 + 1.2345679E20 + 1004.3 + + +-- !query +SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE f.f1 <> '1004.3' +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345679E-20 + 1.2345679E20 + 1004.3 + + +-- !query +SELECT '' AS one, f.* FROM FLOAT4_TBL f WHERE f.f1 = '1004.3' +-- !query schema +struct +-- !query output + + + +-- !query +SELECT '' AS three, f.* FROM FLOAT4_TBL f WHERE '1004.3' > f.f1 +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345679E-20 + 1004.3 + + +-- !query +SELECT '' AS three, f.* FROM FLOAT4_TBL f WHERE f.f1 < '1004.3' +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345679E-20 + 1004.3 + + +-- !query +SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE '1004.3' >= f.f1 +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345679E-20 + 1004.3 + + +-- !query +SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE f.f1 <= '1004.3' +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345679E-20 + 1004.3 + + +-- !query +SELECT '' AS three, f.f1, f.f1 * '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0' +-- !query schema +struct +-- !query output + 1.2345679E-20 -1.2345678720289608E-19 + 1.2345679E20 -1.2345678955701443E21 + 1004.3 -10042.999877929688 + + +-- !query +SELECT '' AS three, f.f1, f.f1 + '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0' +-- !query schema +struct +-- !query output + 1.2345679E-20 -10.0 + 1.2345679E20 1.2345678955701443E20 + 1004.3 994.2999877929688 + + +-- !query +SELECT '' AS three, f.f1, f.f1 / '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0' +-- !query schema +struct +-- !query output + 1.2345679E-20 -1.2345678720289608E-21 + 1.2345679E20 -1.2345678955701443E19 + 1004.3 -100.42999877929688 + + +-- !query +SELECT '' AS three, f.f1, f.f1 - '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0' +-- !query schema +struct +-- !query output + 1.2345679E-20 10.0 + 1.2345679E20 1.2345678955701443E20 + 1004.3 1014.2999877929688 + + +-- !query +SELECT '' AS five, * FROM FLOAT4_TBL +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345679E-20 + 1.2345679E20 + 1004.3 + + +-- !query +SELECT smallint(float('32767.4')) +-- !query schema +struct<32767.4:smallint> +-- !query output +32767 + + +-- !query +SELECT smallint(float('32767.6')) +-- !query schema +struct<32767.6:smallint> +-- !query output +32767 + + +-- !query +SELECT smallint(float('-32768.4')) +-- !query schema +struct<-32768.4:smallint> +-- !query output +-32768 + + +-- !query +SELECT smallint(float('-32768.6')) +-- !query schema +struct<-32768.6:smallint> +-- !query output +-32768 + + +-- !query +SELECT int(float('2147483520')) +-- !query schema +struct<2147483520:int> +-- !query output +2147483520 + + +-- !query +SELECT int(float('2147483647')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"FLOAT\"", + "targetType" : "\"INT\"", + "value" : "2.14748365E9" + } +} + + +-- !query +SELECT int(float('-2147483648.5')) +-- !query schema +struct<-2147483648.5:int> +-- !query output +-2147483648 + + +-- !query +SELECT int(float('-2147483900')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"FLOAT\"", + "targetType" : "\"INT\"", + "value" : "-2.1474839E9" + } +} + + +-- !query +SELECT bigint(float('9223369837831520256')) +-- !query schema +struct<9223369837831520256:bigint> +-- !query output +9223369837831520256 + + +-- !query +SELECT bigint(float('9223372036854775807')) +-- !query schema +struct<9223372036854775807:bigint> +-- !query output +9223372036854775807 + + +-- !query +SELECT bigint(float('-9223372036854775808.5')) +-- !query schema +struct<-9223372036854775808.5:bigint> +-- !query output +-9223372036854775808 + + +-- !query +SELECT bigint(float('-9223380000000000000')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"FLOAT\"", + "targetType" : "\"BIGINT\"", + "value" : "-9.22338E18" + } +} + + +-- !query +DROP TABLE FLOAT4_TBL +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/float4.sql.out.java21 b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/float4.sql.out.java21 new file mode 100644 index 000000000000..6126411071bc --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/float4.sql.out.java21 @@ -0,0 +1,477 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE FLOAT4_TBL (f1 float) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT4_TBL VALUES (float(' 0.0')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT4_TBL VALUES (float('1004.30 ')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT4_TBL VALUES (float(' -34.84 ')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e+20')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT4_TBL VALUES (float('1.2345678901234e-20')) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT float('NaN') +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT float('nan') +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT float(' NAN ') +-- !query schema +struct< NAN :float> +-- !query output +NaN + + +-- !query +SELECT float('infinity') +-- !query schema +struct +-- !query output +Infinity + + +-- !query +SELECT float(' -INFINiTY ') +-- !query schema +struct< -INFINiTY :float> +-- !query output +-Infinity + + +-- !query +SELECT float('N A N') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'N A N'", + "sourceType" : "\"STRING\"", + "targetType" : "\"FLOAT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "float('N A N')" + } ] +} + + +-- !query +SELECT float('NaN x') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'NaN x'", + "sourceType" : "\"STRING\"", + "targetType" : "\"FLOAT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "float('NaN x')" + } ] +} + + +-- !query +SELECT float(' INFINITY x') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "' INFINITY x'", + "sourceType" : "\"STRING\"", + "targetType" : "\"FLOAT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "float(' INFINITY x')" + } ] +} + + +-- !query +SELECT float('Infinity') + 100.0 +-- !query schema +struct<(Infinity + 100.0):double> +-- !query output +Infinity + + +-- !query +SELECT float('Infinity') / float('Infinity') +-- !query schema +struct<(Infinity / Infinity):double> +-- !query output +NaN + + +-- !query +SELECT float('nan') / float('nan') +-- !query schema +struct<(nan / nan):double> +-- !query output +NaN + + +-- !query +SELECT float(decimal('nan')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'nan'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DECIMAL(10,0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 14, + "stopIndex" : 27, + "fragment" : "decimal('nan')" + } ] +} + + +-- !query +SELECT '' AS five, * FROM FLOAT4_TBL +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345679E-20 + 1.2345679E20 + 1004.3 + + +-- !query +SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE f.f1 <> '1004.3' +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345679E-20 + 1.2345679E20 + 1004.3 + + +-- !query +SELECT '' AS one, f.* FROM FLOAT4_TBL f WHERE f.f1 = '1004.3' +-- !query schema +struct +-- !query output + + + +-- !query +SELECT '' AS three, f.* FROM FLOAT4_TBL f WHERE '1004.3' > f.f1 +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345679E-20 + 1004.3 + + +-- !query +SELECT '' AS three, f.* FROM FLOAT4_TBL f WHERE f.f1 < '1004.3' +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345679E-20 + 1004.3 + + +-- !query +SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE '1004.3' >= f.f1 +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345679E-20 + 1004.3 + + +-- !query +SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE f.f1 <= '1004.3' +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345679E-20 + 1004.3 + + +-- !query +SELECT '' AS three, f.f1, f.f1 * '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0' +-- !query schema +struct +-- !query output + 1.2345679E-20 -1.2345678720289608E-19 + 1.2345679E20 -1.2345678955701443E21 + 1004.3 -10042.999877929688 + + +-- !query +SELECT '' AS three, f.f1, f.f1 + '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0' +-- !query schema +struct +-- !query output + 1.2345679E-20 -10.0 + 1.2345679E20 1.2345678955701443E20 + 1004.3 994.2999877929688 + + +-- !query +SELECT '' AS three, f.f1, f.f1 / '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0' +-- !query schema +struct +-- !query output + 1.2345679E-20 -1.2345678720289608E-21 + 1.2345679E20 -1.2345678955701443E19 + 1004.3 -100.42999877929688 + + +-- !query +SELECT '' AS three, f.f1, f.f1 - '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0' +-- !query schema +struct +-- !query output + 1.2345679E-20 10.0 + 1.2345679E20 1.2345678955701443E20 + 1004.3 1014.2999877929688 + + +-- !query +SELECT '' AS five, * FROM FLOAT4_TBL +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345679E-20 + 1.2345679E20 + 1004.3 + + +-- !query +SELECT smallint(float('32767.4')) +-- !query schema +struct<32767.4:smallint> +-- !query output +32767 + + +-- !query +SELECT smallint(float('32767.6')) +-- !query schema +struct<32767.6:smallint> +-- !query output +32767 + + +-- !query +SELECT smallint(float('-32768.4')) +-- !query schema +struct<-32768.4:smallint> +-- !query output +-32768 + + +-- !query +SELECT smallint(float('-32768.6')) +-- !query schema +struct<-32768.6:smallint> +-- !query output +-32768 + + +-- !query +SELECT int(float('2147483520')) +-- !query schema +struct<2147483520:int> +-- !query output +2147483520 + + +-- !query +SELECT int(float('2147483647')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"FLOAT\"", + "targetType" : "\"INT\"", + "value" : "2.1474836E9" + } +} + + +-- !query +SELECT int(float('-2147483648.5')) +-- !query schema +struct<-2147483648.5:int> +-- !query output +-2147483648 + + +-- !query +SELECT int(float('-2147483900')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"FLOAT\"", + "targetType" : "\"INT\"", + "value" : "-2.147484E9" + } +} + + +-- !query +SELECT bigint(float('9223369837831520256')) +-- !query schema +struct<9223369837831520256:bigint> +-- !query output +9223369837831520256 + + +-- !query +SELECT bigint(float('9223372036854775807')) +-- !query schema +struct<9223372036854775807:bigint> +-- !query output +9223372036854775807 + + +-- !query +SELECT bigint(float('-9223372036854775808.5')) +-- !query schema +struct<-9223372036854775808.5:bigint> +-- !query output +-9223372036854775808 + + +-- !query +SELECT bigint(float('-9223380000000000000')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"FLOAT\"", + "targetType" : "\"BIGINT\"", + "value" : "-9.22338E18" + } +} + + +-- !query +DROP TABLE FLOAT4_TBL +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/float8.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/float8.sql.out new file mode 100644 index 000000000000..e1b880f34370 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/float8.sql.out @@ -0,0 +1,914 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE FLOAT8_TBL(f1 double) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double(' 0.0 ')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double('1004.30 ')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double(' -34.84')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double('1.2345678901234e+200')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double('1.2345678901234e-200')) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT double('10e400') +-- !query schema +struct<10e400:double> +-- !query output +Infinity + + +-- !query +SELECT double('-10e400') +-- !query schema +struct<-10e400:double> +-- !query output +-Infinity + + +-- !query +SELECT double('10e-400') +-- !query schema +struct<10e-400:double> +-- !query output +0.0 + + +-- !query +SELECT double('-10e-400') +-- !query schema +struct<-10e-400:double> +-- !query output +-0.0 + + +-- !query +SELECT double('NaN') +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT double('nan') +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT double(' NAN ') +-- !query schema +struct< NAN :double> +-- !query output +NaN + + +-- !query +SELECT double('infinity') +-- !query schema +struct +-- !query output +Infinity + + +-- !query +SELECT double(' -INFINiTY ') +-- !query schema +struct< -INFINiTY :double> +-- !query output +-Infinity + + +-- !query +SELECT double('N A N') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'N A N'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "double('N A N')" + } ] +} + + +-- !query +SELECT double('NaN x') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'NaN x'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "double('NaN x')" + } ] +} + + +-- !query +SELECT double(' INFINITY x') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "' INFINITY x'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "double(' INFINITY x')" + } ] +} + + +-- !query +SELECT double('Infinity') + 100.0 +-- !query schema +struct<(Infinity + 100.0):double> +-- !query output +Infinity + + +-- !query +SELECT double('Infinity') / double('Infinity') +-- !query schema +struct<(Infinity / Infinity):double> +-- !query output +NaN + + +-- !query +SELECT double('NaN') / double('NaN') +-- !query schema +struct<(NaN / NaN):double> +-- !query output +NaN + + +-- !query +SELECT double(decimal('nan')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'nan'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DECIMAL(10,0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 28, + "fragment" : "decimal('nan')" + } ] +} + + +-- !query +SELECT '' AS five, * FROM FLOAT8_TBL +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345678901234E-200 + 1.2345678901234E200 + 1004.3 + + +-- !query +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <> '1004.3' +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345678901234E-200 + 1.2345678901234E200 + + +-- !query +SELECT '' AS one, f.* FROM FLOAT8_TBL f WHERE f.f1 = '1004.3' +-- !query schema +struct +-- !query output + 1004.3 + + +-- !query +SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE '1004.3' > f.f1 +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345678901234E-200 + + +-- !query +SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE f.f1 < '1004.3' +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345678901234E-200 + + +-- !query +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE '1004.3' >= f.f1 +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345678901234E-200 + 1004.3 + + +-- !query +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <= '1004.3' +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345678901234E-200 + 1004.3 + + +-- !query +SELECT '' AS three, f.f1, f.f1 * '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query schema +struct +-- !query output + 1.2345678901234E-200 -1.2345678901234E-199 + 1.2345678901234E200 -1.2345678901234E201 + 1004.3 -10043.0 + + +-- !query +SELECT '' AS three, f.f1, f.f1 + '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query schema +struct +-- !query output + 1.2345678901234E-200 -10.0 + 1.2345678901234E200 1.2345678901234E200 + 1004.3 994.3 + + +-- !query +SELECT '' AS three, f.f1, f.f1 / '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query schema +struct +-- !query output + 1.2345678901234E-200 -1.2345678901234E-201 + 1.2345678901234E200 -1.2345678901234E199 + 1004.3 -100.42999999999999 + + +-- !query +SELECT '' AS three, f.f1, f.f1 - '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query schema +struct +-- !query output + 1.2345678901234E-200 10.0 + 1.2345678901234E200 1.2345678901234E200 + 1004.3 1014.3 + + +-- !query +SELECT '' AS five, f.f1, round(f.f1) AS round_f1 + FROM FLOAT8_TBL f +-- !query schema +struct +-- !query output + -34.84 -35.0 + 0.0 0.0 + 1.2345678901234E-200 0.0 + 1.2345678901234E200 1.2345678901234E200 + 1004.3 1004.0 + + +-- !query +select ceil(f1) as ceil_f1 from float8_tbl f +-- !query schema +struct +-- !query output +-34 +0 +1 +1005 +9223372036854775807 + + +-- !query +select ceiling(f1) as ceiling_f1 from float8_tbl f +-- !query schema +struct +-- !query output +-34 +0 +1 +1005 +9223372036854775807 + + +-- !query +select floor(f1) as floor_f1 from float8_tbl f +-- !query schema +struct +-- !query output +-35 +0 +0 +1004 +9223372036854775807 + + +-- !query +select sign(f1) as sign_f1 from float8_tbl f +-- !query schema +struct +-- !query output +-1.0 +0.0 +1.0 +1.0 +1.0 + + +-- !query +SELECT sqrt(double('64')) AS eight +-- !query schema +struct +-- !query output +8.0 + + +-- !query +SELECT power(double('144'), double('0.5')) +-- !query schema +struct +-- !query output +12.0 + + +-- !query +SELECT power(double('NaN'), double('0.5')) +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT power(double('144'), double('NaN')) +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT power(double('NaN'), double('NaN')) +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT power(double('-1'), double('NaN')) +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT power(double('1'), double('NaN')) +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT power(double('NaN'), double('0')) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT '' AS three, f.f1, exp(ln(f.f1)) AS exp_ln_f1 + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query schema +struct +-- !query output + 1.2345678901234E-200 1.2345678901233948E-200 + 1.2345678901234E200 1.234567890123379E200 + 1004.3 1004.3000000000004 + + +-- !query +SELECT '' AS five, * FROM FLOAT8_TBL +-- !query schema +struct +-- !query output + -34.84 + 0.0 + 1.2345678901234E-200 + 1.2345678901234E200 + 1004.3 + + +-- !query +CREATE TEMPORARY VIEW UPDATED_FLOAT8_TBL as +SELECT + CASE WHEN FLOAT8_TBL.f1 > '0.0' THEN FLOAT8_TBL.f1 * '-1' ELSE FLOAT8_TBL.f1 END AS f1 +FROM FLOAT8_TBL +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT '' AS bad, f.f1 * '1e200' from UPDATED_FLOAT8_TBL f +-- !query schema +struct +-- !query output + -1.0042999999999999E203 + -1.2345678901234 + -3.484E201 + -Infinity + 0.0 + + +-- !query +SELECT '' AS five, * FROM UPDATED_FLOAT8_TBL +-- !query schema +struct +-- !query output + -1.2345678901234E-200 + -1.2345678901234E200 + -1004.3 + -34.84 + 0.0 + + +-- !query +SELECT sinh(double('1')) +-- !query schema +struct +-- !query output +1.1752011936438014 + + +-- !query +SELECT cosh(double('1')) +-- !query schema +struct +-- !query output +1.543080634815244 + + +-- !query +SELECT tanh(double('1')) +-- !query schema +struct +-- !query output +0.7615941559557649 + + +-- !query +SELECT asinh(double('1')) +-- !query schema +struct +-- !query output +0.8813735870195429 + + +-- !query +SELECT acosh(double('2')) +-- !query schema +struct +-- !query output +1.3169578969248166 + + +-- !query +SELECT atanh(double('0.5')) +-- !query schema +struct +-- !query output +0.5493061443340548 + + +-- !query +SELECT sinh(double('Infinity')) +-- !query schema +struct +-- !query output +Infinity + + +-- !query +SELECT sinh(double('-Infinity')) +-- !query schema +struct +-- !query output +-Infinity + + +-- !query +SELECT sinh(double('NaN')) +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT cosh(double('Infinity')) +-- !query schema +struct +-- !query output +Infinity + + +-- !query +SELECT cosh(double('-Infinity')) +-- !query schema +struct +-- !query output +Infinity + + +-- !query +SELECT cosh(double('NaN')) +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT tanh(double('Infinity')) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT tanh(double('-Infinity')) +-- !query schema +struct +-- !query output +-1.0 + + +-- !query +SELECT tanh(double('NaN')) +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT asinh(double('Infinity')) +-- !query schema +struct +-- !query output +Infinity + + +-- !query +SELECT asinh(double('-Infinity')) +-- !query schema +struct +-- !query output +-Infinity + + +-- !query +SELECT asinh(double('NaN')) +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT acosh(double('Infinity')) +-- !query schema +struct +-- !query output +Infinity + + +-- !query +SELECT acosh(double('-Infinity')) +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT acosh(double('NaN')) +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT atanh(double('Infinity')) +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT atanh(double('-Infinity')) +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT atanh(double('NaN')) +-- !query schema +struct +-- !query output +NaN + + +-- !query +TRUNCATE TABLE FLOAT8_TBL +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double('0.0')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double('-34.84')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double('-1004.30')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double('-1.2345678901234e+200')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO FLOAT8_TBL VALUES (double('-1.2345678901234e-200')) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT '' AS five, * FROM FLOAT8_TBL +-- !query schema +struct +-- !query output + -1.2345678901234E-200 + -1.2345678901234E200 + -1004.3 + -34.84 + 0.0 + + +-- !query +SELECT smallint(double('32767.4')) +-- !query schema +struct<32767.4:smallint> +-- !query output +32767 + + +-- !query +SELECT smallint(double('32767.6')) +-- !query schema +struct<32767.6:smallint> +-- !query output +32767 + + +-- !query +SELECT smallint(double('-32768.4')) +-- !query schema +struct<-32768.4:smallint> +-- !query output +-32768 + + +-- !query +SELECT smallint(double('-32768.6')) +-- !query schema +struct<-32768.6:smallint> +-- !query output +-32768 + + +-- !query +SELECT int(double('2147483647.4')) +-- !query schema +struct<2147483647.4:int> +-- !query output +2147483647 + + +-- !query +SELECT int(double('2147483647.6')) +-- !query schema +struct<2147483647.6:int> +-- !query output +2147483647 + + +-- !query +SELECT int(double('-2147483648.4')) +-- !query schema +struct<-2147483648.4:int> +-- !query output +-2147483648 + + +-- !query +SELECT int(double('-2147483648.6')) +-- !query schema +struct<-2147483648.6:int> +-- !query output +-2147483648 + + +-- !query +SELECT bigint(double('9223372036854773760')) +-- !query schema +struct<9223372036854773760:bigint> +-- !query output +9223372036854773760 + + +-- !query +SELECT bigint(double('9223372036854775807')) +-- !query schema +struct<9223372036854775807:bigint> +-- !query output +9223372036854775807 + + +-- !query +SELECT bigint(double('-9223372036854775808.5')) +-- !query schema +struct<-9223372036854775808.5:bigint> +-- !query output +-9223372036854775808 + + +-- !query +SELECT bigint(double('-9223372036854780000')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"DOUBLE\"", + "targetType" : "\"BIGINT\"", + "value" : "-9.22337203685478E18D" + } +} + + +-- !query +DROP TABLE FLOAT8_TBL +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/groupingsets.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/groupingsets.sql.out new file mode 100644 index 000000000000..43fafb4eaae1 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/groupingsets.sql.out @@ -0,0 +1,731 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temp view gstest1(a,b,v) + as values (1,1,10),(1,1,11),(1,2,12),(1,2,13),(1,3,14), + (2,3,15), + (3,3,16),(3,4,17), + (4,1,18),(4,1,19) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table gstest2 (a integer, b integer, c integer, d integer, + e integer, f integer, g integer, h integer) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into gstest2 values + (1, 1, 1, 1, 1, 1, 1, 1), + (1, 1, 1, 1, 1, 1, 1, 2), + (1, 1, 1, 1, 1, 1, 2, 2), + (1, 1, 1, 1, 1, 2, 2, 2), + (1, 1, 1, 1, 2, 2, 2, 2), + (1, 1, 1, 2, 2, 2, 2, 2), + (1, 1, 2, 2, 2, 2, 2, 2), + (1, 2, 2, 2, 2, 2, 2, 2), + (2, 2, 2, 2, 2, 2, 2, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table gstest3 (a integer, b integer, c integer, d integer) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into gstest3 values + (1, 1, 1, 1), + (2, 2, 2, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table gstest4(id integer, v integer, + unhashable_col /* bit(4) */ byte, unsortable_col /* xid */ integer) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into gstest4 +values (1,1,tinyint('0'),1), (2,2,tinyint('1'),1), + (3,4,tinyint('2'),2), (4,8,tinyint('3'),2), + (5,16,tinyint('0'),2), (6,32,tinyint('1'),2), + (7,64,tinyint('2'),1), (8,128,tinyint('3'),1) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table gstest_empty (a integer, b integer, v integer) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by rollup (a,b) +-- !query schema +struct +-- !query output +1 1 0 0 21 2 11 +1 2 0 0 25 2 13 +1 3 0 0 14 1 14 +1 NULL 0 1 60 5 14 +2 3 0 0 15 1 15 +2 NULL 0 1 15 1 15 +3 3 0 0 16 1 16 +3 4 0 0 17 1 17 +3 NULL 0 1 33 2 17 +4 1 0 0 37 2 19 +4 NULL 0 1 37 2 19 +NULL NULL 1 1 145 10 19 + + +-- !query +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by rollup (a,b) order by a,b +-- !query schema +struct +-- !query output +NULL NULL 1 1 145 10 19 +1 NULL 0 1 60 5 14 +1 1 0 0 21 2 11 +1 2 0 0 25 2 13 +1 3 0 0 14 1 14 +2 NULL 0 1 15 1 15 +2 3 0 0 15 1 15 +3 NULL 0 1 33 2 17 +3 3 0 0 16 1 16 +3 4 0 0 17 1 17 +4 NULL 0 1 37 2 19 +4 1 0 0 37 2 19 + + +-- !query +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by rollup (a,b) order by b desc, a +-- !query schema +struct +-- !query output +3 4 0 0 17 1 17 +1 3 0 0 14 1 14 +2 3 0 0 15 1 15 +3 3 0 0 16 1 16 +1 2 0 0 25 2 13 +1 1 0 0 21 2 11 +4 1 0 0 37 2 19 +NULL NULL 1 1 145 10 19 +1 NULL 0 1 60 5 14 +2 NULL 0 1 15 1 15 +3 NULL 0 1 33 2 17 +4 NULL 0 1 37 2 19 + + +-- !query +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by rollup (a,b) order by coalesce(a,0)+coalesce(b,0), a +-- !query schema +struct +-- !query output +NULL NULL 1 1 145 10 19 +1 NULL 0 1 60 5 14 +1 1 0 0 21 2 11 +2 NULL 0 1 15 1 15 +1 2 0 0 25 2 13 +3 NULL 0 1 33 2 17 +1 3 0 0 14 1 14 +4 NULL 0 1 37 2 19 +2 3 0 0 15 1 15 +4 1 0 0 37 2 19 +3 3 0 0 16 1 16 +3 4 0 0 17 1 17 + + +-- !query +select a, b, sum(c), sum(sum(c)) over (order by a,b) as rsum + from gstest2 group by rollup (a,b) order by rsum, a, b +-- !query schema +struct +-- !query output +NULL NULL 12 12 +1 NULL 10 22 +1 1 8 30 +1 2 2 32 +2 NULL 2 34 +2 2 2 36 + + +-- !query +select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),a) +-- !query schema +struct +-- !query output + + + +-- !query +select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),()) +-- !query schema +struct +-- !query output + + + +-- !query +select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),(),(),()) +-- !query schema +struct +-- !query output + + + +-- !query +select sum(v), count(*) from gstest_empty group by grouping sets ((),(),()) +-- !query schema +struct +-- !query output + + + +-- !query +select t1.a, t2.b, sum(t1.v), count(*) from gstest_empty t1, gstest_empty t2 + group by grouping sets ((t1.a,t2.b),()) +-- !query schema +struct +-- !query output + + + +-- !query +select t1.a, t2.b, grouping(t1.a), grouping(t2.b), sum(t1.v), max(t2.a) + from gstest1 t1, gstest2 t2 + group by grouping sets ((t1.a, t2.b), ()) +-- !query schema +struct +-- !query output +1 1 0 0 420 1 +1 2 0 0 120 2 +2 1 0 0 105 1 +2 2 0 0 30 2 +3 1 0 0 231 1 +3 2 0 0 66 2 +4 1 0 0 259 1 +4 2 0 0 74 2 +NULL NULL 1 1 1305 2 + + +-- !query +select t1.a, t2.b, grouping(t1.a), grouping(t2.b), sum(t1.v), max(t2.a) + from gstest1 t1 join gstest2 t2 on (t1.a=t2.a) + group by grouping sets ((t1.a, t2.b), ()) +-- !query schema +struct +-- !query output +1 1 0 0 420 1 +1 2 0 0 60 1 +2 2 0 0 15 2 +NULL NULL 1 1 495 2 + + +-- !query +select a, b, grouping(a), grouping(b), sum(t1.v), max(t2.c) + from gstest1 t1 join gstest2 t2 using (a,b) + group by grouping sets ((a, b), ()) +-- !query schema +struct +-- !query output +1 1 0 0 147 2 +1 2 0 0 25 2 +NULL NULL 1 1 172 2 + + +-- !query +select four, x + from (select four, ten, 'foo' as x from tenk1) as t + group by grouping sets (four, x) + having x = 'foo' +-- !query schema +struct +-- !query output +NULL foo + + +-- !query +select four, x || 'x' + from (select four, ten, 'foo' as x from tenk1) as t + group by grouping sets (four, x) + order by four +-- !query schema +struct +-- !query output +NULL foox +0 NULL +1 NULL +2 NULL +3 NULL + + +-- !query +select (x+y)*1, sum(z) + from (select 1 as x, 2 as y, 3 as z) s + group by grouping sets (x+y, x) +-- !query schema +struct<((x + y) * 1):int,sum(z):bigint> +-- !query output +3 3 +NULL 3 + + +-- !query +CREATE TEMP VIEW int8_tbl AS SELECT * FROM VALUES + (123L, 456L), + (123L, 4567890123456789L), + (4567890123456789L, 123L), + (4567890123456789L, 4567890123456789L), + (4567890123456789L, -4567890123456789L) as int8_tbl(q1, q2) +-- !query schema +struct<> +-- !query output + + + +-- !query +select x, not x as not_x, q2 from + (select *, q1 = 1 as x from int8_tbl i1) as t + group by grouping sets(x, q2) + order by x, q2 +-- !query schema +struct +-- !query output +NULL NULL -4567890123456789 +NULL NULL 123 +NULL NULL 456 +NULL NULL 4567890123456789 +false true NULL + + +-- !query +DROP VIEW int8_tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +select ten, sum(distinct four) from onek a +group by grouping sets((ten,four),(ten)) +having exists (select 1 from onek b where sum(distinct a.four) = b.four) +-- !query schema +struct +-- !query output +0 0 +0 2 +0 2 +1 1 +1 3 +2 0 +2 2 +2 2 +3 1 +3 3 +4 0 +4 2 +4 2 +5 1 +5 3 +6 0 +6 2 +6 2 +7 1 +7 3 +8 0 +8 2 +8 2 +9 1 +9 3 + + +-- !query +select a,count(*) from gstest2 group by rollup(a) order by a +-- !query schema +struct +-- !query output +NULL 9 +1 8 +2 1 + + +-- !query +select a,count(*) from gstest2 group by rollup(a) having a is distinct from 1 order by a +-- !query schema +struct +-- !query output +NULL 9 +2 1 + + +-- !query +select ten, grouping(ten) from onek +group by grouping sets(ten) having grouping(ten) >= 0 +order by 2,1 +-- !query schema +struct +-- !query output +0 0 +1 0 +2 0 +3 0 +4 0 +5 0 +6 0 +7 0 +8 0 +9 0 + + +-- !query +select ten, grouping(ten) from onek +group by grouping sets(ten, four) having grouping(ten) > 0 +order by 2,1 +-- !query schema +struct +-- !query output +NULL 1 +NULL 1 +NULL 1 +NULL 1 + + +-- !query +select ten, grouping(ten) from onek +group by rollup(ten) having grouping(ten) > 0 +order by 2,1 +-- !query schema +struct +-- !query output +NULL 1 + + +-- !query +select ten, grouping(ten) from onek +group by cube(ten) having grouping(ten) > 0 +order by 2,1 +-- !query schema +struct +-- !query output +NULL 1 + + +-- !query +select ten, sum(distinct four) filter (where string(four) like '123') from onek a +group by rollup(ten) +-- !query schema +struct +-- !query output +0 NULL +1 NULL +2 NULL +3 NULL +4 NULL +5 NULL +6 NULL +7 NULL +8 NULL +9 NULL +NULL NULL + + +-- !query +select count(*) from gstest4 group by rollup(unhashable_col,unsortable_col) +-- !query schema +struct +-- !query output +1 +1 +1 +1 +1 +1 +1 +1 +2 +2 +2 +2 +8 + + +-- !query +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by grouping sets ((a),(b)) order by 3,4,1,2 /* 3,1,2 */ +-- !query schema +struct +-- !query output +1 NULL 0 1 60 5 14 +2 NULL 0 1 15 1 15 +3 NULL 0 1 33 2 17 +4 NULL 0 1 37 2 19 +NULL 1 1 0 58 4 19 +NULL 2 1 0 25 2 13 +NULL 3 1 0 45 3 16 +NULL 4 1 0 17 1 17 + + +-- !query +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by cube(a,b) order by 3,4,1,2 /* 3,1,2 */ +-- !query schema +struct +-- !query output +1 1 0 0 21 2 11 +1 2 0 0 25 2 13 +1 3 0 0 14 1 14 +2 3 0 0 15 1 15 +3 3 0 0 16 1 16 +3 4 0 0 17 1 17 +4 1 0 0 37 2 19 +1 NULL 0 1 60 5 14 +2 NULL 0 1 15 1 15 +3 NULL 0 1 33 2 17 +4 NULL 0 1 37 2 19 +NULL 1 1 0 58 4 19 +NULL 2 1 0 25 2 13 +NULL 3 1 0 45 3 16 +NULL 4 1 0 17 1 17 +NULL NULL 1 1 145 10 19 + + +-- !query +select unsortable_col, count(*) + from gstest4 group by grouping sets ((unsortable_col),(unsortable_col)) + order by string(unsortable_col) +-- !query schema +struct +-- !query output +1 4 +1 4 +2 4 +2 4 + + +-- !query +select unhashable_col, unsortable_col, + grouping(unhashable_col), grouping(unsortable_col), + count(*), sum(v) + from gstest4 group by grouping sets ((unhashable_col),(unsortable_col)) + order by 3, 4, 6 /* 3, 5 */ +-- !query schema +struct +-- !query output +0 NULL 0 1 2 17 +1 NULL 0 1 2 34 +2 NULL 0 1 2 68 +3 NULL 0 1 2 136 +NULL 2 1 0 4 60 +NULL 1 1 0 4 195 + + +-- !query +select unhashable_col, unsortable_col, + grouping(unhashable_col), grouping(unsortable_col), + count(*), sum(v) + from gstest4 group by grouping sets ((v,unhashable_col),(v,unsortable_col)) + order by 3, 4, 6 /* 3,5 */ +-- !query schema +struct +-- !query output +0 NULL 0 1 1 1 +1 NULL 0 1 1 2 +2 NULL 0 1 1 4 +3 NULL 0 1 1 8 +0 NULL 0 1 1 16 +1 NULL 0 1 1 32 +2 NULL 0 1 1 64 +3 NULL 0 1 1 128 +NULL 1 1 0 1 1 +NULL 1 1 0 1 2 +NULL 2 1 0 1 4 +NULL 2 1 0 1 8 +NULL 2 1 0 1 16 +NULL 2 1 0 1 32 +NULL 1 1 0 1 64 +NULL 1 1 0 1 128 + + +-- !query +select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),a) +-- !query schema +struct +-- !query output + + + +-- !query +select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),()) +-- !query schema +struct +-- !query output + + + +-- !query +select a, b, sum(v), count(*) from gstest_empty group by grouping sets ((a,b),(),(),()) +-- !query schema +struct +-- !query output + + + +-- !query +select sum(v), count(*) from gstest_empty group by grouping sets ((),(),()) +-- !query schema +struct +-- !query output + + + +-- !query +select a, b, grouping(a), grouping(b), sum(v), count(*), max(v) + from gstest1 group by grouping sets ((a,b),(a+1,b+1),(a+2,b+2)) order by 3,4,7 /* 3,6 */ +-- !query schema +struct +-- !query output +1 1 0 0 21 2 11 +1 2 0 0 25 2 13 +1 3 0 0 14 1 14 +2 3 0 0 15 1 15 +3 3 0 0 16 1 16 +3 4 0 0 17 1 17 +4 1 0 0 37 2 19 +NULL NULL 1 1 21 2 11 +NULL NULL 1 1 21 2 11 +NULL NULL 1 1 25 2 13 +NULL NULL 1 1 25 2 13 +NULL NULL 1 1 14 1 14 +NULL NULL 1 1 14 1 14 +NULL NULL 1 1 15 1 15 +NULL NULL 1 1 15 1 15 +NULL NULL 1 1 16 1 16 +NULL NULL 1 1 16 1 16 +NULL NULL 1 1 17 1 17 +NULL NULL 1 1 17 1 17 +NULL NULL 1 1 37 2 19 +NULL NULL 1 1 37 2 19 + + +-- !query +select a, b, sum(c), sum(sum(c)) over (order by a,b) as rsum + from gstest2 group by cube (a,b) order by rsum, a, b +-- !query schema +struct +-- !query output +NULL NULL 12 12 +NULL 1 8 20 +NULL 2 4 24 +1 NULL 10 34 +1 1 8 42 +1 2 2 44 +2 NULL 2 46 +2 2 2 48 + + +-- !query +SELECT a, b, count(*), max(a), max(b) FROM gstest3 GROUP BY GROUPING SETS(a, b,()) ORDER BY a, b +-- !query schema +struct +-- !query output +NULL NULL 2 2 2 +NULL 1 1 1 1 +NULL 2 1 2 2 +1 NULL 1 1 1 +2 NULL 1 2 2 + + +-- !query +select v||'a', case grouping(v||'a') when 1 then 1 else 0 end, count(*) + from values (1, 'a'), (1, 'b') u(i,v) + group by rollup(i, v||'a') order by 1,3 +-- !query schema +struct +-- !query output +NULL 1 2 +NULL 1 2 +aa 0 1 +ba 0 1 + + +-- !query +select v||'a', case when grouping(v||'a') = 1 then 1 else 0 end, count(*) + from values (1, 'a'), (1, 'b') u(i,v) + group by rollup(i, v||'a') order by 1,3 +-- !query schema +struct +-- !query output +NULL 1 2 +NULL 1 2 +aa 0 1 +ba 0 1 + + +-- !query +DROP VIEW gstest1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE gstest2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE gstest3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE gstest4 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE gstest_empty +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/insert.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/insert.sql.out new file mode 100644 index 000000000000..91e2f9ed439e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/insert.sql.out @@ -0,0 +1,78 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create table inserttest (col1 int, col2 int /* NOT NULL */, col3 string /* default 'testing' */) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into inserttest values (NULL, 3, 'testing') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into inserttest values (NULL, 5, 'testing') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into inserttest values (NULL, 5, 'test') +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into inserttest values (NULL, 7, 'testing') +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from inserttest +-- !query schema +struct +-- !query output +NULL 3 testing +NULL 5 test +NULL 5 testing +NULL 7 testing + + +-- !query +insert into inserttest values(30, 50, repeat('x', 10000)) +-- !query schema +struct<> +-- !query output + + + +-- !query +select col1, col2, char_length(col3) from inserttest +-- !query schema +struct +-- !query output +30 50 10000 +NULL 3 7 +NULL 5 4 +NULL 5 7 +NULL 7 7 + + +-- !query +drop table inserttest +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/int2.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/int2.sql.out new file mode 100755 index 000000000000..ca55b6accc66 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/int2.sql.out @@ -0,0 +1,360 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE INT2_TBL(f1 smallint) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT2_TBL VALUES (smallint(trim('0 '))) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT2_TBL VALUES (smallint(trim(' 1234 '))) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT2_TBL VALUES (smallint(trim(' -1234'))) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT2_TBL VALUES (smallint('32767')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT2_TBL VALUES (smallint('-32767')) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT '' AS five, * FROM INT2_TBL +-- !query schema +struct +-- !query output + -1234 + -32767 + 0 + 1234 + 32767 + + +-- !query +SELECT '' AS four, i.* FROM INT2_TBL i WHERE i.f1 <> smallint('0') +-- !query schema +struct +-- !query output + -1234 + -32767 + 1234 + 32767 + + +-- !query +SELECT '' AS four, i.* FROM INT2_TBL i WHERE i.f1 <> int('0') +-- !query schema +struct +-- !query output + -1234 + -32767 + 1234 + 32767 + + +-- !query +SELECT '' AS one, i.* FROM INT2_TBL i WHERE i.f1 = smallint('0') +-- !query schema +struct +-- !query output + 0 + + +-- !query +SELECT '' AS one, i.* FROM INT2_TBL i WHERE i.f1 = int('0') +-- !query schema +struct +-- !query output + 0 + + +-- !query +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 < smallint('0') +-- !query schema +struct +-- !query output + -1234 + -32767 + + +-- !query +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 < int('0') +-- !query schema +struct +-- !query output + -1234 + -32767 + + +-- !query +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 <= smallint('0') +-- !query schema +struct +-- !query output + -1234 + -32767 + 0 + + +-- !query +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 <= int('0') +-- !query schema +struct +-- !query output + -1234 + -32767 + 0 + + +-- !query +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 > smallint('0') +-- !query schema +struct +-- !query output + 1234 + 32767 + + +-- !query +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 > int('0') +-- !query schema +struct +-- !query output + 1234 + 32767 + + +-- !query +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 >= smallint('0') +-- !query schema +struct +-- !query output + 0 + 1234 + 32767 + + +-- !query +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 >= int('0') +-- !query schema +struct +-- !query output + 0 + 1234 + 32767 + + +-- !query +SELECT '' AS one, i.* FROM INT2_TBL i WHERE (i.f1 % smallint('2')) = smallint('1') +-- !query schema +struct +-- !query output + 32767 + + +-- !query +SELECT '' AS three, i.* FROM INT2_TBL i WHERE (i.f1 % int('2')) = smallint('0') +-- !query schema +struct +-- !query output + -1234 + 0 + 1234 + + +-- !query +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT2_TBL i +WHERE abs(f1) < 16384 +-- !query schema +struct +-- !query output + -1234 -2468 + 0 0 + 1234 2468 + + +-- !query +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT2_TBL i +-- !query schema +struct +-- !query output + -1234 -2468 + -32767 -65534 + 0 0 + 1234 2468 + 32767 65534 + + +-- !query +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT2_TBL i +WHERE f1 < 32766 +-- !query schema +struct +-- !query output + -1234 -1232 + -32767 -32765 + 0 2 + 1234 1236 + + +-- !query +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT2_TBL i +-- !query schema +struct +-- !query output + -1234 -1232 + -32767 -32765 + 0 2 + 1234 1236 + 32767 32769 + + +-- !query +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT2_TBL i +WHERE f1 > -32767 +-- !query schema +struct +-- !query output + -1234 -1236 + 0 -2 + 1234 1232 + 32767 32765 + + +-- !query +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT2_TBL i +-- !query schema +struct +-- !query output + -1234 -1236 + -32767 -32769 + 0 -2 + 1234 1232 + 32767 32765 + + +-- !query +SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT2_TBL i +-- !query schema +struct +-- !query output + -1234 -617.0 + -32767 -16383.5 + 0 0.0 + 1234 617.0 + 32767 16383.5 + + +-- !query +SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT2_TBL i +-- !query schema +struct +-- !query output + -1234 -617.0 + -32767 -16383.5 + 0 0.0 + 1234 617.0 + 32767 16383.5 + + +-- !query +SELECT string(shiftleft(smallint(-1), 15)) +-- !query schema +struct +-- !query output +-32768 + + +-- !query +SELECT string(smallint(shiftleft(smallint(-1), 15))+1) +-- !query schema +struct<(shiftleft(-1, 15) + 1):string> +-- !query output +-32767 + + +-- !query +SELECT smallint(-32768) % smallint(-1) +-- !query schema +struct<(-32768 % -1):smallint> +-- !query output +0 + + +-- !query +SELECT x, smallint(x) AS int2_value +FROM (VALUES float(-2.5), + float(-1.5), + float(-0.5), + float(0.0), + float(0.5), + float(1.5), + float(2.5)) t(x) +-- !query schema +struct +-- !query output +-0.5 0 +-1.5 -1 +-2.5 -2 +0.0 0 +0.5 0 +1.5 1 +2.5 2 + + +-- !query +SELECT x, smallint(x) AS int2_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x) +-- !query schema +struct +-- !query output +-0.500000000000000000 0 +-1.500000000000000000 -1 +-2.500000000000000000 -2 +0.000000000000000000 0 +0.500000000000000000 0 +1.500000000000000000 1 +2.500000000000000000 2 + + +-- !query +DROP TABLE INT2_TBL +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/int4.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/int4.sql.out new file mode 100755 index 000000000000..16c18c86f291 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/int4.sql.out @@ -0,0 +1,599 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE INT4_TBL(f1 int) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT4_TBL VALUES (int(trim(' 0 '))) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT4_TBL VALUES (int(trim('123456 '))) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT4_TBL VALUES (int(trim(' -123456'))) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT4_TBL VALUES (int('2147483647')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT4_TBL VALUES (int('-2147483647')) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT '' AS five, * FROM INT4_TBL +-- !query schema +struct +-- !query output + -123456 + -2147483647 + 0 + 123456 + 2147483647 + + +-- !query +SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> smallint('0') +-- !query schema +struct +-- !query output + -123456 + -2147483647 + 123456 + 2147483647 + + +-- !query +SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> int('0') +-- !query schema +struct +-- !query output + -123456 + -2147483647 + 123456 + 2147483647 + + +-- !query +SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = smallint('0') +-- !query schema +struct +-- !query output + 0 + + +-- !query +SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = int('0') +-- !query schema +struct +-- !query output + 0 + + +-- !query +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < smallint('0') +-- !query schema +struct +-- !query output + -123456 + -2147483647 + + +-- !query +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < int('0') +-- !query schema +struct +-- !query output + -123456 + -2147483647 + + +-- !query +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= smallint('0') +-- !query schema +struct +-- !query output + -123456 + -2147483647 + 0 + + +-- !query +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= int('0') +-- !query schema +struct +-- !query output + -123456 + -2147483647 + 0 + + +-- !query +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > smallint('0') +-- !query schema +struct +-- !query output + 123456 + 2147483647 + + +-- !query +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > int('0') +-- !query schema +struct +-- !query output + 123456 + 2147483647 + + +-- !query +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= smallint('0') +-- !query schema +struct +-- !query output + 0 + 123456 + 2147483647 + + +-- !query +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= int('0') +-- !query schema +struct +-- !query output + 0 + 123456 + 2147483647 + + +-- !query +SELECT '' AS one, i.* FROM INT4_TBL i WHERE (i.f1 % smallint('2')) = smallint('1') +-- !query schema +struct +-- !query output + 2147483647 + + +-- !query +SELECT '' AS three, i.* FROM INT4_TBL i WHERE (i.f1 % int('2')) = smallint('0') +-- !query schema +struct +-- !query output + -123456 + 0 + 123456 + + +-- !query +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_multiply' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 45, + "fragment" : "i.f1 * smallint('2')" + } ] +} + + +-- !query +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i +WHERE abs(f1) < 1073741824 +-- !query schema +struct +-- !query output + -123456 -246912 + 0 0 + 123456 246912 + + +-- !query +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_multiply' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 40, + "fragment" : "i.f1 * int('2')" + } ] +} + + +-- !query +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i +WHERE abs(f1) < 1073741824 +-- !query schema +struct +-- !query output + -123456 -246912 + 0 0 + 123456 246912 + + +-- !query +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 45, + "fragment" : "i.f1 + smallint('2')" + } ] +} + + +-- !query +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i +WHERE f1 < 2147483646 +-- !query schema +struct +-- !query output + -123456 -123454 + -2147483647 -2147483645 + 0 2 + 123456 123458 + + +-- !query +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 40, + "fragment" : "i.f1 + int('2')" + } ] +} + + +-- !query +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i +WHERE f1 < 2147483646 +-- !query schema +struct +-- !query output + -123456 -123454 + -2147483647 -2147483645 + 0 2 + 123456 123458 + + +-- !query +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_subtract' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 45, + "fragment" : "i.f1 - smallint('2')" + } ] +} + + +-- !query +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i +WHERE f1 > -2147483647 +-- !query schema +struct +-- !query output + -123456 -123458 + 0 -2 + 123456 123454 + 2147483647 2147483645 + + +-- !query +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_subtract' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 40, + "fragment" : "i.f1 - int('2')" + } ] +} + + +-- !query +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i +WHERE f1 > -2147483647 +-- !query schema +struct +-- !query output + -123456 -123458 + 0 -2 + 123456 123454 + 2147483647 2147483645 + + +-- !query +SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT4_TBL i +-- !query schema +struct +-- !query output + -123456 -61728.0 + -2147483647 -1.0737418235E9 + 0 0.0 + 123456 61728.0 + 2147483647 1.0737418235E9 + + +-- !query +SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT4_TBL i +-- !query schema +struct +-- !query output + -123456 -61728.0 + -2147483647 -1.0737418235E9 + 0 0.0 + 123456 61728.0 + 2147483647 1.0737418235E9 + + +-- !query +SELECT -2+3 AS one +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT 4-2 AS two +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT 2- -1 AS three +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT 2 - -2 AS four +-- !query schema +struct +-- !query output +4 + + +-- !query +SELECT smallint('2') * smallint('2') = smallint('16') / smallint('4') AS true +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT int('2') * smallint('2') = smallint('16') / int('4') AS true +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT smallint('2') * int('2') = int('16') / smallint('4') AS true +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT int('1000') < int('999') AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 AS ten +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT 2 + 2 / 2 AS three +-- !query schema +struct +-- !query output +3.0 + + +-- !query +SELECT (2 + 2) / 2 AS two +-- !query schema +struct +-- !query output +2.0 + + +-- !query +SELECT string(shiftleft(int(-1), 31)) +-- !query schema +struct +-- !query output +-2147483648 + + +-- !query +SELECT string(int(shiftleft(int(-1), 31))+1) +-- !query schema +struct<(shiftleft(-1, 31) + 1):string> +-- !query output +-2147483647 + + +-- !query +SELECT int(-2147483648) % int(-1) +-- !query schema +struct<(-2147483648 % -1):int> +-- !query output +0 + + +-- !query +SELECT int(-2147483648) % smallint(-1) +-- !query schema +struct<(-2147483648 % -1):int> +-- !query output +0 + + +-- !query +SELECT x, int(x) AS int4_value +FROM (VALUES double(-2.5), + double(-1.5), + double(-0.5), + double(0.0), + double(0.5), + double(1.5), + double(2.5)) t(x) +-- !query schema +struct +-- !query output +-0.5 0 +-1.5 -1 +-2.5 -2 +0.0 0 +0.5 0 +1.5 1 +2.5 2 + + +-- !query +SELECT x, int(x) AS int4_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x) +-- !query schema +struct +-- !query output +-0.500000000000000000 0 +-1.500000000000000000 -1 +-2.500000000000000000 -2 +0.000000000000000000 0 +0.500000000000000000 0 +1.500000000000000000 1 +2.500000000000000000 2 + + +-- !query +DROP TABLE INT4_TBL +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/int8.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/int8.sql.out new file mode 100755 index 000000000000..f6e4bd8bd7e0 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/int8.sql.out @@ -0,0 +1,1076 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE INT8_TBL(q1 bigint, q2 bigint) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT8_TBL VALUES(bigint(trim(' 123 ')),bigint(trim(' 456'))) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT8_TBL VALUES(bigint(trim('123 ')),bigint('4567890123456789')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT8_TBL VALUES(bigint('4567890123456789'),bigint('123')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT8_TBL VALUES(+4567890123456789,bigint('4567890123456789')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT8_TBL VALUES(bigint('+4567890123456789'),bigint('-4567890123456789')) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM INT8_TBL +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 = 4567890123456789 +-- !query schema +struct +-- !query output +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 <> 4567890123456789 +-- !query schema +struct +-- !query output +123 456 +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 < 4567890123456789 +-- !query schema +struct +-- !query output +123 456 +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 > 4567890123456789 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 <= 4567890123456789 +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 >= 4567890123456789 +-- !query schema +struct +-- !query output +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 = 456 +-- !query schema +struct +-- !query output +123 456 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 <> 456 +-- !query schema +struct +-- !query output +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 < 456 +-- !query schema +struct +-- !query output +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 > 456 +-- !query schema +struct +-- !query output +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 <= 456 +-- !query schema +struct +-- !query output +123 456 +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 >= 456 +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE 123 = q1 +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE 123 <> q1 +-- !query schema +struct +-- !query output +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE 123 < q1 +-- !query schema +struct +-- !query output +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE 123 > q1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM INT8_TBL WHERE 123 <= q1 +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE 123 >= q1 +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 = smallint('456') +-- !query schema +struct +-- !query output +123 456 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 <> smallint('456') +-- !query schema +struct +-- !query output +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 < smallint('456') +-- !query schema +struct +-- !query output +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 > smallint('456') +-- !query schema +struct +-- !query output +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 <= smallint('456') +-- !query schema +struct +-- !query output +123 456 +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 >= smallint('456') +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE smallint('123') = q1 +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE smallint('123') <> q1 +-- !query schema +struct +-- !query output +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE smallint('123') < q1 +-- !query schema +struct +-- !query output +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE smallint('123') > q1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM INT8_TBL WHERE smallint('123') <= q1 +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE smallint('123') >= q1 +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 + + +-- !query +SELECT '' AS five, q1 AS plus, -q1 AS `minus` FROM INT8_TBL +-- !query schema +struct +-- !query output + 123 -123 + 123 -123 + 4567890123456789 -4567890123456789 + 4567890123456789 -4567890123456789 + 4567890123456789 -4567890123456789 + + +-- !query +SELECT '' AS five, q1, q2, q1 + q2 AS plus FROM INT8_TBL +-- !query schema +struct +-- !query output + 123 456 579 + 123 4567890123456789 4567890123456912 + 4567890123456789 -4567890123456789 0 + 4567890123456789 123 4567890123456912 + 4567890123456789 4567890123456789 9135780246913578 + + +-- !query +SELECT '' AS five, q1, q2, q1 - q2 AS `minus` FROM INT8_TBL +-- !query schema +struct +-- !query output + 123 456 -333 + 123 4567890123456789 -4567890123456666 + 4567890123456789 -4567890123456789 9135780246913578 + 4567890123456789 123 4567890123456666 + 4567890123456789 4567890123456789 0 + + +-- !query +SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_multiply' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 35, + "fragment" : "q1 * q2" + } ] +} + + +-- !query +SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL + WHERE q1 < 1000 or (q2 > 0 and q2 < 1000) +-- !query schema +struct +-- !query output + 123 456 56088 + 123 4567890123456789 561850485185185047 + 4567890123456789 123 561850485185185047 + + +-- !query +SELECT '' AS five, q1, q2, q1 / q2 AS divide, q1 % q2 AS mod FROM INT8_TBL +-- !query schema +struct +-- !query output + 123 456 0.26973684210526316 123 + 123 4567890123456789 2.6927092525360204E-14 123 + 4567890123456789 -4567890123456789 -1.0 0 + 4567890123456789 123 3.713731807688446E13 57 + 4567890123456789 4567890123456789 1.0 0 + + +-- !query +SELECT '' AS five, q1, double(q1) FROM INT8_TBL +-- !query schema +struct +-- !query output + 123 123.0 + 123 123.0 + 4567890123456789 4.567890123456789E15 + 4567890123456789 4.567890123456789E15 + 4567890123456789 4.567890123456789E15 + + +-- !query +SELECT '' AS five, q2, double(q2) FROM INT8_TBL +-- !query schema +struct +-- !query output + -4567890123456789 -4.567890123456789E15 + 123 123.0 + 456 456.0 + 4567890123456789 4.567890123456789E15 + 4567890123456789 4.567890123456789E15 + + +-- !query +SELECT 37 + q1 AS plus4 FROM INT8_TBL +-- !query schema +struct +-- !query output +160 +160 +4567890123456826 +4567890123456826 +4567890123456826 + + +-- !query +SELECT 37 - q1 AS minus4 FROM INT8_TBL +-- !query schema +struct +-- !query output +-4567890123456752 +-4567890123456752 +-4567890123456752 +-86 +-86 + + +-- !query +SELECT '' AS five, 2 * q1 AS `twice int4` FROM INT8_TBL +-- !query schema +struct +-- !query output + 246 + 246 + 9135780246913578 + 9135780246913578 + 9135780246913578 + + +-- !query +SELECT '' AS five, q1 * 2 AS `twice int4` FROM INT8_TBL +-- !query schema +struct +-- !query output + 246 + 246 + 9135780246913578 + 9135780246913578 + 9135780246913578 + + +-- !query +SELECT q1 + int(42) AS `8plus4`, q1 - int(42) AS `8minus4`, q1 * int(42) AS `8mul4`, q1 / int(42) AS `8div4` FROM INT8_TBL +-- !query schema +struct<8plus4:bigint,8minus4:bigint,8mul4:bigint,8div4:double> +-- !query output +165 81 5166 2.9285714285714284 +165 81 5166 2.9285714285714284 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 + + +-- !query +SELECT int(246) + q1 AS `4plus8`, int(246) - q1 AS `4minus8`, int(246) * q1 AS `4mul8`, int(246) / q1 AS `4div8` FROM INT8_TBL +-- !query schema +struct<4plus8:bigint,4minus8:bigint,4mul8:bigint,4div8:double> +-- !query output +369 123 30258 2.0 +369 123 30258 2.0 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 + + +-- !query +SELECT q1 + smallint(42) AS `8plus2`, q1 - smallint(42) AS `8minus2`, q1 * smallint(42) AS `8mul2`, q1 / smallint(42) AS `8div2` FROM INT8_TBL +-- !query schema +struct<8plus2:bigint,8minus2:bigint,8mul2:bigint,8div2:double> +-- !query output +165 81 5166 2.9285714285714284 +165 81 5166 2.9285714285714284 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 + + +-- !query +SELECT smallint(246) + q1 AS `2plus8`, smallint(246) - q1 AS `2minus8`, smallint(246) * q1 AS `2mul8`, smallint(246) / q1 AS `2div8` FROM INT8_TBL +-- !query schema +struct<2plus8:bigint,2minus8:bigint,2mul8:bigint,2div8:double> +-- !query output +369 123 30258 2.0 +369 123 30258 2.0 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 + + +-- !query +SELECT q2, abs(q2) FROM INT8_TBL +-- !query schema +struct +-- !query output +-4567890123456789 4567890123456789 +123 123 +456 456 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query +SELECT min(q1), min(q2) FROM INT8_TBL +-- !query schema +struct +-- !query output +123 -4567890123456789 + + +-- !query +SELECT max(q1), max(q2) FROM INT8_TBL +-- !query schema +struct +-- !query output +4567890123456789 4567890123456789 + + +-- !query +SELECT '' AS to_char_1, to_char(q1, '9G999G999G999G999G999'), to_char(q2, '9,999,999,999,999,999') +FROM INT8_TBL +-- !query schema +struct +-- !query output + 123 456 + 123 4,567,890,123,456,789 + 4,567,890,123,456,789 123 + 4,567,890,123,456,789 4,567,890,123,456,789 + 4,567,890,123,456,789 4,567,890,123,456,789 + + +-- !query +SELECT '' AS to_char_3, to_char( (q1 * -1), '9999999999999999PR'), to_char( (q2 * -1), '9999999999999999.999PR') +FROM INT8_TBL +-- !query schema +struct +-- !query output + <123> <456.000> + <123> <4567890123456789.000> + <4567890123456789> <123.000> + <4567890123456789> 4567890123456789.000 + <4567890123456789> <4567890123456789.000> + + +-- !query +SELECT '' AS to_char_4, to_char( (q1 * -1), '9999999999999999S'), to_char( (q2 * -1), 'S9999999999999999') +FROM INT8_TBL +-- !query schema +struct +-- !query output + 123- -456 + 123- -4567890123456789 + 4567890123456789- -123 + 4567890123456789- +4567890123456789 + 4567890123456789- -4567890123456789 + + +-- !query +SELECT '' AS to_char_5, to_char(q2, 'MI9999999999999999') FROM INT8_TBL +-- !query schema +struct +-- !query output + 123 + 456 + 4567890123456789 + 4567890123456789 + -4567890123456789 + + +-- !query +SELECT '' AS to_char_9, to_char(q2, '0999999999999999') FROM INT8_TBL +-- !query schema +struct +-- !query output + 0000000000000123 + 0000000000000456 + 4567890123456789 + 4567890123456789 + 4567890123456789 + + +-- !query +SELECT '' AS to_char_10, to_char(q2, 'S0999999999999999') FROM INT8_TBL +-- !query schema +struct +-- !query output + +0000000000000123 + +0000000000000456 + +4567890123456789 + +4567890123456789 + -4567890123456789 + + +-- !query +select bigint('9223372036854775800') / bigint('0') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "bigint('9223372036854775800') / bigint('0')" + } ] +} + + +-- !query +select bigint('-9223372036854775808') / smallint('0') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "bigint('-9223372036854775808') / smallint('0')" + } ] +} + + +-- !query +select smallint('100') / bigint('0') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "smallint('100') / bigint('0')" + } ] +} + + +-- !query +SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 = 456 +-- !query schema +struct +-- !query output +123 + + +-- !query +SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 <> 456 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"BIGINT\"", + "targetType" : "\"INT\"", + "value" : "4567890123456789L" + } +} + + +-- !query +SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 = 456 +-- !query schema +struct +-- !query output +123 + + +-- !query +SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 <> 456 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"BIGINT\"", + "targetType" : "\"SMALLINT\"", + "value" : "4567890123456789L" + } +} + + +-- !query +SELECT CAST(smallint('42') AS bigint), CAST(smallint('-37') AS bigint) +-- !query schema +struct +-- !query output +42 -37 + + +-- !query +SELECT CAST(q1 AS float), CAST(q2 AS double) FROM INT8_TBL +-- !query schema +struct +-- !query output +123.0 4.567890123456789E15 +123.0 456.0 +4.5678899E15 -4.567890123456789E15 +4.5678899E15 123.0 +4.5678899E15 4.567890123456789E15 + + +-- !query +SELECT CAST(float('36854775807.0') AS bigint) +-- !query schema +struct +-- !query output +36854775808 + + +-- !query +SELECT CAST(double('922337203685477580700.0') AS bigint) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"DOUBLE\"", + "targetType" : "\"BIGINT\"", + "value" : "9.223372036854776E20D" + } +} + + +-- !query +SELECT q1, q2, q1 & q2 AS `and`, q1 | q2 AS `or`, ~q1 AS `not` FROM INT8_TBL +-- !query schema +struct +-- !query output +123 456 72 507 -124 +123 4567890123456789 17 4567890123456895 -124 +4567890123456789 -4567890123456789 1 -1 -4567890123456790 +4567890123456789 123 17 4567890123456895 -4567890123456790 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 -4567890123456790 + + +-- !query +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799')) +-- !query schema +struct +-- !query output +4567890123456789 +4567890123456790 +4567890123456791 +4567890123456792 +4567890123456793 +4567890123456794 +4567890123456795 +4567890123456796 +4567890123456797 +4567890123456798 + + +-- !query +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799'), 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "FAILED_FUNCTION_CALL", + "sqlState" : "38000", + "messageParameters" : { + "funcName" : "`range`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 80, + "fragment" : "range(bigint('+4567890123456789'), bigint('+4567890123456799'), 0)" + } ] +} + + +-- !query +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799'), 2) +-- !query schema +struct +-- !query output +4567890123456789 +4567890123456791 +4567890123456793 +4567890123456795 +4567890123456797 + + +-- !query +SELECT string(shiftleft(bigint(-1), 63)) +-- !query schema +struct +-- !query output +-9223372036854775808 + + +-- !query +SELECT string(int(shiftleft(bigint(-1), 63))+1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"BIGINT\"", + "targetType" : "\"INT\"", + "value" : "-9223372036854775808L" + } +} + + +-- !query +SELECT bigint((-9223372036854775808)) * bigint((-1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_multiply' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "bigint((-9223372036854775808)) * bigint((-1))" + } ] +} + + +-- !query +SELECT bigint((-9223372036854775808)) / bigint((-1)) +-- !query schema +struct<(-9223372036854775808 / -1):double> +-- !query output +9.223372036854776E18 + + +-- !query +SELECT bigint((-9223372036854775808)) % bigint((-1)) +-- !query schema +struct<(-9223372036854775808 % -1):bigint> +-- !query output +0 + + +-- !query +SELECT bigint((-9223372036854775808)) * int((-1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_multiply' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "bigint((-9223372036854775808)) * int((-1))" + } ] +} + + +-- !query +SELECT bigint((-9223372036854775808)) / int((-1)) +-- !query schema +struct<(-9223372036854775808 / -1):double> +-- !query output +9.223372036854776E18 + + +-- !query +SELECT bigint((-9223372036854775808)) % int((-1)) +-- !query schema +struct<(-9223372036854775808 % -1):bigint> +-- !query output +0 + + +-- !query +SELECT bigint((-9223372036854775808)) * smallint((-1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_multiply' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "bigint((-9223372036854775808)) * smallint((-1))" + } ] +} + + +-- !query +SELECT bigint((-9223372036854775808)) / smallint((-1)) +-- !query schema +struct<(-9223372036854775808 / -1):double> +-- !query output +9.223372036854776E18 + + +-- !query +SELECT bigint((-9223372036854775808)) % smallint((-1)) +-- !query schema +struct<(-9223372036854775808 % -1):bigint> +-- !query output +0 + + +-- !query +SELECT x, bigint(x) AS int8_value +FROM (VALUES (double(-2.5)), + (double(-1.5)), + (double(-0.5)), + (double(0.0)), + (double(0.5)), + (double(1.5)), + (double(2.5))) t(x) +-- !query schema +struct +-- !query output +-0.5 0 +-1.5 -1 +-2.5 -2 +0.0 0 +0.5 0 +1.5 1 +2.5 2 + + +-- !query +SELECT x, bigint(x) AS int8_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x) +-- !query schema +struct +-- !query output +-0.500000000000000000 0 +-1.500000000000000000 -1 +-2.500000000000000000 -2 +0.000000000000000000 0 +0.500000000000000000 0 +1.500000000000000000 1 +2.500000000000000000 2 + + +-- !query +DROP TABLE INT8_TBL +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/int8.sql.out.java21 b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/int8.sql.out.java21 new file mode 100755 index 000000000000..ee3f8625da8a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/int8.sql.out.java21 @@ -0,0 +1,1076 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE INT8_TBL(q1 bigint, q2 bigint) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT8_TBL VALUES(bigint(trim(' 123 ')),bigint(trim(' 456'))) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT8_TBL VALUES(bigint(trim('123 ')),bigint('4567890123456789')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT8_TBL VALUES(bigint('4567890123456789'),bigint('123')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT8_TBL VALUES(+4567890123456789,bigint('4567890123456789')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO INT8_TBL VALUES(bigint('+4567890123456789'),bigint('-4567890123456789')) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM INT8_TBL +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 = 4567890123456789 +-- !query schema +struct +-- !query output +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 <> 4567890123456789 +-- !query schema +struct +-- !query output +123 456 +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 < 4567890123456789 +-- !query schema +struct +-- !query output +123 456 +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 > 4567890123456789 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 <= 4567890123456789 +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 >= 4567890123456789 +-- !query schema +struct +-- !query output +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 = 456 +-- !query schema +struct +-- !query output +123 456 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 <> 456 +-- !query schema +struct +-- !query output +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 < 456 +-- !query schema +struct +-- !query output +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 > 456 +-- !query schema +struct +-- !query output +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 <= 456 +-- !query schema +struct +-- !query output +123 456 +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 >= 456 +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE 123 = q1 +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE 123 <> q1 +-- !query schema +struct +-- !query output +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE 123 < q1 +-- !query schema +struct +-- !query output +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE 123 > q1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM INT8_TBL WHERE 123 <= q1 +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE 123 >= q1 +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 = smallint('456') +-- !query schema +struct +-- !query output +123 456 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 <> smallint('456') +-- !query schema +struct +-- !query output +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 < smallint('456') +-- !query schema +struct +-- !query output +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 > smallint('456') +-- !query schema +struct +-- !query output +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 <= smallint('456') +-- !query schema +struct +-- !query output +123 456 +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query +SELECT * FROM INT8_TBL WHERE q2 >= smallint('456') +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE smallint('123') = q1 +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE smallint('123') <> q1 +-- !query schema +struct +-- !query output +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE smallint('123') < q1 +-- !query schema +struct +-- !query output +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE smallint('123') > q1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM INT8_TBL WHERE smallint('123') <= q1 +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +SELECT * FROM INT8_TBL WHERE smallint('123') >= q1 +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 + + +-- !query +SELECT '' AS five, q1 AS plus, -q1 AS `minus` FROM INT8_TBL +-- !query schema +struct +-- !query output + 123 -123 + 123 -123 + 4567890123456789 -4567890123456789 + 4567890123456789 -4567890123456789 + 4567890123456789 -4567890123456789 + + +-- !query +SELECT '' AS five, q1, q2, q1 + q2 AS plus FROM INT8_TBL +-- !query schema +struct +-- !query output + 123 456 579 + 123 4567890123456789 4567890123456912 + 4567890123456789 -4567890123456789 0 + 4567890123456789 123 4567890123456912 + 4567890123456789 4567890123456789 9135780246913578 + + +-- !query +SELECT '' AS five, q1, q2, q1 - q2 AS `minus` FROM INT8_TBL +-- !query schema +struct +-- !query output + 123 456 -333 + 123 4567890123456789 -4567890123456666 + 4567890123456789 -4567890123456789 9135780246913578 + 4567890123456789 123 4567890123456666 + 4567890123456789 4567890123456789 0 + + +-- !query +SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_multiply' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 35, + "fragment" : "q1 * q2" + } ] +} + + +-- !query +SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL + WHERE q1 < 1000 or (q2 > 0 and q2 < 1000) +-- !query schema +struct +-- !query output + 123 456 56088 + 123 4567890123456789 561850485185185047 + 4567890123456789 123 561850485185185047 + + +-- !query +SELECT '' AS five, q1, q2, q1 / q2 AS divide, q1 % q2 AS mod FROM INT8_TBL +-- !query schema +struct +-- !query output + 123 456 0.26973684210526316 123 + 123 4567890123456789 2.6927092525360204E-14 123 + 4567890123456789 -4567890123456789 -1.0 0 + 4567890123456789 123 3.713731807688446E13 57 + 4567890123456789 4567890123456789 1.0 0 + + +-- !query +SELECT '' AS five, q1, double(q1) FROM INT8_TBL +-- !query schema +struct +-- !query output + 123 123.0 + 123 123.0 + 4567890123456789 4.567890123456789E15 + 4567890123456789 4.567890123456789E15 + 4567890123456789 4.567890123456789E15 + + +-- !query +SELECT '' AS five, q2, double(q2) FROM INT8_TBL +-- !query schema +struct +-- !query output + -4567890123456789 -4.567890123456789E15 + 123 123.0 + 456 456.0 + 4567890123456789 4.567890123456789E15 + 4567890123456789 4.567890123456789E15 + + +-- !query +SELECT 37 + q1 AS plus4 FROM INT8_TBL +-- !query schema +struct +-- !query output +160 +160 +4567890123456826 +4567890123456826 +4567890123456826 + + +-- !query +SELECT 37 - q1 AS minus4 FROM INT8_TBL +-- !query schema +struct +-- !query output +-4567890123456752 +-4567890123456752 +-4567890123456752 +-86 +-86 + + +-- !query +SELECT '' AS five, 2 * q1 AS `twice int4` FROM INT8_TBL +-- !query schema +struct +-- !query output + 246 + 246 + 9135780246913578 + 9135780246913578 + 9135780246913578 + + +-- !query +SELECT '' AS five, q1 * 2 AS `twice int4` FROM INT8_TBL +-- !query schema +struct +-- !query output + 246 + 246 + 9135780246913578 + 9135780246913578 + 9135780246913578 + + +-- !query +SELECT q1 + int(42) AS `8plus4`, q1 - int(42) AS `8minus4`, q1 * int(42) AS `8mul4`, q1 / int(42) AS `8div4` FROM INT8_TBL +-- !query schema +struct<8plus4:bigint,8minus4:bigint,8mul4:bigint,8div4:double> +-- !query output +165 81 5166 2.9285714285714284 +165 81 5166 2.9285714285714284 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 + + +-- !query +SELECT int(246) + q1 AS `4plus8`, int(246) - q1 AS `4minus8`, int(246) * q1 AS `4mul8`, int(246) / q1 AS `4div8` FROM INT8_TBL +-- !query schema +struct<4plus8:bigint,4minus8:bigint,4mul8:bigint,4div8:double> +-- !query output +369 123 30258 2.0 +369 123 30258 2.0 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 + + +-- !query +SELECT q1 + smallint(42) AS `8plus2`, q1 - smallint(42) AS `8minus2`, q1 * smallint(42) AS `8mul2`, q1 / smallint(42) AS `8div2` FROM INT8_TBL +-- !query schema +struct<8plus2:bigint,8minus2:bigint,8mul2:bigint,8div2:double> +-- !query output +165 81 5166 2.9285714285714284 +165 81 5166 2.9285714285714284 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 +4567890123456831 4567890123456747 191851385185185138 1.0875928865373308E14 + + +-- !query +SELECT smallint(246) + q1 AS `2plus8`, smallint(246) - q1 AS `2minus8`, smallint(246) * q1 AS `2mul8`, smallint(246) / q1 AS `2div8` FROM INT8_TBL +-- !query schema +struct<2plus8:bigint,2minus8:bigint,2mul8:bigint,2div8:double> +-- !query output +369 123 30258 2.0 +369 123 30258 2.0 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 +4567890123457035 -4567890123456543 1123700970370370094 5.385418505072041E-14 + + +-- !query +SELECT q2, abs(q2) FROM INT8_TBL +-- !query schema +struct +-- !query output +-4567890123456789 4567890123456789 +123 123 +456 456 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query +SELECT min(q1), min(q2) FROM INT8_TBL +-- !query schema +struct +-- !query output +123 -4567890123456789 + + +-- !query +SELECT max(q1), max(q2) FROM INT8_TBL +-- !query schema +struct +-- !query output +4567890123456789 4567890123456789 + + +-- !query +SELECT '' AS to_char_1, to_char(q1, '9G999G999G999G999G999'), to_char(q2, '9,999,999,999,999,999') +FROM INT8_TBL +-- !query schema +struct +-- !query output + 123 456 + 123 4,567,890,123,456,789 + 4,567,890,123,456,789 123 + 4,567,890,123,456,789 4,567,890,123,456,789 + 4,567,890,123,456,789 4,567,890,123,456,789 + + +-- !query +SELECT '' AS to_char_3, to_char( (q1 * -1), '9999999999999999PR'), to_char( (q2 * -1), '9999999999999999.999PR') +FROM INT8_TBL +-- !query schema +struct +-- !query output + <123> <456.000> + <123> <4567890123456789.000> + <4567890123456789> <123.000> + <4567890123456789> 4567890123456789.000 + <4567890123456789> <4567890123456789.000> + + +-- !query +SELECT '' AS to_char_4, to_char( (q1 * -1), '9999999999999999S'), to_char( (q2 * -1), 'S9999999999999999') +FROM INT8_TBL +-- !query schema +struct +-- !query output + 123- -456 + 123- -4567890123456789 + 4567890123456789- -123 + 4567890123456789- +4567890123456789 + 4567890123456789- -4567890123456789 + + +-- !query +SELECT '' AS to_char_5, to_char(q2, 'MI9999999999999999') FROM INT8_TBL +-- !query schema +struct +-- !query output + 123 + 456 + 4567890123456789 + 4567890123456789 + -4567890123456789 + + +-- !query +SELECT '' AS to_char_9, to_char(q2, '0999999999999999') FROM INT8_TBL +-- !query schema +struct +-- !query output + 0000000000000123 + 0000000000000456 + 4567890123456789 + 4567890123456789 + 4567890123456789 + + +-- !query +SELECT '' AS to_char_10, to_char(q2, 'S0999999999999999') FROM INT8_TBL +-- !query schema +struct +-- !query output + +0000000000000123 + +0000000000000456 + +4567890123456789 + +4567890123456789 + -4567890123456789 + + +-- !query +select bigint('9223372036854775800') / bigint('0') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "bigint('9223372036854775800') / bigint('0')" + } ] +} + + +-- !query +select bigint('-9223372036854775808') / smallint('0') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "bigint('-9223372036854775808') / smallint('0')" + } ] +} + + +-- !query +select smallint('100') / bigint('0') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "smallint('100') / bigint('0')" + } ] +} + + +-- !query +SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 = 456 +-- !query schema +struct +-- !query output +123 + + +-- !query +SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 <> 456 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"BIGINT\"", + "targetType" : "\"INT\"", + "value" : "4567890123456789L" + } +} + + +-- !query +SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 = 456 +-- !query schema +struct +-- !query output +123 + + +-- !query +SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 <> 456 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"BIGINT\"", + "targetType" : "\"SMALLINT\"", + "value" : "4567890123456789L" + } +} + + +-- !query +SELECT CAST(smallint('42') AS bigint), CAST(smallint('-37') AS bigint) +-- !query schema +struct +-- !query output +42 -37 + + +-- !query +SELECT CAST(q1 AS float), CAST(q2 AS double) FROM INT8_TBL +-- !query schema +struct +-- !query output +123.0 4.567890123456789E15 +123.0 456.0 +4.56789E15 -4.567890123456789E15 +4.56789E15 123.0 +4.56789E15 4.567890123456789E15 + + +-- !query +SELECT CAST(float('36854775807.0') AS bigint) +-- !query schema +struct +-- !query output +36854775808 + + +-- !query +SELECT CAST(double('922337203685477580700.0') AS bigint) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"DOUBLE\"", + "targetType" : "\"BIGINT\"", + "value" : "9.223372036854776E20D" + } +} + + +-- !query +SELECT q1, q2, q1 & q2 AS `and`, q1 | q2 AS `or`, ~q1 AS `not` FROM INT8_TBL +-- !query schema +struct +-- !query output +123 456 72 507 -124 +123 4567890123456789 17 4567890123456895 -124 +4567890123456789 -4567890123456789 1 -1 -4567890123456790 +4567890123456789 123 17 4567890123456895 -4567890123456790 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 -4567890123456790 + + +-- !query +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799')) +-- !query schema +struct +-- !query output +4567890123456789 +4567890123456790 +4567890123456791 +4567890123456792 +4567890123456793 +4567890123456794 +4567890123456795 +4567890123456796 +4567890123456797 +4567890123456798 + + +-- !query +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799'), 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "FAILED_FUNCTION_CALL", + "sqlState" : "38000", + "messageParameters" : { + "funcName" : "`range`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 80, + "fragment" : "range(bigint('+4567890123456789'), bigint('+4567890123456799'), 0)" + } ] +} + + +-- !query +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799'), 2) +-- !query schema +struct +-- !query output +4567890123456789 +4567890123456791 +4567890123456793 +4567890123456795 +4567890123456797 + + +-- !query +SELECT string(shiftleft(bigint(-1), 63)) +-- !query schema +struct +-- !query output +-9223372036854775808 + + +-- !query +SELECT string(int(shiftleft(bigint(-1), 63))+1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"BIGINT\"", + "targetType" : "\"INT\"", + "value" : "-9223372036854775808L" + } +} + + +-- !query +SELECT bigint((-9223372036854775808)) * bigint((-1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_multiply' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "bigint((-9223372036854775808)) * bigint((-1))" + } ] +} + + +-- !query +SELECT bigint((-9223372036854775808)) / bigint((-1)) +-- !query schema +struct<(-9223372036854775808 / -1):double> +-- !query output +9.223372036854776E18 + + +-- !query +SELECT bigint((-9223372036854775808)) % bigint((-1)) +-- !query schema +struct<(-9223372036854775808 % -1):bigint> +-- !query output +0 + + +-- !query +SELECT bigint((-9223372036854775808)) * int((-1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_multiply' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "bigint((-9223372036854775808)) * int((-1))" + } ] +} + + +-- !query +SELECT bigint((-9223372036854775808)) / int((-1)) +-- !query schema +struct<(-9223372036854775808 / -1):double> +-- !query output +9.223372036854776E18 + + +-- !query +SELECT bigint((-9223372036854775808)) % int((-1)) +-- !query schema +struct<(-9223372036854775808 % -1):bigint> +-- !query output +0 + + +-- !query +SELECT bigint((-9223372036854775808)) * smallint((-1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_multiply' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "bigint((-9223372036854775808)) * smallint((-1))" + } ] +} + + +-- !query +SELECT bigint((-9223372036854775808)) / smallint((-1)) +-- !query schema +struct<(-9223372036854775808 / -1):double> +-- !query output +9.223372036854776E18 + + +-- !query +SELECT bigint((-9223372036854775808)) % smallint((-1)) +-- !query schema +struct<(-9223372036854775808 % -1):bigint> +-- !query output +0 + + +-- !query +SELECT x, bigint(x) AS int8_value +FROM (VALUES (double(-2.5)), + (double(-1.5)), + (double(-0.5)), + (double(0.0)), + (double(0.5)), + (double(1.5)), + (double(2.5))) t(x) +-- !query schema +struct +-- !query output +-0.5 0 +-1.5 -1 +-2.5 -2 +0.0 0 +0.5 0 +1.5 1 +2.5 2 + + +-- !query +SELECT x, bigint(x) AS int8_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x) +-- !query schema +struct +-- !query output +-0.500000000000000000 0 +-1.500000000000000000 -1 +-2.500000000000000000 -2 +0.000000000000000000 0 +0.500000000000000000 0 +1.500000000000000000 1 +2.500000000000000000 2 + + +-- !query +DROP TABLE INT8_TBL +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/interval.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/interval.sql.out new file mode 100644 index 000000000000..3855d922361b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/interval.sql.out @@ -0,0 +1,361 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT interval '999' second +-- !query schema +struct +-- !query output +16 minutes 39 seconds + + +-- !query +SELECT interval '999' minute +-- !query schema +struct +-- !query output +16 hours 39 minutes + + +-- !query +SELECT interval '999' hour +-- !query schema +struct +-- !query output +999 hours + + +-- !query +SELECT interval '999' day +-- !query schema +struct +-- !query output +999 days + + +-- !query +SELECT interval '999' month +-- !query schema +struct +-- !query output +83 years 3 months + + +-- !query +SELECT interval '1' year +-- !query schema +struct +-- !query output +1 years + + +-- !query +SELECT interval '2' month +-- !query schema +struct +-- !query output +2 months + + +-- !query +SELECT interval '3' day +-- !query schema +struct +-- !query output +3 days + + +-- !query +SELECT interval '4' hour +-- !query schema +struct +-- !query output +4 hours + + +-- !query +SELECT interval '5' minute +-- !query schema +struct +-- !query output +5 minutes + + +-- !query +SELECT interval '6' second +-- !query schema +struct +-- !query output +6 seconds + + +-- !query +SELECT interval '1-2' year to month +-- !query schema +struct +-- !query output +1 years 2 months + + +-- !query +SELECT interval '1 2:03' day to hour +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "1 2:03", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR`", + "typeName" : "interval day to hour" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 36, + "fragment" : "'1 2:03' day to hour" + } ] +} + + +-- !query +SELECT interval '1 2:03:04' day to hour +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "1 2:03:04", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR`", + "typeName" : "interval day to hour" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 39, + "fragment" : "'1 2:03:04' day to hour" + } ] +} + + +-- !query +SELECT interval '1 2:03' day to minute +-- !query schema +struct +-- !query output +1 days 2 hours 3 minutes + + +-- !query +SELECT interval '1 2:03:04' day to minute +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "1 2:03:04", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]d h:m`, `INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE`", + "typeName" : "interval day to minute" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 41, + "fragment" : "'1 2:03:04' day to minute" + } ] +} + + +-- !query +SELECT interval '1 2:03' day to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "1 2:03", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]d h:m:s.n`, `INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND`", + "typeName" : "interval day to second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 38, + "fragment" : "'1 2:03' day to second" + } ] +} + + +-- !query +SELECT interval '1 2:03:04' day to second +-- !query schema +struct +-- !query output +1 days 2 hours 3 minutes 4 seconds + + +-- !query +SELECT interval '1 2:03' hour to minute +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "1 2:03", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE`", + "typeName" : "interval hour to minute" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 39, + "fragment" : "'1 2:03' hour to minute" + } ] +} + + +-- !query +SELECT interval '1 2:03:04' hour to minute +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "1 2:03:04", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE`", + "typeName" : "interval hour to minute" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 42, + "fragment" : "'1 2:03:04' hour to minute" + } ] +} + + +-- !query +SELECT interval '1 2:03' hour to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "1 2:03", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND`", + "typeName" : "interval hour to second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 39, + "fragment" : "'1 2:03' hour to second" + } ] +} + + +-- !query +SELECT interval '1 2:03:04' hour to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "1 2:03:04", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND`", + "typeName" : "interval hour to second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 42, + "fragment" : "'1 2:03:04' hour to second" + } ] +} + + +-- !query +SELECT interval '1 2:03' minute to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "1 2:03", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND`", + "typeName" : "interval minute to second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 41, + "fragment" : "'1 2:03' minute to second" + } ] +} + + +-- !query +SELECT interval '1 2:03:04' minute to second +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.UNMATCHED_FORMAT_STRING_WITH_NOTICE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "1 2:03:04", + "intervalStr" : "day-time", + "supportedFormat" : "`[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND`", + "typeName" : "interval minute to second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 44, + "fragment" : "'1 2:03:04' minute to second" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/join.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/join.sql.out new file mode 100644 index 000000000000..55aaff818ea9 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/join.sql.out @@ -0,0 +1,3799 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW INT2_TBL(f1) AS VALUES + (smallint(trim('0 '))), + (smallint(trim(' 1234 '))), + (smallint(trim(' -1234'))), + (smallint('32767')), + (smallint('-32767')) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM + (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) + AS v(f1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM + (VALUES + (123, 456), + (123, 4567890123456789), + (4567890123456789, 123), + (4567890123456789, 4567890123456789), + (4567890123456789, -4567890123456789)) + AS v(q1, q2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW FLOAT8_TBL AS SELECT * FROM + (VALUES (0.0), (1004.30), (-34.84), + (cast('1.2345678901234e+200' as double)), (cast('1.2345678901234e-200' as double))) + AS v(f1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW TEXT_TBL AS SELECT * FROM + (VALUES ('doh!'), ('hi de ho neighbor')) + AS v(f1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE J1_TBL ( + i integer, + j integer, + t string +) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE J2_TBL ( + i integer, + k integer +) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (1, 4, 'one') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (2, 3, 'two') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (3, 2, 'three') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (4, 1, 'four') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (5, 0, 'five') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (6, 6, 'six') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (7, 7, 'seven') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (8, 8, 'eight') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (0, NULL, 'zero') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (NULL, NULL, 'null') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (NULL, 0, 'zero') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J2_TBL VALUES (1, -1) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J2_TBL VALUES (2, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J2_TBL VALUES (3, -3) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J2_TBL VALUES (2, 4) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J2_TBL VALUES (5, -5) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J2_TBL VALUES (5, -5) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J2_TBL VALUES (0, NULL) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J2_TBL VALUES (NULL, NULL) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J2_TBL VALUES (NULL, 0) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL AS tx +-- !query schema +struct +-- !query output + 0 NULL zero + 1 4 one + 2 3 two + 3 2 three + 4 1 four + 5 0 five + 6 6 six + 7 7 seven + 8 8 eight + NULL 0 zero + NULL NULL null + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL tx +-- !query schema +struct +-- !query output + 0 NULL zero + 1 4 one + 2 3 two + 3 2 three + 4 1 four + 5 0 five + 6 6 six + 7 7 seven + 8 8 eight + NULL 0 zero + NULL NULL null + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL AS t1 (a, b, c) +-- !query schema +struct +-- !query output + 0 NULL zero + 1 4 one + 2 3 two + 3 2 three + 4 1 four + 5 0 five + 6 6 six + 7 7 seven + 8 8 eight + NULL 0 zero + NULL NULL null + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL t1 (a, b, c) +-- !query schema +struct +-- !query output + 0 NULL zero + 1 4 one + 2 3 two + 3 2 three + 4 1 four + 5 0 five + 6 6 six + 7 7 seven + 8 8 eight + NULL 0 zero + NULL NULL null + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL t1 (a, b, c), J2_TBL t2 (d, e) +-- !query schema +struct +-- !query output + 0 NULL zero 0 NULL + 0 NULL zero 1 -1 + 0 NULL zero 2 2 + 0 NULL zero 2 4 + 0 NULL zero 3 -3 + 0 NULL zero 5 -5 + 0 NULL zero 5 -5 + 0 NULL zero NULL 0 + 0 NULL zero NULL NULL + 1 4 one 0 NULL + 1 4 one 1 -1 + 1 4 one 2 2 + 1 4 one 2 4 + 1 4 one 3 -3 + 1 4 one 5 -5 + 1 4 one 5 -5 + 1 4 one NULL 0 + 1 4 one NULL NULL + 2 3 two 0 NULL + 2 3 two 1 -1 + 2 3 two 2 2 + 2 3 two 2 4 + 2 3 two 3 -3 + 2 3 two 5 -5 + 2 3 two 5 -5 + 2 3 two NULL 0 + 2 3 two NULL NULL + 3 2 three 0 NULL + 3 2 three 1 -1 + 3 2 three 2 2 + 3 2 three 2 4 + 3 2 three 3 -3 + 3 2 three 5 -5 + 3 2 three 5 -5 + 3 2 three NULL 0 + 3 2 three NULL NULL + 4 1 four 0 NULL + 4 1 four 1 -1 + 4 1 four 2 2 + 4 1 four 2 4 + 4 1 four 3 -3 + 4 1 four 5 -5 + 4 1 four 5 -5 + 4 1 four NULL 0 + 4 1 four NULL NULL + 5 0 five 0 NULL + 5 0 five 1 -1 + 5 0 five 2 2 + 5 0 five 2 4 + 5 0 five 3 -3 + 5 0 five 5 -5 + 5 0 five 5 -5 + 5 0 five NULL 0 + 5 0 five NULL NULL + 6 6 six 0 NULL + 6 6 six 1 -1 + 6 6 six 2 2 + 6 6 six 2 4 + 6 6 six 3 -3 + 6 6 six 5 -5 + 6 6 six 5 -5 + 6 6 six NULL 0 + 6 6 six NULL NULL + 7 7 seven 0 NULL + 7 7 seven 1 -1 + 7 7 seven 2 2 + 7 7 seven 2 4 + 7 7 seven 3 -3 + 7 7 seven 5 -5 + 7 7 seven 5 -5 + 7 7 seven NULL 0 + 7 7 seven NULL NULL + 8 8 eight 0 NULL + 8 8 eight 1 -1 + 8 8 eight 2 2 + 8 8 eight 2 4 + 8 8 eight 3 -3 + 8 8 eight 5 -5 + 8 8 eight 5 -5 + 8 8 eight NULL 0 + 8 8 eight NULL NULL + NULL 0 zero 0 NULL + NULL 0 zero 1 -1 + NULL 0 zero 2 2 + NULL 0 zero 2 4 + NULL 0 zero 3 -3 + NULL 0 zero 5 -5 + NULL 0 zero 5 -5 + NULL 0 zero NULL 0 + NULL 0 zero NULL NULL + NULL NULL null 0 NULL + NULL NULL null 1 -1 + NULL NULL null 2 2 + NULL NULL null 2 4 + NULL NULL null 3 -3 + NULL NULL null 5 -5 + NULL NULL null 5 -5 + NULL NULL null NULL 0 + NULL NULL null NULL NULL + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL CROSS JOIN J2_TBL +-- !query schema +struct +-- !query output + 0 NULL zero 0 NULL + 0 NULL zero 1 -1 + 0 NULL zero 2 2 + 0 NULL zero 2 4 + 0 NULL zero 3 -3 + 0 NULL zero 5 -5 + 0 NULL zero 5 -5 + 0 NULL zero NULL 0 + 0 NULL zero NULL NULL + 1 4 one 0 NULL + 1 4 one 1 -1 + 1 4 one 2 2 + 1 4 one 2 4 + 1 4 one 3 -3 + 1 4 one 5 -5 + 1 4 one 5 -5 + 1 4 one NULL 0 + 1 4 one NULL NULL + 2 3 two 0 NULL + 2 3 two 1 -1 + 2 3 two 2 2 + 2 3 two 2 4 + 2 3 two 3 -3 + 2 3 two 5 -5 + 2 3 two 5 -5 + 2 3 two NULL 0 + 2 3 two NULL NULL + 3 2 three 0 NULL + 3 2 three 1 -1 + 3 2 three 2 2 + 3 2 three 2 4 + 3 2 three 3 -3 + 3 2 three 5 -5 + 3 2 three 5 -5 + 3 2 three NULL 0 + 3 2 three NULL NULL + 4 1 four 0 NULL + 4 1 four 1 -1 + 4 1 four 2 2 + 4 1 four 2 4 + 4 1 four 3 -3 + 4 1 four 5 -5 + 4 1 four 5 -5 + 4 1 four NULL 0 + 4 1 four NULL NULL + 5 0 five 0 NULL + 5 0 five 1 -1 + 5 0 five 2 2 + 5 0 five 2 4 + 5 0 five 3 -3 + 5 0 five 5 -5 + 5 0 five 5 -5 + 5 0 five NULL 0 + 5 0 five NULL NULL + 6 6 six 0 NULL + 6 6 six 1 -1 + 6 6 six 2 2 + 6 6 six 2 4 + 6 6 six 3 -3 + 6 6 six 5 -5 + 6 6 six 5 -5 + 6 6 six NULL 0 + 6 6 six NULL NULL + 7 7 seven 0 NULL + 7 7 seven 1 -1 + 7 7 seven 2 2 + 7 7 seven 2 4 + 7 7 seven 3 -3 + 7 7 seven 5 -5 + 7 7 seven 5 -5 + 7 7 seven NULL 0 + 7 7 seven NULL NULL + 8 8 eight 0 NULL + 8 8 eight 1 -1 + 8 8 eight 2 2 + 8 8 eight 2 4 + 8 8 eight 3 -3 + 8 8 eight 5 -5 + 8 8 eight 5 -5 + 8 8 eight NULL 0 + 8 8 eight NULL NULL + NULL 0 zero 0 NULL + NULL 0 zero 1 -1 + NULL 0 zero 2 2 + NULL 0 zero 2 4 + NULL 0 zero 3 -3 + NULL 0 zero 5 -5 + NULL 0 zero 5 -5 + NULL 0 zero NULL 0 + NULL 0 zero NULL NULL + NULL NULL null 0 NULL + NULL NULL null 1 -1 + NULL NULL null 2 2 + NULL NULL null 2 4 + NULL NULL null 3 -3 + NULL NULL null 5 -5 + NULL NULL null 5 -5 + NULL NULL null NULL 0 + NULL NULL null NULL NULL + + +-- !query +SELECT '' AS `xxx`, i, k, t + FROM J1_TBL CROSS JOIN J2_TBL +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`i`", + "referenceNames" : "[`spark_catalog`.`default`.`j1_tbl`.`i`, `spark_catalog`.`default`.`j2_tbl`.`i`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 21, + "stopIndex" : 21, + "fragment" : "i" + } ] +} + + +-- !query +SELECT '' AS `xxx`, t1.i, k, t + FROM J1_TBL t1 CROSS JOIN J2_TBL t2 +-- !query schema +struct +-- !query output + 0 -1 zero + 0 -3 zero + 0 -5 zero + 0 -5 zero + 0 0 zero + 0 2 zero + 0 4 zero + 0 NULL zero + 0 NULL zero + 1 -1 one + 1 -3 one + 1 -5 one + 1 -5 one + 1 0 one + 1 2 one + 1 4 one + 1 NULL one + 1 NULL one + 2 -1 two + 2 -3 two + 2 -5 two + 2 -5 two + 2 0 two + 2 2 two + 2 4 two + 2 NULL two + 2 NULL two + 3 -1 three + 3 -3 three + 3 -5 three + 3 -5 three + 3 0 three + 3 2 three + 3 4 three + 3 NULL three + 3 NULL three + 4 -1 four + 4 -3 four + 4 -5 four + 4 -5 four + 4 0 four + 4 2 four + 4 4 four + 4 NULL four + 4 NULL four + 5 -1 five + 5 -3 five + 5 -5 five + 5 -5 five + 5 0 five + 5 2 five + 5 4 five + 5 NULL five + 5 NULL five + 6 -1 six + 6 -3 six + 6 -5 six + 6 -5 six + 6 0 six + 6 2 six + 6 4 six + 6 NULL six + 6 NULL six + 7 -1 seven + 7 -3 seven + 7 -5 seven + 7 -5 seven + 7 0 seven + 7 2 seven + 7 4 seven + 7 NULL seven + 7 NULL seven + 8 -1 eight + 8 -3 eight + 8 -5 eight + 8 -5 eight + 8 0 eight + 8 2 eight + 8 4 eight + 8 NULL eight + 8 NULL eight + NULL -1 null + NULL -1 zero + NULL -3 null + NULL -3 zero + NULL -5 null + NULL -5 null + NULL -5 zero + NULL -5 zero + NULL 0 null + NULL 0 zero + NULL 2 null + NULL 2 zero + NULL 4 null + NULL 4 zero + NULL NULL null + NULL NULL null + NULL NULL zero + NULL NULL zero + + +-- !query +SELECT '' AS `xxx`, ii, tt, kk + FROM (J1_TBL CROSS JOIN J2_TBL) + AS tx (ii, jj, tt, ii2, kk) +-- !query schema +struct +-- !query output + 0 zero -1 + 0 zero -3 + 0 zero -5 + 0 zero -5 + 0 zero 0 + 0 zero 2 + 0 zero 4 + 0 zero NULL + 0 zero NULL + 1 one -1 + 1 one -3 + 1 one -5 + 1 one -5 + 1 one 0 + 1 one 2 + 1 one 4 + 1 one NULL + 1 one NULL + 2 two -1 + 2 two -3 + 2 two -5 + 2 two -5 + 2 two 0 + 2 two 2 + 2 two 4 + 2 two NULL + 2 two NULL + 3 three -1 + 3 three -3 + 3 three -5 + 3 three -5 + 3 three 0 + 3 three 2 + 3 three 4 + 3 three NULL + 3 three NULL + 4 four -1 + 4 four -3 + 4 four -5 + 4 four -5 + 4 four 0 + 4 four 2 + 4 four 4 + 4 four NULL + 4 four NULL + 5 five -1 + 5 five -3 + 5 five -5 + 5 five -5 + 5 five 0 + 5 five 2 + 5 five 4 + 5 five NULL + 5 five NULL + 6 six -1 + 6 six -3 + 6 six -5 + 6 six -5 + 6 six 0 + 6 six 2 + 6 six 4 + 6 six NULL + 6 six NULL + 7 seven -1 + 7 seven -3 + 7 seven -5 + 7 seven -5 + 7 seven 0 + 7 seven 2 + 7 seven 4 + 7 seven NULL + 7 seven NULL + 8 eight -1 + 8 eight -3 + 8 eight -5 + 8 eight -5 + 8 eight 0 + 8 eight 2 + 8 eight 4 + 8 eight NULL + 8 eight NULL + NULL null -1 + NULL null -3 + NULL null -5 + NULL null -5 + NULL null 0 + NULL null 2 + NULL null 4 + NULL null NULL + NULL null NULL + NULL zero -1 + NULL zero -3 + NULL zero -5 + NULL zero -5 + NULL zero 0 + NULL zero 2 + NULL zero 4 + NULL zero NULL + NULL zero NULL + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL CROSS JOIN J2_TBL a CROSS JOIN J2_TBL b +-- !query schema +struct +-- !query output + 0 NULL zero 0 NULL 0 NULL + 0 NULL zero 0 NULL 1 -1 + 0 NULL zero 0 NULL 2 2 + 0 NULL zero 0 NULL 2 4 + 0 NULL zero 0 NULL 3 -3 + 0 NULL zero 0 NULL 5 -5 + 0 NULL zero 0 NULL 5 -5 + 0 NULL zero 0 NULL NULL 0 + 0 NULL zero 0 NULL NULL NULL + 0 NULL zero 1 -1 0 NULL + 0 NULL zero 1 -1 1 -1 + 0 NULL zero 1 -1 2 2 + 0 NULL zero 1 -1 2 4 + 0 NULL zero 1 -1 3 -3 + 0 NULL zero 1 -1 5 -5 + 0 NULL zero 1 -1 5 -5 + 0 NULL zero 1 -1 NULL 0 + 0 NULL zero 1 -1 NULL NULL + 0 NULL zero 2 2 0 NULL + 0 NULL zero 2 2 1 -1 + 0 NULL zero 2 2 2 2 + 0 NULL zero 2 2 2 4 + 0 NULL zero 2 2 3 -3 + 0 NULL zero 2 2 5 -5 + 0 NULL zero 2 2 5 -5 + 0 NULL zero 2 2 NULL 0 + 0 NULL zero 2 2 NULL NULL + 0 NULL zero 2 4 0 NULL + 0 NULL zero 2 4 1 -1 + 0 NULL zero 2 4 2 2 + 0 NULL zero 2 4 2 4 + 0 NULL zero 2 4 3 -3 + 0 NULL zero 2 4 5 -5 + 0 NULL zero 2 4 5 -5 + 0 NULL zero 2 4 NULL 0 + 0 NULL zero 2 4 NULL NULL + 0 NULL zero 3 -3 0 NULL + 0 NULL zero 3 -3 1 -1 + 0 NULL zero 3 -3 2 2 + 0 NULL zero 3 -3 2 4 + 0 NULL zero 3 -3 3 -3 + 0 NULL zero 3 -3 5 -5 + 0 NULL zero 3 -3 5 -5 + 0 NULL zero 3 -3 NULL 0 + 0 NULL zero 3 -3 NULL NULL + 0 NULL zero 5 -5 0 NULL + 0 NULL zero 5 -5 0 NULL + 0 NULL zero 5 -5 1 -1 + 0 NULL zero 5 -5 1 -1 + 0 NULL zero 5 -5 2 2 + 0 NULL zero 5 -5 2 2 + 0 NULL zero 5 -5 2 4 + 0 NULL zero 5 -5 2 4 + 0 NULL zero 5 -5 3 -3 + 0 NULL zero 5 -5 3 -3 + 0 NULL zero 5 -5 5 -5 + 0 NULL zero 5 -5 5 -5 + 0 NULL zero 5 -5 5 -5 + 0 NULL zero 5 -5 5 -5 + 0 NULL zero 5 -5 NULL 0 + 0 NULL zero 5 -5 NULL 0 + 0 NULL zero 5 -5 NULL NULL + 0 NULL zero 5 -5 NULL NULL + 0 NULL zero NULL 0 0 NULL + 0 NULL zero NULL 0 1 -1 + 0 NULL zero NULL 0 2 2 + 0 NULL zero NULL 0 2 4 + 0 NULL zero NULL 0 3 -3 + 0 NULL zero NULL 0 5 -5 + 0 NULL zero NULL 0 5 -5 + 0 NULL zero NULL 0 NULL 0 + 0 NULL zero NULL 0 NULL NULL + 0 NULL zero NULL NULL 0 NULL + 0 NULL zero NULL NULL 1 -1 + 0 NULL zero NULL NULL 2 2 + 0 NULL zero NULL NULL 2 4 + 0 NULL zero NULL NULL 3 -3 + 0 NULL zero NULL NULL 5 -5 + 0 NULL zero NULL NULL 5 -5 + 0 NULL zero NULL NULL NULL 0 + 0 NULL zero NULL NULL NULL NULL + 1 4 one 0 NULL 0 NULL + 1 4 one 0 NULL 1 -1 + 1 4 one 0 NULL 2 2 + 1 4 one 0 NULL 2 4 + 1 4 one 0 NULL 3 -3 + 1 4 one 0 NULL 5 -5 + 1 4 one 0 NULL 5 -5 + 1 4 one 0 NULL NULL 0 + 1 4 one 0 NULL NULL NULL + 1 4 one 1 -1 0 NULL + 1 4 one 1 -1 1 -1 + 1 4 one 1 -1 2 2 + 1 4 one 1 -1 2 4 + 1 4 one 1 -1 3 -3 + 1 4 one 1 -1 5 -5 + 1 4 one 1 -1 5 -5 + 1 4 one 1 -1 NULL 0 + 1 4 one 1 -1 NULL NULL + 1 4 one 2 2 0 NULL + 1 4 one 2 2 1 -1 + 1 4 one 2 2 2 2 + 1 4 one 2 2 2 4 + 1 4 one 2 2 3 -3 + 1 4 one 2 2 5 -5 + 1 4 one 2 2 5 -5 + 1 4 one 2 2 NULL 0 + 1 4 one 2 2 NULL NULL + 1 4 one 2 4 0 NULL + 1 4 one 2 4 1 -1 + 1 4 one 2 4 2 2 + 1 4 one 2 4 2 4 + 1 4 one 2 4 3 -3 + 1 4 one 2 4 5 -5 + 1 4 one 2 4 5 -5 + 1 4 one 2 4 NULL 0 + 1 4 one 2 4 NULL NULL + 1 4 one 3 -3 0 NULL + 1 4 one 3 -3 1 -1 + 1 4 one 3 -3 2 2 + 1 4 one 3 -3 2 4 + 1 4 one 3 -3 3 -3 + 1 4 one 3 -3 5 -5 + 1 4 one 3 -3 5 -5 + 1 4 one 3 -3 NULL 0 + 1 4 one 3 -3 NULL NULL + 1 4 one 5 -5 0 NULL + 1 4 one 5 -5 0 NULL + 1 4 one 5 -5 1 -1 + 1 4 one 5 -5 1 -1 + 1 4 one 5 -5 2 2 + 1 4 one 5 -5 2 2 + 1 4 one 5 -5 2 4 + 1 4 one 5 -5 2 4 + 1 4 one 5 -5 3 -3 + 1 4 one 5 -5 3 -3 + 1 4 one 5 -5 5 -5 + 1 4 one 5 -5 5 -5 + 1 4 one 5 -5 5 -5 + 1 4 one 5 -5 5 -5 + 1 4 one 5 -5 NULL 0 + 1 4 one 5 -5 NULL 0 + 1 4 one 5 -5 NULL NULL + 1 4 one 5 -5 NULL NULL + 1 4 one NULL 0 0 NULL + 1 4 one NULL 0 1 -1 + 1 4 one NULL 0 2 2 + 1 4 one NULL 0 2 4 + 1 4 one NULL 0 3 -3 + 1 4 one NULL 0 5 -5 + 1 4 one NULL 0 5 -5 + 1 4 one NULL 0 NULL 0 + 1 4 one NULL 0 NULL NULL + 1 4 one NULL NULL 0 NULL + 1 4 one NULL NULL 1 -1 + 1 4 one NULL NULL 2 2 + 1 4 one NULL NULL 2 4 + 1 4 one NULL NULL 3 -3 + 1 4 one NULL NULL 5 -5 + 1 4 one NULL NULL 5 -5 + 1 4 one NULL NULL NULL 0 + 1 4 one NULL NULL NULL NULL + 2 3 two 0 NULL 0 NULL + 2 3 two 0 NULL 1 -1 + 2 3 two 0 NULL 2 2 + 2 3 two 0 NULL 2 4 + 2 3 two 0 NULL 3 -3 + 2 3 two 0 NULL 5 -5 + 2 3 two 0 NULL 5 -5 + 2 3 two 0 NULL NULL 0 + 2 3 two 0 NULL NULL NULL + 2 3 two 1 -1 0 NULL + 2 3 two 1 -1 1 -1 + 2 3 two 1 -1 2 2 + 2 3 two 1 -1 2 4 + 2 3 two 1 -1 3 -3 + 2 3 two 1 -1 5 -5 + 2 3 two 1 -1 5 -5 + 2 3 two 1 -1 NULL 0 + 2 3 two 1 -1 NULL NULL + 2 3 two 2 2 0 NULL + 2 3 two 2 2 1 -1 + 2 3 two 2 2 2 2 + 2 3 two 2 2 2 4 + 2 3 two 2 2 3 -3 + 2 3 two 2 2 5 -5 + 2 3 two 2 2 5 -5 + 2 3 two 2 2 NULL 0 + 2 3 two 2 2 NULL NULL + 2 3 two 2 4 0 NULL + 2 3 two 2 4 1 -1 + 2 3 two 2 4 2 2 + 2 3 two 2 4 2 4 + 2 3 two 2 4 3 -3 + 2 3 two 2 4 5 -5 + 2 3 two 2 4 5 -5 + 2 3 two 2 4 NULL 0 + 2 3 two 2 4 NULL NULL + 2 3 two 3 -3 0 NULL + 2 3 two 3 -3 1 -1 + 2 3 two 3 -3 2 2 + 2 3 two 3 -3 2 4 + 2 3 two 3 -3 3 -3 + 2 3 two 3 -3 5 -5 + 2 3 two 3 -3 5 -5 + 2 3 two 3 -3 NULL 0 + 2 3 two 3 -3 NULL NULL + 2 3 two 5 -5 0 NULL + 2 3 two 5 -5 0 NULL + 2 3 two 5 -5 1 -1 + 2 3 two 5 -5 1 -1 + 2 3 two 5 -5 2 2 + 2 3 two 5 -5 2 2 + 2 3 two 5 -5 2 4 + 2 3 two 5 -5 2 4 + 2 3 two 5 -5 3 -3 + 2 3 two 5 -5 3 -3 + 2 3 two 5 -5 5 -5 + 2 3 two 5 -5 5 -5 + 2 3 two 5 -5 5 -5 + 2 3 two 5 -5 5 -5 + 2 3 two 5 -5 NULL 0 + 2 3 two 5 -5 NULL 0 + 2 3 two 5 -5 NULL NULL + 2 3 two 5 -5 NULL NULL + 2 3 two NULL 0 0 NULL + 2 3 two NULL 0 1 -1 + 2 3 two NULL 0 2 2 + 2 3 two NULL 0 2 4 + 2 3 two NULL 0 3 -3 + 2 3 two NULL 0 5 -5 + 2 3 two NULL 0 5 -5 + 2 3 two NULL 0 NULL 0 + 2 3 two NULL 0 NULL NULL + 2 3 two NULL NULL 0 NULL + 2 3 two NULL NULL 1 -1 + 2 3 two NULL NULL 2 2 + 2 3 two NULL NULL 2 4 + 2 3 two NULL NULL 3 -3 + 2 3 two NULL NULL 5 -5 + 2 3 two NULL NULL 5 -5 + 2 3 two NULL NULL NULL 0 + 2 3 two NULL NULL NULL NULL + 3 2 three 0 NULL 0 NULL + 3 2 three 0 NULL 1 -1 + 3 2 three 0 NULL 2 2 + 3 2 three 0 NULL 2 4 + 3 2 three 0 NULL 3 -3 + 3 2 three 0 NULL 5 -5 + 3 2 three 0 NULL 5 -5 + 3 2 three 0 NULL NULL 0 + 3 2 three 0 NULL NULL NULL + 3 2 three 1 -1 0 NULL + 3 2 three 1 -1 1 -1 + 3 2 three 1 -1 2 2 + 3 2 three 1 -1 2 4 + 3 2 three 1 -1 3 -3 + 3 2 three 1 -1 5 -5 + 3 2 three 1 -1 5 -5 + 3 2 three 1 -1 NULL 0 + 3 2 three 1 -1 NULL NULL + 3 2 three 2 2 0 NULL + 3 2 three 2 2 1 -1 + 3 2 three 2 2 2 2 + 3 2 three 2 2 2 4 + 3 2 three 2 2 3 -3 + 3 2 three 2 2 5 -5 + 3 2 three 2 2 5 -5 + 3 2 three 2 2 NULL 0 + 3 2 three 2 2 NULL NULL + 3 2 three 2 4 0 NULL + 3 2 three 2 4 1 -1 + 3 2 three 2 4 2 2 + 3 2 three 2 4 2 4 + 3 2 three 2 4 3 -3 + 3 2 three 2 4 5 -5 + 3 2 three 2 4 5 -5 + 3 2 three 2 4 NULL 0 + 3 2 three 2 4 NULL NULL + 3 2 three 3 -3 0 NULL + 3 2 three 3 -3 1 -1 + 3 2 three 3 -3 2 2 + 3 2 three 3 -3 2 4 + 3 2 three 3 -3 3 -3 + 3 2 three 3 -3 5 -5 + 3 2 three 3 -3 5 -5 + 3 2 three 3 -3 NULL 0 + 3 2 three 3 -3 NULL NULL + 3 2 three 5 -5 0 NULL + 3 2 three 5 -5 0 NULL + 3 2 three 5 -5 1 -1 + 3 2 three 5 -5 1 -1 + 3 2 three 5 -5 2 2 + 3 2 three 5 -5 2 2 + 3 2 three 5 -5 2 4 + 3 2 three 5 -5 2 4 + 3 2 three 5 -5 3 -3 + 3 2 three 5 -5 3 -3 + 3 2 three 5 -5 5 -5 + 3 2 three 5 -5 5 -5 + 3 2 three 5 -5 5 -5 + 3 2 three 5 -5 5 -5 + 3 2 three 5 -5 NULL 0 + 3 2 three 5 -5 NULL 0 + 3 2 three 5 -5 NULL NULL + 3 2 three 5 -5 NULL NULL + 3 2 three NULL 0 0 NULL + 3 2 three NULL 0 1 -1 + 3 2 three NULL 0 2 2 + 3 2 three NULL 0 2 4 + 3 2 three NULL 0 3 -3 + 3 2 three NULL 0 5 -5 + 3 2 three NULL 0 5 -5 + 3 2 three NULL 0 NULL 0 + 3 2 three NULL 0 NULL NULL + 3 2 three NULL NULL 0 NULL + 3 2 three NULL NULL 1 -1 + 3 2 three NULL NULL 2 2 + 3 2 three NULL NULL 2 4 + 3 2 three NULL NULL 3 -3 + 3 2 three NULL NULL 5 -5 + 3 2 three NULL NULL 5 -5 + 3 2 three NULL NULL NULL 0 + 3 2 three NULL NULL NULL NULL + 4 1 four 0 NULL 0 NULL + 4 1 four 0 NULL 1 -1 + 4 1 four 0 NULL 2 2 + 4 1 four 0 NULL 2 4 + 4 1 four 0 NULL 3 -3 + 4 1 four 0 NULL 5 -5 + 4 1 four 0 NULL 5 -5 + 4 1 four 0 NULL NULL 0 + 4 1 four 0 NULL NULL NULL + 4 1 four 1 -1 0 NULL + 4 1 four 1 -1 1 -1 + 4 1 four 1 -1 2 2 + 4 1 four 1 -1 2 4 + 4 1 four 1 -1 3 -3 + 4 1 four 1 -1 5 -5 + 4 1 four 1 -1 5 -5 + 4 1 four 1 -1 NULL 0 + 4 1 four 1 -1 NULL NULL + 4 1 four 2 2 0 NULL + 4 1 four 2 2 1 -1 + 4 1 four 2 2 2 2 + 4 1 four 2 2 2 4 + 4 1 four 2 2 3 -3 + 4 1 four 2 2 5 -5 + 4 1 four 2 2 5 -5 + 4 1 four 2 2 NULL 0 + 4 1 four 2 2 NULL NULL + 4 1 four 2 4 0 NULL + 4 1 four 2 4 1 -1 + 4 1 four 2 4 2 2 + 4 1 four 2 4 2 4 + 4 1 four 2 4 3 -3 + 4 1 four 2 4 5 -5 + 4 1 four 2 4 5 -5 + 4 1 four 2 4 NULL 0 + 4 1 four 2 4 NULL NULL + 4 1 four 3 -3 0 NULL + 4 1 four 3 -3 1 -1 + 4 1 four 3 -3 2 2 + 4 1 four 3 -3 2 4 + 4 1 four 3 -3 3 -3 + 4 1 four 3 -3 5 -5 + 4 1 four 3 -3 5 -5 + 4 1 four 3 -3 NULL 0 + 4 1 four 3 -3 NULL NULL + 4 1 four 5 -5 0 NULL + 4 1 four 5 -5 0 NULL + 4 1 four 5 -5 1 -1 + 4 1 four 5 -5 1 -1 + 4 1 four 5 -5 2 2 + 4 1 four 5 -5 2 2 + 4 1 four 5 -5 2 4 + 4 1 four 5 -5 2 4 + 4 1 four 5 -5 3 -3 + 4 1 four 5 -5 3 -3 + 4 1 four 5 -5 5 -5 + 4 1 four 5 -5 5 -5 + 4 1 four 5 -5 5 -5 + 4 1 four 5 -5 5 -5 + 4 1 four 5 -5 NULL 0 + 4 1 four 5 -5 NULL 0 + 4 1 four 5 -5 NULL NULL + 4 1 four 5 -5 NULL NULL + 4 1 four NULL 0 0 NULL + 4 1 four NULL 0 1 -1 + 4 1 four NULL 0 2 2 + 4 1 four NULL 0 2 4 + 4 1 four NULL 0 3 -3 + 4 1 four NULL 0 5 -5 + 4 1 four NULL 0 5 -5 + 4 1 four NULL 0 NULL 0 + 4 1 four NULL 0 NULL NULL + 4 1 four NULL NULL 0 NULL + 4 1 four NULL NULL 1 -1 + 4 1 four NULL NULL 2 2 + 4 1 four NULL NULL 2 4 + 4 1 four NULL NULL 3 -3 + 4 1 four NULL NULL 5 -5 + 4 1 four NULL NULL 5 -5 + 4 1 four NULL NULL NULL 0 + 4 1 four NULL NULL NULL NULL + 5 0 five 0 NULL 0 NULL + 5 0 five 0 NULL 1 -1 + 5 0 five 0 NULL 2 2 + 5 0 five 0 NULL 2 4 + 5 0 five 0 NULL 3 -3 + 5 0 five 0 NULL 5 -5 + 5 0 five 0 NULL 5 -5 + 5 0 five 0 NULL NULL 0 + 5 0 five 0 NULL NULL NULL + 5 0 five 1 -1 0 NULL + 5 0 five 1 -1 1 -1 + 5 0 five 1 -1 2 2 + 5 0 five 1 -1 2 4 + 5 0 five 1 -1 3 -3 + 5 0 five 1 -1 5 -5 + 5 0 five 1 -1 5 -5 + 5 0 five 1 -1 NULL 0 + 5 0 five 1 -1 NULL NULL + 5 0 five 2 2 0 NULL + 5 0 five 2 2 1 -1 + 5 0 five 2 2 2 2 + 5 0 five 2 2 2 4 + 5 0 five 2 2 3 -3 + 5 0 five 2 2 5 -5 + 5 0 five 2 2 5 -5 + 5 0 five 2 2 NULL 0 + 5 0 five 2 2 NULL NULL + 5 0 five 2 4 0 NULL + 5 0 five 2 4 1 -1 + 5 0 five 2 4 2 2 + 5 0 five 2 4 2 4 + 5 0 five 2 4 3 -3 + 5 0 five 2 4 5 -5 + 5 0 five 2 4 5 -5 + 5 0 five 2 4 NULL 0 + 5 0 five 2 4 NULL NULL + 5 0 five 3 -3 0 NULL + 5 0 five 3 -3 1 -1 + 5 0 five 3 -3 2 2 + 5 0 five 3 -3 2 4 + 5 0 five 3 -3 3 -3 + 5 0 five 3 -3 5 -5 + 5 0 five 3 -3 5 -5 + 5 0 five 3 -3 NULL 0 + 5 0 five 3 -3 NULL NULL + 5 0 five 5 -5 0 NULL + 5 0 five 5 -5 0 NULL + 5 0 five 5 -5 1 -1 + 5 0 five 5 -5 1 -1 + 5 0 five 5 -5 2 2 + 5 0 five 5 -5 2 2 + 5 0 five 5 -5 2 4 + 5 0 five 5 -5 2 4 + 5 0 five 5 -5 3 -3 + 5 0 five 5 -5 3 -3 + 5 0 five 5 -5 5 -5 + 5 0 five 5 -5 5 -5 + 5 0 five 5 -5 5 -5 + 5 0 five 5 -5 5 -5 + 5 0 five 5 -5 NULL 0 + 5 0 five 5 -5 NULL 0 + 5 0 five 5 -5 NULL NULL + 5 0 five 5 -5 NULL NULL + 5 0 five NULL 0 0 NULL + 5 0 five NULL 0 1 -1 + 5 0 five NULL 0 2 2 + 5 0 five NULL 0 2 4 + 5 0 five NULL 0 3 -3 + 5 0 five NULL 0 5 -5 + 5 0 five NULL 0 5 -5 + 5 0 five NULL 0 NULL 0 + 5 0 five NULL 0 NULL NULL + 5 0 five NULL NULL 0 NULL + 5 0 five NULL NULL 1 -1 + 5 0 five NULL NULL 2 2 + 5 0 five NULL NULL 2 4 + 5 0 five NULL NULL 3 -3 + 5 0 five NULL NULL 5 -5 + 5 0 five NULL NULL 5 -5 + 5 0 five NULL NULL NULL 0 + 5 0 five NULL NULL NULL NULL + 6 6 six 0 NULL 0 NULL + 6 6 six 0 NULL 1 -1 + 6 6 six 0 NULL 2 2 + 6 6 six 0 NULL 2 4 + 6 6 six 0 NULL 3 -3 + 6 6 six 0 NULL 5 -5 + 6 6 six 0 NULL 5 -5 + 6 6 six 0 NULL NULL 0 + 6 6 six 0 NULL NULL NULL + 6 6 six 1 -1 0 NULL + 6 6 six 1 -1 1 -1 + 6 6 six 1 -1 2 2 + 6 6 six 1 -1 2 4 + 6 6 six 1 -1 3 -3 + 6 6 six 1 -1 5 -5 + 6 6 six 1 -1 5 -5 + 6 6 six 1 -1 NULL 0 + 6 6 six 1 -1 NULL NULL + 6 6 six 2 2 0 NULL + 6 6 six 2 2 1 -1 + 6 6 six 2 2 2 2 + 6 6 six 2 2 2 4 + 6 6 six 2 2 3 -3 + 6 6 six 2 2 5 -5 + 6 6 six 2 2 5 -5 + 6 6 six 2 2 NULL 0 + 6 6 six 2 2 NULL NULL + 6 6 six 2 4 0 NULL + 6 6 six 2 4 1 -1 + 6 6 six 2 4 2 2 + 6 6 six 2 4 2 4 + 6 6 six 2 4 3 -3 + 6 6 six 2 4 5 -5 + 6 6 six 2 4 5 -5 + 6 6 six 2 4 NULL 0 + 6 6 six 2 4 NULL NULL + 6 6 six 3 -3 0 NULL + 6 6 six 3 -3 1 -1 + 6 6 six 3 -3 2 2 + 6 6 six 3 -3 2 4 + 6 6 six 3 -3 3 -3 + 6 6 six 3 -3 5 -5 + 6 6 six 3 -3 5 -5 + 6 6 six 3 -3 NULL 0 + 6 6 six 3 -3 NULL NULL + 6 6 six 5 -5 0 NULL + 6 6 six 5 -5 0 NULL + 6 6 six 5 -5 1 -1 + 6 6 six 5 -5 1 -1 + 6 6 six 5 -5 2 2 + 6 6 six 5 -5 2 2 + 6 6 six 5 -5 2 4 + 6 6 six 5 -5 2 4 + 6 6 six 5 -5 3 -3 + 6 6 six 5 -5 3 -3 + 6 6 six 5 -5 5 -5 + 6 6 six 5 -5 5 -5 + 6 6 six 5 -5 5 -5 + 6 6 six 5 -5 5 -5 + 6 6 six 5 -5 NULL 0 + 6 6 six 5 -5 NULL 0 + 6 6 six 5 -5 NULL NULL + 6 6 six 5 -5 NULL NULL + 6 6 six NULL 0 0 NULL + 6 6 six NULL 0 1 -1 + 6 6 six NULL 0 2 2 + 6 6 six NULL 0 2 4 + 6 6 six NULL 0 3 -3 + 6 6 six NULL 0 5 -5 + 6 6 six NULL 0 5 -5 + 6 6 six NULL 0 NULL 0 + 6 6 six NULL 0 NULL NULL + 6 6 six NULL NULL 0 NULL + 6 6 six NULL NULL 1 -1 + 6 6 six NULL NULL 2 2 + 6 6 six NULL NULL 2 4 + 6 6 six NULL NULL 3 -3 + 6 6 six NULL NULL 5 -5 + 6 6 six NULL NULL 5 -5 + 6 6 six NULL NULL NULL 0 + 6 6 six NULL NULL NULL NULL + 7 7 seven 0 NULL 0 NULL + 7 7 seven 0 NULL 1 -1 + 7 7 seven 0 NULL 2 2 + 7 7 seven 0 NULL 2 4 + 7 7 seven 0 NULL 3 -3 + 7 7 seven 0 NULL 5 -5 + 7 7 seven 0 NULL 5 -5 + 7 7 seven 0 NULL NULL 0 + 7 7 seven 0 NULL NULL NULL + 7 7 seven 1 -1 0 NULL + 7 7 seven 1 -1 1 -1 + 7 7 seven 1 -1 2 2 + 7 7 seven 1 -1 2 4 + 7 7 seven 1 -1 3 -3 + 7 7 seven 1 -1 5 -5 + 7 7 seven 1 -1 5 -5 + 7 7 seven 1 -1 NULL 0 + 7 7 seven 1 -1 NULL NULL + 7 7 seven 2 2 0 NULL + 7 7 seven 2 2 1 -1 + 7 7 seven 2 2 2 2 + 7 7 seven 2 2 2 4 + 7 7 seven 2 2 3 -3 + 7 7 seven 2 2 5 -5 + 7 7 seven 2 2 5 -5 + 7 7 seven 2 2 NULL 0 + 7 7 seven 2 2 NULL NULL + 7 7 seven 2 4 0 NULL + 7 7 seven 2 4 1 -1 + 7 7 seven 2 4 2 2 + 7 7 seven 2 4 2 4 + 7 7 seven 2 4 3 -3 + 7 7 seven 2 4 5 -5 + 7 7 seven 2 4 5 -5 + 7 7 seven 2 4 NULL 0 + 7 7 seven 2 4 NULL NULL + 7 7 seven 3 -3 0 NULL + 7 7 seven 3 -3 1 -1 + 7 7 seven 3 -3 2 2 + 7 7 seven 3 -3 2 4 + 7 7 seven 3 -3 3 -3 + 7 7 seven 3 -3 5 -5 + 7 7 seven 3 -3 5 -5 + 7 7 seven 3 -3 NULL 0 + 7 7 seven 3 -3 NULL NULL + 7 7 seven 5 -5 0 NULL + 7 7 seven 5 -5 0 NULL + 7 7 seven 5 -5 1 -1 + 7 7 seven 5 -5 1 -1 + 7 7 seven 5 -5 2 2 + 7 7 seven 5 -5 2 2 + 7 7 seven 5 -5 2 4 + 7 7 seven 5 -5 2 4 + 7 7 seven 5 -5 3 -3 + 7 7 seven 5 -5 3 -3 + 7 7 seven 5 -5 5 -5 + 7 7 seven 5 -5 5 -5 + 7 7 seven 5 -5 5 -5 + 7 7 seven 5 -5 5 -5 + 7 7 seven 5 -5 NULL 0 + 7 7 seven 5 -5 NULL 0 + 7 7 seven 5 -5 NULL NULL + 7 7 seven 5 -5 NULL NULL + 7 7 seven NULL 0 0 NULL + 7 7 seven NULL 0 1 -1 + 7 7 seven NULL 0 2 2 + 7 7 seven NULL 0 2 4 + 7 7 seven NULL 0 3 -3 + 7 7 seven NULL 0 5 -5 + 7 7 seven NULL 0 5 -5 + 7 7 seven NULL 0 NULL 0 + 7 7 seven NULL 0 NULL NULL + 7 7 seven NULL NULL 0 NULL + 7 7 seven NULL NULL 1 -1 + 7 7 seven NULL NULL 2 2 + 7 7 seven NULL NULL 2 4 + 7 7 seven NULL NULL 3 -3 + 7 7 seven NULL NULL 5 -5 + 7 7 seven NULL NULL 5 -5 + 7 7 seven NULL NULL NULL 0 + 7 7 seven NULL NULL NULL NULL + 8 8 eight 0 NULL 0 NULL + 8 8 eight 0 NULL 1 -1 + 8 8 eight 0 NULL 2 2 + 8 8 eight 0 NULL 2 4 + 8 8 eight 0 NULL 3 -3 + 8 8 eight 0 NULL 5 -5 + 8 8 eight 0 NULL 5 -5 + 8 8 eight 0 NULL NULL 0 + 8 8 eight 0 NULL NULL NULL + 8 8 eight 1 -1 0 NULL + 8 8 eight 1 -1 1 -1 + 8 8 eight 1 -1 2 2 + 8 8 eight 1 -1 2 4 + 8 8 eight 1 -1 3 -3 + 8 8 eight 1 -1 5 -5 + 8 8 eight 1 -1 5 -5 + 8 8 eight 1 -1 NULL 0 + 8 8 eight 1 -1 NULL NULL + 8 8 eight 2 2 0 NULL + 8 8 eight 2 2 1 -1 + 8 8 eight 2 2 2 2 + 8 8 eight 2 2 2 4 + 8 8 eight 2 2 3 -3 + 8 8 eight 2 2 5 -5 + 8 8 eight 2 2 5 -5 + 8 8 eight 2 2 NULL 0 + 8 8 eight 2 2 NULL NULL + 8 8 eight 2 4 0 NULL + 8 8 eight 2 4 1 -1 + 8 8 eight 2 4 2 2 + 8 8 eight 2 4 2 4 + 8 8 eight 2 4 3 -3 + 8 8 eight 2 4 5 -5 + 8 8 eight 2 4 5 -5 + 8 8 eight 2 4 NULL 0 + 8 8 eight 2 4 NULL NULL + 8 8 eight 3 -3 0 NULL + 8 8 eight 3 -3 1 -1 + 8 8 eight 3 -3 2 2 + 8 8 eight 3 -3 2 4 + 8 8 eight 3 -3 3 -3 + 8 8 eight 3 -3 5 -5 + 8 8 eight 3 -3 5 -5 + 8 8 eight 3 -3 NULL 0 + 8 8 eight 3 -3 NULL NULL + 8 8 eight 5 -5 0 NULL + 8 8 eight 5 -5 0 NULL + 8 8 eight 5 -5 1 -1 + 8 8 eight 5 -5 1 -1 + 8 8 eight 5 -5 2 2 + 8 8 eight 5 -5 2 2 + 8 8 eight 5 -5 2 4 + 8 8 eight 5 -5 2 4 + 8 8 eight 5 -5 3 -3 + 8 8 eight 5 -5 3 -3 + 8 8 eight 5 -5 5 -5 + 8 8 eight 5 -5 5 -5 + 8 8 eight 5 -5 5 -5 + 8 8 eight 5 -5 5 -5 + 8 8 eight 5 -5 NULL 0 + 8 8 eight 5 -5 NULL 0 + 8 8 eight 5 -5 NULL NULL + 8 8 eight 5 -5 NULL NULL + 8 8 eight NULL 0 0 NULL + 8 8 eight NULL 0 1 -1 + 8 8 eight NULL 0 2 2 + 8 8 eight NULL 0 2 4 + 8 8 eight NULL 0 3 -3 + 8 8 eight NULL 0 5 -5 + 8 8 eight NULL 0 5 -5 + 8 8 eight NULL 0 NULL 0 + 8 8 eight NULL 0 NULL NULL + 8 8 eight NULL NULL 0 NULL + 8 8 eight NULL NULL 1 -1 + 8 8 eight NULL NULL 2 2 + 8 8 eight NULL NULL 2 4 + 8 8 eight NULL NULL 3 -3 + 8 8 eight NULL NULL 5 -5 + 8 8 eight NULL NULL 5 -5 + 8 8 eight NULL NULL NULL 0 + 8 8 eight NULL NULL NULL NULL + NULL 0 zero 0 NULL 0 NULL + NULL 0 zero 0 NULL 1 -1 + NULL 0 zero 0 NULL 2 2 + NULL 0 zero 0 NULL 2 4 + NULL 0 zero 0 NULL 3 -3 + NULL 0 zero 0 NULL 5 -5 + NULL 0 zero 0 NULL 5 -5 + NULL 0 zero 0 NULL NULL 0 + NULL 0 zero 0 NULL NULL NULL + NULL 0 zero 1 -1 0 NULL + NULL 0 zero 1 -1 1 -1 + NULL 0 zero 1 -1 2 2 + NULL 0 zero 1 -1 2 4 + NULL 0 zero 1 -1 3 -3 + NULL 0 zero 1 -1 5 -5 + NULL 0 zero 1 -1 5 -5 + NULL 0 zero 1 -1 NULL 0 + NULL 0 zero 1 -1 NULL NULL + NULL 0 zero 2 2 0 NULL + NULL 0 zero 2 2 1 -1 + NULL 0 zero 2 2 2 2 + NULL 0 zero 2 2 2 4 + NULL 0 zero 2 2 3 -3 + NULL 0 zero 2 2 5 -5 + NULL 0 zero 2 2 5 -5 + NULL 0 zero 2 2 NULL 0 + NULL 0 zero 2 2 NULL NULL + NULL 0 zero 2 4 0 NULL + NULL 0 zero 2 4 1 -1 + NULL 0 zero 2 4 2 2 + NULL 0 zero 2 4 2 4 + NULL 0 zero 2 4 3 -3 + NULL 0 zero 2 4 5 -5 + NULL 0 zero 2 4 5 -5 + NULL 0 zero 2 4 NULL 0 + NULL 0 zero 2 4 NULL NULL + NULL 0 zero 3 -3 0 NULL + NULL 0 zero 3 -3 1 -1 + NULL 0 zero 3 -3 2 2 + NULL 0 zero 3 -3 2 4 + NULL 0 zero 3 -3 3 -3 + NULL 0 zero 3 -3 5 -5 + NULL 0 zero 3 -3 5 -5 + NULL 0 zero 3 -3 NULL 0 + NULL 0 zero 3 -3 NULL NULL + NULL 0 zero 5 -5 0 NULL + NULL 0 zero 5 -5 0 NULL + NULL 0 zero 5 -5 1 -1 + NULL 0 zero 5 -5 1 -1 + NULL 0 zero 5 -5 2 2 + NULL 0 zero 5 -5 2 2 + NULL 0 zero 5 -5 2 4 + NULL 0 zero 5 -5 2 4 + NULL 0 zero 5 -5 3 -3 + NULL 0 zero 5 -5 3 -3 + NULL 0 zero 5 -5 5 -5 + NULL 0 zero 5 -5 5 -5 + NULL 0 zero 5 -5 5 -5 + NULL 0 zero 5 -5 5 -5 + NULL 0 zero 5 -5 NULL 0 + NULL 0 zero 5 -5 NULL 0 + NULL 0 zero 5 -5 NULL NULL + NULL 0 zero 5 -5 NULL NULL + NULL 0 zero NULL 0 0 NULL + NULL 0 zero NULL 0 1 -1 + NULL 0 zero NULL 0 2 2 + NULL 0 zero NULL 0 2 4 + NULL 0 zero NULL 0 3 -3 + NULL 0 zero NULL 0 5 -5 + NULL 0 zero NULL 0 5 -5 + NULL 0 zero NULL 0 NULL 0 + NULL 0 zero NULL 0 NULL NULL + NULL 0 zero NULL NULL 0 NULL + NULL 0 zero NULL NULL 1 -1 + NULL 0 zero NULL NULL 2 2 + NULL 0 zero NULL NULL 2 4 + NULL 0 zero NULL NULL 3 -3 + NULL 0 zero NULL NULL 5 -5 + NULL 0 zero NULL NULL 5 -5 + NULL 0 zero NULL NULL NULL 0 + NULL 0 zero NULL NULL NULL NULL + NULL NULL null 0 NULL 0 NULL + NULL NULL null 0 NULL 1 -1 + NULL NULL null 0 NULL 2 2 + NULL NULL null 0 NULL 2 4 + NULL NULL null 0 NULL 3 -3 + NULL NULL null 0 NULL 5 -5 + NULL NULL null 0 NULL 5 -5 + NULL NULL null 0 NULL NULL 0 + NULL NULL null 0 NULL NULL NULL + NULL NULL null 1 -1 0 NULL + NULL NULL null 1 -1 1 -1 + NULL NULL null 1 -1 2 2 + NULL NULL null 1 -1 2 4 + NULL NULL null 1 -1 3 -3 + NULL NULL null 1 -1 5 -5 + NULL NULL null 1 -1 5 -5 + NULL NULL null 1 -1 NULL 0 + NULL NULL null 1 -1 NULL NULL + NULL NULL null 2 2 0 NULL + NULL NULL null 2 2 1 -1 + NULL NULL null 2 2 2 2 + NULL NULL null 2 2 2 4 + NULL NULL null 2 2 3 -3 + NULL NULL null 2 2 5 -5 + NULL NULL null 2 2 5 -5 + NULL NULL null 2 2 NULL 0 + NULL NULL null 2 2 NULL NULL + NULL NULL null 2 4 0 NULL + NULL NULL null 2 4 1 -1 + NULL NULL null 2 4 2 2 + NULL NULL null 2 4 2 4 + NULL NULL null 2 4 3 -3 + NULL NULL null 2 4 5 -5 + NULL NULL null 2 4 5 -5 + NULL NULL null 2 4 NULL 0 + NULL NULL null 2 4 NULL NULL + NULL NULL null 3 -3 0 NULL + NULL NULL null 3 -3 1 -1 + NULL NULL null 3 -3 2 2 + NULL NULL null 3 -3 2 4 + NULL NULL null 3 -3 3 -3 + NULL NULL null 3 -3 5 -5 + NULL NULL null 3 -3 5 -5 + NULL NULL null 3 -3 NULL 0 + NULL NULL null 3 -3 NULL NULL + NULL NULL null 5 -5 0 NULL + NULL NULL null 5 -5 0 NULL + NULL NULL null 5 -5 1 -1 + NULL NULL null 5 -5 1 -1 + NULL NULL null 5 -5 2 2 + NULL NULL null 5 -5 2 2 + NULL NULL null 5 -5 2 4 + NULL NULL null 5 -5 2 4 + NULL NULL null 5 -5 3 -3 + NULL NULL null 5 -5 3 -3 + NULL NULL null 5 -5 5 -5 + NULL NULL null 5 -5 5 -5 + NULL NULL null 5 -5 5 -5 + NULL NULL null 5 -5 5 -5 + NULL NULL null 5 -5 NULL 0 + NULL NULL null 5 -5 NULL 0 + NULL NULL null 5 -5 NULL NULL + NULL NULL null 5 -5 NULL NULL + NULL NULL null NULL 0 0 NULL + NULL NULL null NULL 0 1 -1 + NULL NULL null NULL 0 2 2 + NULL NULL null NULL 0 2 4 + NULL NULL null NULL 0 3 -3 + NULL NULL null NULL 0 5 -5 + NULL NULL null NULL 0 5 -5 + NULL NULL null NULL 0 NULL 0 + NULL NULL null NULL 0 NULL NULL + NULL NULL null NULL NULL 0 NULL + NULL NULL null NULL NULL 1 -1 + NULL NULL null NULL NULL 2 2 + NULL NULL null NULL NULL 2 4 + NULL NULL null NULL NULL 3 -3 + NULL NULL null NULL NULL 5 -5 + NULL NULL null NULL NULL 5 -5 + NULL NULL null NULL NULL NULL 0 + NULL NULL null NULL NULL NULL NULL + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL INNER JOIN J2_TBL USING (i) +-- !query schema +struct +-- !query output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL JOIN J2_TBL USING (i) +-- !query schema +struct +-- !query output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL t1 (a, b, c) JOIN J2_TBL t2 (a, d) USING (a) + ORDER BY a, d +-- !query schema +struct +-- !query output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL NATURAL JOIN J2_TBL +-- !query schema +struct +-- !query output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (a, d) +-- !query schema +struct +-- !query output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (d, a) +-- !query schema +struct +-- !query output + 0 NULL zero NULL + 2 3 two 2 + 4 1 four 2 + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = J2_TBL.i) +-- !query schema +struct +-- !query output + 0 NULL zero 0 NULL + 1 4 one 1 -1 + 2 3 two 2 2 + 2 3 two 2 4 + 3 2 three 3 -3 + 5 0 five 5 -5 + 5 0 five 5 -5 + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = J2_TBL.k) +-- !query schema +struct +-- !query output + 0 NULL zero NULL 0 + 2 3 two 2 2 + 4 1 four 2 4 + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i <= J2_TBL.k) +-- !query schema +struct +-- !query output + 0 NULL zero 2 2 + 0 NULL zero 2 4 + 0 NULL zero NULL 0 + 1 4 one 2 2 + 1 4 one 2 4 + 2 3 two 2 2 + 2 3 two 2 4 + 3 2 three 2 4 + 4 1 four 2 4 + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL LEFT OUTER JOIN J2_TBL USING (i) + ORDER BY i, k, t +-- !query schema +struct +-- !query output + NULL NULL null NULL + NULL 0 zero NULL + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 4 1 four NULL + 5 0 five -5 + 5 0 five -5 + 6 6 six NULL + 7 7 seven NULL + 8 8 eight NULL + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL LEFT JOIN J2_TBL USING (i) + ORDER BY i, k, t +-- !query schema +struct +-- !query output + NULL NULL null NULL + NULL 0 zero NULL + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 4 1 four NULL + 5 0 five -5 + 5 0 five -5 + 6 6 six NULL + 7 7 seven NULL + 8 8 eight NULL + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL RIGHT OUTER JOIN J2_TBL USING (i) +-- !query schema +struct +-- !query output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + NULL NULL NULL 0 + NULL NULL NULL NULL + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL RIGHT JOIN J2_TBL USING (i) +-- !query schema +struct +-- !query output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + NULL NULL NULL 0 + NULL NULL NULL NULL + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL FULL OUTER JOIN J2_TBL USING (i) + ORDER BY i, k, t +-- !query schema +struct +-- !query output + NULL NULL NULL NULL + NULL NULL null NULL + NULL 0 zero NULL + NULL NULL NULL 0 + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 4 1 four NULL + 5 0 five -5 + 5 0 five -5 + 6 6 six NULL + 7 7 seven NULL + 8 8 eight NULL + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL FULL JOIN J2_TBL USING (i) + ORDER BY i, k, t +-- !query schema +struct +-- !query output + NULL NULL NULL NULL + NULL NULL null NULL + NULL 0 zero NULL + NULL NULL NULL 0 + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 4 1 four NULL + 5 0 five -5 + 5 0 five -5 + 6 6 six NULL + 7 7 seven NULL + 8 8 eight NULL + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (k = 1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT '' AS `xxx`, * + FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (i = 1) +-- !query schema +struct +-- !query output + 1 4 one -1 + + +-- !query +CREATE TABLE t1 (name STRING, n INTEGER) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t2 (name STRING, n INTEGER) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t3 (name STRING, n INTEGER) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t1 VALUES ( 'bb', 11 ) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t2 VALUES ( 'bb', 12 ) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t2 VALUES ( 'cc', 22 ) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t2 VALUES ( 'ee', 42 ) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t3 VALUES ( 'bb', 13 ) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t3 VALUES ( 'cc', 23 ) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t3 VALUES ( 'dd', 33 ) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name) +-- !query schema +struct +-- !query output +bb 11 12 13 +cc NULL 22 23 +dd NULL NULL 33 +ee NULL 42 NULL + + +-- !query +SELECT * FROM +(SELECT * FROM t2) as s2 +INNER JOIN +(SELECT * FROM t3) s3 +USING (name) +-- !query schema +struct +-- !query output +bb 12 13 +cc 22 23 + + +-- !query +SELECT * FROM +(SELECT * FROM t2) as s2 +LEFT JOIN +(SELECT * FROM t3) s3 +USING (name) +-- !query schema +struct +-- !query output +bb 12 13 +cc 22 23 +ee 42 NULL + + +-- !query +SELECT * FROM +(SELECT * FROM t2) as s2 +FULL JOIN +(SELECT * FROM t3) s3 +USING (name) +-- !query schema +struct +-- !query output +bb 12 13 +cc 22 23 +dd NULL 33 +ee 42 NULL + + +-- !query +SELECT * FROM +(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL INNER JOIN +(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 +-- !query schema +struct +-- !query output +bb 12 2 13 3 +cc 22 2 23 3 + + +-- !query +SELECT * FROM +(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL LEFT JOIN +(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 +-- !query schema +struct +-- !query output +bb 12 2 13 3 +cc 22 2 23 3 +ee 42 2 NULL NULL + + +-- !query +SELECT * FROM +(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL FULL JOIN +(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 +-- !query schema +struct +-- !query output +bb 12 2 13 3 +cc 22 2 23 3 +dd NULL NULL 33 3 +ee 42 2 NULL NULL + + +-- !query +SELECT * FROM +(SELECT name, n as s1_n, 1 as s1_1 FROM t1) as s1 +NATURAL INNER JOIN +(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL INNER JOIN +(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 +-- !query schema +struct +-- !query output +bb 11 1 12 2 13 3 + + +-- !query +SELECT * FROM +(SELECT name, n as s1_n, 1 as s1_1 FROM t1) as s1 +NATURAL FULL JOIN +(SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL FULL JOIN +(SELECT name, n as s3_n, 3 as s3_2 FROM t3) s3 +-- !query schema +struct +-- !query output +bb 11 1 12 2 13 3 +cc NULL NULL 22 2 23 3 +dd NULL NULL NULL NULL 33 3 +ee NULL NULL 42 2 NULL NULL + + +-- !query +SELECT * FROM +(SELECT name, n as s1_n FROM t1) as s1 +NATURAL FULL JOIN + (SELECT * FROM + (SELECT name, n as s2_n FROM t2) as s2 + NATURAL FULL JOIN + (SELECT name, n as s3_n FROM t3) as s3 + ) ss2 +-- !query schema +struct +-- !query output +bb 11 12 13 +cc NULL 22 23 +dd NULL NULL 33 +ee NULL 42 NULL + + +-- !query +SELECT * FROM +(SELECT name, n as s1_n FROM t1) as s1 +NATURAL FULL JOIN + (SELECT * FROM + (SELECT name, n as s2_n, 2 as s2_2 FROM t2) as s2 + NATURAL FULL JOIN + (SELECT name, n as s3_n FROM t3) as s3 + ) ss2 +-- !query schema +struct +-- !query output +bb 11 12 2 13 +cc NULL 22 2 23 +dd NULL NULL NULL 33 +ee NULL 42 2 NULL + + +-- !query +SELECT * FROM + (SELECT name, n as s1_n FROM t1) as s1 +FULL JOIN + (SELECT name, 2 as s2_n FROM t2) as s2 +ON (s1_n = s2_n) +-- !query schema +struct +-- !query output +NULL NULL bb 2 +NULL NULL cc 2 +NULL NULL ee 2 +bb 11 NULL NULL + + +-- !query +create or replace temporary view x as select * from + (values (1,11), (2,22), (3,null), (4,44), (5,null)) + as v(x1, x2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view y as select * from + (values (1,111), (2,222), (3,333), (4,null)) + as v(y1, y2) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from x +-- !query schema +struct +-- !query output +1 11 +2 22 +3 NULL +4 44 +5 NULL + + +-- !query +select * from y +-- !query schema +struct +-- !query output +1 111 +2 222 +3 333 +4 NULL + + +-- !query +select * from x left join y on (x1 = y1 and x2 is not null) +-- !query schema +struct +-- !query output +1 11 1 111 +2 22 2 222 +3 NULL NULL NULL +4 44 4 NULL +5 NULL NULL NULL + + +-- !query +select * from x left join y on (x1 = y1 and y2 is not null) +-- !query schema +struct +-- !query output +1 11 1 111 +2 22 2 222 +3 NULL 3 333 +4 44 NULL NULL +5 NULL NULL NULL + + +-- !query +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1) +-- !query schema +struct +-- !query output +1 11 1 111 1 11 +2 22 2 222 2 22 +3 NULL 3 333 3 NULL +4 44 4 NULL 4 44 +5 NULL NULL NULL 5 NULL + + +-- !query +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1 and x2 is not null) +-- !query schema +struct +-- !query output +1 11 1 111 1 11 +2 22 2 222 2 22 +3 NULL 3 333 NULL NULL +4 44 4 NULL 4 44 +5 NULL NULL NULL NULL NULL + + +-- !query +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1 and y2 is not null) +-- !query schema +struct +-- !query output +1 11 1 111 1 11 +2 22 2 222 2 22 +3 NULL 3 333 3 NULL +4 44 4 NULL NULL NULL +5 NULL NULL NULL NULL NULL + + +-- !query +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1 and xx2 is not null) +-- !query schema +struct +-- !query output +1 11 1 111 1 11 +2 22 2 222 2 22 +3 NULL 3 333 NULL NULL +4 44 4 NULL 4 44 +5 NULL NULL NULL NULL NULL + + +-- !query +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1) where (x2 is not null) +-- !query schema +struct +-- !query output +1 11 1 111 1 11 +2 22 2 222 2 22 +4 44 4 NULL 4 44 + + +-- !query +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1) where (y2 is not null) +-- !query schema +struct +-- !query output +1 11 1 111 1 11 +2 22 2 222 2 22 +3 NULL 3 333 3 NULL + + +-- !query +select * from (x left join y on (x1 = y1)) left join x xx(xx1,xx2) +on (x1 = xx1) where (xx2 is not null) +-- !query schema +struct +-- !query output +1 11 1 111 1 11 +2 22 2 222 2 22 +4 44 4 NULL 4 44 + + +-- !query +select count(*) from tenk1 a where unique1 in + (select unique1 from tenk1 b join tenk1 c using (unique1) + where b.unique2 = 42) +-- !query schema +struct +-- !query output +1 + + +-- !query +select count(*) from tenk1 x where + x.unique1 in (select a.f1 from int4_tbl a,float8_tbl b where a.f1=b.f1) and + x.unique1 = 0 and + x.unique1 in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=bb.f1) +-- !query schema +struct +-- !query output +1 + + +-- !query +select count(*) from tenk1 x where + x.unique1 in (select a.f1 from int4_tbl a,float8_tbl b where a.f1=b.f1) and + x.unique1 = 0 and + x.unique1 in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=bb.f1) +-- !query schema +struct +-- !query output +1 + + +-- !query +select * from int8_tbl i1 left join (int8_tbl i2 join + (select 123 as x) ss on i2.q1 = x) on i1.q2 = i2.q2 +order by 1, 2 +-- !query schema +struct +-- !query output +123 456 123 456 123 +123 4567890123456789 123 4567890123456789 123 +4567890123456789 -4567890123456789 NULL NULL NULL +4567890123456789 123 NULL NULL NULL +4567890123456789 4567890123456789 123 4567890123456789 123 + + +-- !query +select count(*) +from + (select t3.tenthous as x1, coalesce(t1.stringu1, t2.stringu1) as x2 + from tenk1 t1 + left join tenk1 t2 on t1.unique1 = t2.unique1 + join tenk1 t3 on t1.unique2 = t3.unique2) ss, + tenk1 t4, + tenk1 t5 +where t4.thousand = t5.unique1 and ss.x1 = t4.tenthous and ss.x2 = t5.stringu1 +-- !query schema +struct +-- !query output +1000 + + +-- !query +select a.f1, b.f1, t.thousand, t.tenthous from + tenk1 t, + (select sum(f1)+1 as f1 from int4_tbl i4a) a, + (select sum(f1) as f1 from int4_tbl i4b) b +where b.f1 = t.thousand and a.f1 = b.f1 and (a.f1+b.f1+999) = t.tenthous +-- !query schema +struct +-- !query output + + + +-- !query +select * from + j1_tbl full join + (select * from j2_tbl order by j2_tbl.i desc, j2_tbl.k asc) j2_tbl + on j1_tbl.i = j2_tbl.i and j1_tbl.i = j2_tbl.k +-- !query schema +struct +-- !query output +0 NULL zero NULL NULL +1 4 one NULL NULL +2 3 two 2 2 +3 2 three NULL NULL +4 1 four NULL NULL +5 0 five NULL NULL +6 6 six NULL NULL +7 7 seven NULL NULL +8 8 eight NULL NULL +NULL 0 zero NULL NULL +NULL NULL NULL 0 NULL +NULL NULL NULL 1 -1 +NULL NULL NULL 2 4 +NULL NULL NULL 3 -3 +NULL NULL NULL 5 -5 +NULL NULL NULL 5 -5 +NULL NULL NULL NULL 0 +NULL NULL NULL NULL NULL +NULL NULL null NULL NULL + + +-- !query +select count(*) from + (select * from tenk1 x order by x.thousand, x.twothousand, x.fivethous) x + left join + (select * from tenk1 y order by y.unique2) y + on x.thousand = y.unique2 and x.twothousand = y.hundred and x.fivethous = y.unique2 +-- !query schema +struct +-- !query output +10000 + + +-- !query +DROP TABLE t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE t3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE J1_TBL +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE J2_TBL +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view tt1 as select * from + (values (1, 11), (2, NULL)) + as v(tt1_id, joincol) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view tt2 as select * from + (values (21, 11), (22, 11)) + as v(tt2_id, joincol) +-- !query schema +struct<> +-- !query output + + + +-- !query +select tt1.*, tt2.* from tt1 left join tt2 on tt1.joincol = tt2.joincol +-- !query schema +struct +-- !query output +1 11 21 11 +1 11 22 11 +2 NULL NULL NULL + + +-- !query +select tt1.*, tt2.* from tt2 right join tt1 on tt1.joincol = tt2.joincol +-- !query schema +struct +-- !query output +1 11 21 11 +1 11 22 11 +2 NULL NULL NULL + + +-- !query +select count(*) from tenk1 a, tenk1 b + where a.hundred = b.thousand and (b.fivethous % 10) < 10 +-- !query schema +struct +-- !query output +100000 + + +-- !query +create or replace temporary view tt3 as select * from + (SELECT cast(x.id as int), repeat('xyzzy', 100) FROM range(1,10001) x) + as v(f1, f2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view tt4 as select * from + (values (0), (1), (9999)) + as v(f1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT a.f1 +FROM tt4 a +LEFT JOIN ( + SELECT b.f1 + FROM tt3 b LEFT JOIN tt3 c ON (b.f1 = c.f1) + WHERE c.f1 IS NULL +) AS d ON (a.f1 = d.f1) +WHERE d.f1 IS NULL +-- !query schema +struct +-- !query output +0 +1 +9999 + + +-- !query +create or replace temporary view tt5 as select * from + (values (1, 10), (1, 11)) + as v(f1, f2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view tt6 as select * from + (values (1, 9), (1, 2), (2, 9)) + as v(f1, f2) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from tt5,tt6 where tt5.f1 = tt6.f1 and tt5.f1 = tt5.f2 - tt6.f2 +-- !query schema +struct +-- !query output +1 10 1 9 + + +-- !query +create or replace temporary view xx as select * from + (values (1), (2), (3)) + as v(pkxx) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view yy as select * from + (values (101, 1), (201, 2), (301, NULL)) + as v(pkyy, pkxx) +-- !query schema +struct<> +-- !query output + + + +-- !query +select yy.pkyy as yy_pkyy, yy.pkxx as yy_pkxx, yya.pkyy as yya_pkyy, + xxa.pkxx as xxa_pkxx, xxb.pkxx as xxb_pkxx +from yy + left join (SELECT * FROM yy where pkyy = 101) as yya ON yy.pkyy = yya.pkyy + left join xx xxa on yya.pkxx = xxa.pkxx + left join xx xxb on coalesce (xxa.pkxx, 1) = xxb.pkxx +-- !query schema +struct +-- !query output +101 1 101 1 1 +201 2 NULL NULL 1 +301 NULL NULL NULL 1 + + +-- !query +create or replace temporary view zt1 as select * from + (values (53)) + as v(f1) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view zt2 as select * from + (values (53)) + as v(f2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view zt3(f3 int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from + zt2 left join zt3 on (f2 = f3) + left join zt1 on (f3 = f1) +where f2 = 53 +-- !query schema +struct +-- !query output +53 NULL NULL + + +-- !query +create temp view zv1 as select *,'dummy' AS junk from zt1 +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from + zt2 left join zt3 on (f2 = f3) + left join zv1 on (f3 = f1) +where f2 = 53 +-- !query schema +struct +-- !query output +53 NULL NULL NULL + + +-- !query +select a.unique2, a.ten, b.tenthous, b.unique2, b.hundred +from tenk1 a left join tenk1 b on a.unique2 = b.tenthous +where a.unique1 = 42 and + ((b.unique2 is null and a.ten = 2) or b.hundred = 3) +-- !query schema +struct +-- !query output + + + +-- !query +create or replace temporary view a (i integer) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view b (x integer, y integer) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from a left join b on i = x and i = y and x = i +-- !query schema +struct +-- !query output + + + +-- !query +select t1.q2, count(t2.q1, t2.q2) +from int8_tbl t1 left join int8_tbl t2 on (t1.q2 = t2.q1) +group by t1.q2 order by 1 +-- !query schema +struct +-- !query output +-4567890123456789 0 +123 2 +456 0 +4567890123456789 6 + + +-- !query +select t1.q2, count(t2.q1, t2.q2) +from int8_tbl t1 left join (select * from int8_tbl) t2 on (t1.q2 = t2.q1) +group by t1.q2 order by 1 +-- !query schema +struct +-- !query output +-4567890123456789 0 +123 2 +456 0 +4567890123456789 6 + + +-- !query +select t1.q2, count(t2.q1, t2.q2) +from int8_tbl t1 left join + (select q1, case when q2=1 then 1 else q2 end as q2 from int8_tbl) t2 + on (t1.q2 = t2.q1) +group by t1.q2 order by 1 +-- !query schema +struct +-- !query output +-4567890123456789 0 +123 2 +456 0 +4567890123456789 6 + + +-- !query +create or replace temporary view a as select * from + (values ('p'), ('q')) + as v(code) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view b as select * from + (values ('p', 1), ('p', 2)) + as v(a, num) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view c as select * from + (values ('A', 'p'), ('B', 'q'), ('C', null)) + as v(name, a) +-- !query schema +struct<> +-- !query output + + + +-- !query +select c.name, ss.code, ss.b_cnt, ss.const +from c left join + (select a.code, coalesce(b_grp.cnt, 0) as b_cnt, -1 as const + from a left join + (select count(1) as cnt, b.a from b group by b.a) as b_grp + on a.code = b_grp.a + ) as ss + on (c.a = ss.code) +order by c.name +-- !query schema +struct +-- !query output +A p 2 -1 +B q 0 -1 +C NULL NULL NULL + + +-- !query +SELECT * FROM +( SELECT 1 as key1 ) sub1 +LEFT JOIN +( SELECT sub3.key3, sub4.value2, COALESCE(sub4.value2, 66) as value3 FROM + ( SELECT 1 as key3 ) sub3 + LEFT JOIN + ( SELECT sub5.key5, COALESCE(sub6.value1, 1) as value2 FROM + ( SELECT 1 as key5 ) sub5 + LEFT JOIN + ( SELECT 2 as key6, 42 as value1 ) sub6 + ON sub5.key5 = sub6.key6 + ) sub4 + ON sub4.key5 = sub3.key3 +) sub2 +ON sub1.key1 = sub2.key3 +-- !query schema +struct +-- !query output +1 1 1 1 + + +-- !query +SELECT * FROM +( SELECT 1 as key1 ) sub1 +LEFT JOIN +( SELECT sub3.key3, value2, COALESCE(value2, 66) as value3 FROM + ( SELECT 1 as key3 ) sub3 + LEFT JOIN + ( SELECT sub5.key5, COALESCE(sub6.value1, 1) as value2 FROM + ( SELECT 1 as key5 ) sub5 + LEFT JOIN + ( SELECT 2 as key6, 42 as value1 ) sub6 + ON sub5.key5 = sub6.key6 + ) sub4 + ON sub4.key5 = sub3.key3 +) sub2 +ON sub1.key1 = sub2.key3 +-- !query schema +struct +-- !query output +1 1 1 1 + + +-- !query +SELECT qq, unique1 + FROM + ( SELECT COALESCE(q1, 0) AS qq FROM int8_tbl a ) AS ss1 + FULL OUTER JOIN + ( SELECT COALESCE(q2, -1) AS qq FROM int8_tbl b ) AS ss2 + USING (qq) + INNER JOIN tenk1 c ON qq = unique2 +-- !query schema +struct +-- !query output +123 4596 +123 4596 +456 7318 + + +-- !query +create or replace temporary view nt1 as select * from + (values(1,true,true), (2,true,false), (3,false,false)) + as v(id, a1, a2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view nt2 as select * from + (values(1,1,true,true), (2,2,true,false), (3,3,false,false)) + as v(id, nt1_id, b1, b2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view nt3 as select * from + (values(1,1,true), (2,2,false), (3,3,true)) + as v(id, nt2_id, c1) +-- !query schema +struct<> +-- !query output + + + +-- !query +select nt3.id +from nt3 as nt3 + left join + (select nt2.*, (nt2.b1 and ss1.a3) AS b3 + from nt2 as nt2 + left join + (select nt1.*, (nt1.id is not null) as a3 from nt1) as ss1 + on ss1.id = nt2.nt1_id + ) as ss2 + on ss2.id = nt3.nt2_id +where nt3.id = 1 and ss2.b3 +-- !query schema +struct +-- !query output +1 + + +-- !query +select * from int4_tbl a full join int4_tbl b on true +-- !query schema +struct +-- !query output +-123456 -123456 +-123456 -2147483647 +-123456 0 +-123456 123456 +-123456 2147483647 +-2147483647 -123456 +-2147483647 -2147483647 +-2147483647 0 +-2147483647 123456 +-2147483647 2147483647 +0 -123456 +0 -2147483647 +0 0 +0 123456 +0 2147483647 +123456 -123456 +123456 -2147483647 +123456 0 +123456 123456 +123456 2147483647 +2147483647 -123456 +2147483647 -2147483647 +2147483647 0 +2147483647 123456 +2147483647 2147483647 + + +-- !query +select * from int4_tbl a full join int4_tbl b on false +-- !query schema +struct +-- !query output +-123456 NULL +-2147483647 NULL +0 NULL +123456 NULL +2147483647 NULL +NULL -123456 +NULL -2147483647 +NULL 0 +NULL 123456 +NULL 2147483647 + + +-- !query +select count(*) from + tenk1 a join tenk1 b on a.unique1 = b.unique2 + left join tenk1 c on a.unique2 = b.unique1 and c.thousand = a.thousand + join int4_tbl on b.thousand = f1 +-- !query schema +struct +-- !query output +10 + + +-- !query +select b.unique1 from + tenk1 a join tenk1 b on a.unique1 = b.unique2 + left join tenk1 c on b.unique1 = 42 and c.thousand = a.thousand + join int4_tbl i1 on b.thousand = f1 + right join int4_tbl i2 on i2.f1 = b.tenthous + order by 1 +-- !query schema +struct +-- !query output +NULL +NULL +NULL +NULL +0 + + +-- !query +select * from +( + select unique1, q1, coalesce(unique1, -1) + q1 as fault + from int8_tbl left join tenk1 on (q2 = unique2) +) ss +where fault = 122 +order by fault +-- !query schema +struct +-- !query output +NULL 123 122 + + +-- !query +select q1, unique2, thousand, hundred + from int8_tbl a left join tenk1 b on q1 = unique2 + where coalesce(thousand,123) = q1 and q1 = coalesce(hundred,123) +-- !query schema +struct +-- !query output + + + +-- !query +select f1, unique2, case when unique2 is null then f1 else 0 end + from int4_tbl a left join tenk1 b on f1 = unique2 + where (case when unique2 is null then f1 else 0 end) = 0 +-- !query schema +struct +-- !query output +0 0 0 + + +-- !query +select a.unique1, b.unique1, c.unique1, coalesce(b.twothousand, a.twothousand) + from tenk1 a left join tenk1 b on b.thousand = a.unique1 left join tenk1 c on c.unique2 = coalesce(b.twothousand, a.twothousand) + where a.unique2 < 10 and coalesce(b.twothousand, a.twothousand) = 44 +-- !query schema +struct +-- !query output + + + +-- !query +select * from + text_tbl t1 + inner join int8_tbl i8 + on i8.q2 = 456 + right join text_tbl t2 + on t1.f1 = 'doh!' + left join int4_tbl i4 + on i8.q1 = i4.f1 +-- !query schema +struct +-- !query output +doh! 123 456 doh! NULL +doh! 123 456 hi de ho neighbor NULL + + +-- !query +select * from + text_tbl t1 + left join int8_tbl i8 + on i8.q2 = 123, + lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss +where t1.f1 = ss.f1 +-- !query schema +struct +-- !query output +doh! 4567890123456789 123 4567890123456789 doh! + + +-- !query +select * from + text_tbl t1 + left join int8_tbl i8 + on i8.q2 = 123, + lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss +where t1.f1 = ss.f1 +-- !query schema +struct +-- !query output +doh! 4567890123456789 123 4567890123456789 doh! + + +-- !query +select * from + text_tbl t1 + left join int8_tbl i8 + on i8.q2 = 123, + lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss1, + lateral (select ss1.* from text_tbl t3 limit 1) as ss2 +where t1.f1 = ss2.f1 +-- !query schema +struct +-- !query output +doh! 4567890123456789 123 4567890123456789 doh! 4567890123456789 doh! + + +-- !query +select * from + text_tbl t1 + left join int8_tbl i8 + on i8.q2 = 123, + lateral (select i8.q1, t2.f1 from text_tbl t2 limit 1) as ss1, + lateral (select ss1.* from text_tbl t3 limit 1) as ss2 +where t1.f1 = ss2.f1 +-- !query schema +struct +-- !query output +doh! 4567890123456789 123 4567890123456789 doh! 4567890123456789 doh! + + +-- !query +select 1 from + text_tbl as tt1 + inner join text_tbl as tt2 on (tt1.f1 = 'foo') + left join text_tbl as tt3 on (tt3.f1 = 'foo') + left join text_tbl as tt4 on (tt3.f1 = tt4.f1), + lateral (select tt4.f1 as c0 from text_tbl as tt5 limit 1) as ss1 +where tt1.f1 = ss1.c0 +-- !query schema +struct<1:int> +-- !query output + + + +-- !query +select 1 from + text_tbl as tt1 + inner join text_tbl as tt2 on (tt1.f1 = 'foo') + left join text_tbl as tt3 on (tt3.f1 = 'foo') + left join text_tbl as tt4 on (tt3.f1 = tt4.f1), + lateral (select tt4.f1 as c0 from text_tbl as tt5 limit 1) as ss1 +where tt1.f1 = ss1.c0 +-- !query schema +struct<1:int> +-- !query output + + + +-- !query +select * from + (select 1 as id) as xx + left join + (tenk1 as a1 full join (select 1 as id) as yy on (a1.unique1 = yy.id)) + on (xx.id = coalesce(yy.id)) +-- !query schema +struct +-- !query output +1 1 2838 1 1 1 1 1 1 1 1 1 2 3 BAAAAA EFEAAA OOOOxx 1 + + +-- !query +select a.q2, b.q1 + from int8_tbl a left join int8_tbl b on a.q2 = coalesce(b.q1, 1) + where coalesce(b.q1, 1) > 0 +-- !query schema +struct +-- !query output +-4567890123456789 NULL +123 123 +123 123 +456 NULL +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query +create or replace temporary view parent as select * from + (values (1, 10), (2, 20), (3, 30)) + as v(k, pd) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view child as select * from + (values (1, 100), (4, 400)) + as v(k, cd) +-- !query schema +struct<> +-- !query output + + + +-- !query +select p.* from parent p left join child c on (p.k = c.k) +-- !query schema +struct +-- !query output +1 10 +2 20 +3 30 + + +-- !query +select p.*, linked from parent p + left join (select c.*, true as linked from child c) as ss + on (p.k = ss.k) +-- !query schema +struct +-- !query output +1 10 true +2 20 NULL +3 30 NULL + + +-- !query +select p.* from + parent p left join child c on (p.k = c.k) + where p.k = 1 and p.k = 2 +-- !query schema +struct +-- !query output + + + +-- !query +select p.* from + (parent p left join child c on (p.k = c.k)) join parent x on p.k = x.k + where p.k = 1 and p.k = 2 +-- !query schema +struct +-- !query output + + + +-- !query +create or replace temporary view a as select * from + (values (0), (1)) + as v(id) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view b as select * from + (values (0, 0), (1, NULL)) + as v(id, a_id) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM b LEFT JOIN a ON (b.a_id = a.id) WHERE (a.id IS NULL OR a.id > 0) +-- !query schema +struct +-- !query output +1 NULL NULL + + +-- !query +SELECT b.* FROM b LEFT JOIN a ON (b.a_id = a.id) WHERE (a.id IS NULL OR a.id > 0) +-- !query schema +struct +-- !query output +1 NULL + + +-- !query +create or replace temporary view innertab as select * from + (values (123L, 42L)) + as v(id, dat1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM + (SELECT 1 AS x) ss1 + LEFT JOIN + (SELECT q1, q2, COALESCE(dat1, q1) AS y + FROM int8_tbl LEFT JOIN innertab ON q2 = id) ss2 + ON true +-- !query schema +struct +-- !query output +1 123 456 123 +1 123 4567890123456789 123 +1 4567890123456789 -4567890123456789 4567890123456789 +1 4567890123456789 123 42 +1 4567890123456789 4567890123456789 4567890123456789 + + +-- !query +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = f1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`f1`", + "referenceNames" : "[`j`.`f1`, `j`.`f1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 78, + "stopIndex" : 79, + "fragment" : "f1" + } ] +} + + +-- !query +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y) j on q1 = y.f1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`y`.`f1`", + "proposal" : "`j`.`f1`, `j`.`f1`, `x`.`q1`, `x`.`q2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 78, + "stopIndex" : 81, + "fragment" : "y.f1" + } ] +} + + +-- !query +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on q1 = f1 +-- !query schema +struct +-- !query output + + + +-- !query +select t1.uunique1 from + tenk1 t1 join tenk2 t2 on t1.two = t2.two +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t1`.`uunique1`", + "proposal" : "`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`four`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "t1.uunique1" + } ] +} + + +-- !query +select t2.uunique1 from + tenk1 t1 join tenk2 t2 on t1.two = t2.two +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t2`.`uunique1`", + "proposal" : "`t2`.`unique1`, `t1`.`unique1`, `t2`.`unique2`, `t1`.`unique2`, `t2`.`four`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "t2.uunique1" + } ] +} + + +-- !query +select uunique1 from + tenk1 t1 join tenk2 t2 on t1.two = t2.two +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`uunique1`", + "proposal" : "`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`ten`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "uunique1" + } ] +} + + +-- !query +select unique2, x.* +from tenk1 a, lateral (select * from int4_tbl b where f1 = a.unique1) x +-- !query schema +struct +-- !query output +9998 0 + + +-- !query +select unique2, x.* +from int4_tbl x, lateral (select unique2 from tenk1 where f1 = unique1) ss +-- !query schema +struct +-- !query output +9998 0 + + +-- !query +select unique2, x.* +from int4_tbl x left join lateral (select unique1, unique2 from tenk1 where f1 = unique1) ss on true +-- !query schema +struct +-- !query output +9998 0 +NULL -123456 +NULL -2147483647 +NULL 123456 +NULL 2147483647 + + +-- !query +select * from (select f1/2 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1, + lateral (select x) ss2(y) +-- !query schema +struct +-- !query output +0.0 0 0.0 + + +-- !query +select * from ((select f1/2 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1) j, + lateral (select x) ss2(y) +-- !query schema +struct +-- !query output +0.0 0 0.0 + + +-- !query +select * from + int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1, + lateral (select x.q1,y.q1,y.q2) v(xq1,yq1,yq2) +-- !query schema +struct +-- !query output +123 456 NULL NULL 123 NULL NULL +123 4567890123456789 4567890123456789 -4567890123456789 123 4567890123456789 -4567890123456789 +123 4567890123456789 4567890123456789 123 123 4567890123456789 123 +123 4567890123456789 4567890123456789 4567890123456789 123 4567890123456789 4567890123456789 +4567890123456789 -4567890123456789 NULL NULL 4567890123456789 NULL NULL +4567890123456789 123 123 456 4567890123456789 123 456 +4567890123456789 123 123 4567890123456789 4567890123456789 123 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 -4567890123456789 4567890123456789 4567890123456789 -4567890123456789 +4567890123456789 4567890123456789 4567890123456789 123 4567890123456789 4567890123456789 123 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 + + +-- !query +select x.* from + int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1, + lateral (select x.q1,y.q1,y.q2) v(xq1,yq1,yq2) +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 +123 4567890123456789 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 123 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query +select * from + int8_tbl a left join + lateral (select *, a.q2 as x from int8_tbl b) ss on a.q2 = ss.q1 +-- !query schema +struct +-- !query output +123 456 NULL NULL NULL +123 4567890123456789 4567890123456789 -4567890123456789 4567890123456789 +123 4567890123456789 4567890123456789 123 4567890123456789 +123 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 -4567890123456789 NULL NULL NULL +4567890123456789 123 123 456 123 +4567890123456789 123 123 4567890123456789 123 +4567890123456789 4567890123456789 4567890123456789 -4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 123 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 + + +-- !query +select * from + int8_tbl a left join + lateral (select *, coalesce(a.q2, 42) as x from int8_tbl b) ss on a.q2 = ss.q1 +-- !query schema +struct +-- !query output +123 456 NULL NULL NULL +123 4567890123456789 4567890123456789 -4567890123456789 4567890123456789 +123 4567890123456789 4567890123456789 123 4567890123456789 +123 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 -4567890123456789 NULL NULL NULL +4567890123456789 123 123 456 123 +4567890123456789 123 123 4567890123456789 123 +4567890123456789 4567890123456789 4567890123456789 -4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 123 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 + + +-- !query +select * from int4_tbl i left join + lateral (select * from int2_tbl j where i.f1 = j.f1) k on true +-- !query schema +struct +-- !query output +-123456 NULL +-2147483647 NULL +0 0 +123456 NULL +2147483647 NULL + + +-- !query +select * from + int8_tbl a left join lateral + (select b.q1 as bq1, c.q1 as cq1, least(a.q1,b.q1,c.q1) from + int8_tbl b cross join int8_tbl c) ss + on a.q2 = ss.bq1 +-- !query schema +struct +-- !query output +123 456 NULL NULL NULL +123 4567890123456789 4567890123456789 123 123 +123 4567890123456789 4567890123456789 123 123 +123 4567890123456789 4567890123456789 123 123 +123 4567890123456789 4567890123456789 123 123 +123 4567890123456789 4567890123456789 123 123 +123 4567890123456789 4567890123456789 123 123 +123 4567890123456789 4567890123456789 4567890123456789 123 +123 4567890123456789 4567890123456789 4567890123456789 123 +123 4567890123456789 4567890123456789 4567890123456789 123 +123 4567890123456789 4567890123456789 4567890123456789 123 +123 4567890123456789 4567890123456789 4567890123456789 123 +123 4567890123456789 4567890123456789 4567890123456789 123 +123 4567890123456789 4567890123456789 4567890123456789 123 +123 4567890123456789 4567890123456789 4567890123456789 123 +123 4567890123456789 4567890123456789 4567890123456789 123 +4567890123456789 -4567890123456789 NULL NULL NULL +4567890123456789 123 123 123 123 +4567890123456789 123 123 123 123 +4567890123456789 123 123 123 123 +4567890123456789 123 123 123 123 +4567890123456789 123 123 4567890123456789 123 +4567890123456789 123 123 4567890123456789 123 +4567890123456789 123 123 4567890123456789 123 +4567890123456789 123 123 4567890123456789 123 +4567890123456789 123 123 4567890123456789 123 +4567890123456789 123 123 4567890123456789 123 +4567890123456789 4567890123456789 4567890123456789 123 123 +4567890123456789 4567890123456789 4567890123456789 123 123 +4567890123456789 4567890123456789 4567890123456789 123 123 +4567890123456789 4567890123456789 4567890123456789 123 123 +4567890123456789 4567890123456789 4567890123456789 123 123 +4567890123456789 4567890123456789 4567890123456789 123 123 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 + + +-- !query +select f1,g from int4_tbl a, (select f1 as g) ss +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`f1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 38, + "stopIndex" : 39, + "fragment" : "f1" + } ] +} + + +-- !query +select f1,g from int4_tbl a, (select a.f1 as g) ss +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`.`f1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 38, + "stopIndex" : 41, + "fragment" : "a.f1" + } ] +} + + +-- !query +select f1,g from int4_tbl a cross join (select f1 as g) ss +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`f1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 48, + "stopIndex" : 49, + "fragment" : "f1" + } ] +} + + +-- !query +select f1,g from int4_tbl a cross join (select a.f1 as g) ss +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`.`f1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 48, + "stopIndex" : 51, + "fragment" : "a.f1" + } ] +} + + +-- !query +CREATE TABLE j1 (id1 int, id2 int) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE j2 (id1 int, id2 int) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO j1 values(1,1),(1,2) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO j2 values(1,1) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO j2 values(1,2) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from j1 +inner join j2 on j1.id1 = j2.id1 and j1.id2 = j2.id2 +where j1.id1 % 1000 = 1 and j2.id1 % 1000 = 1 +-- !query schema +struct +-- !query output +1 1 1 1 +1 2 1 2 + + +-- !query +drop table j1 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table j2 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/limit.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/limit.sql.out new file mode 100644 index 000000000000..1addd56b8f0e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/limit.sql.out @@ -0,0 +1,190 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT '' AS two, unique1, unique2, stringu1 + FROM onek WHERE unique1 > 50 + ORDER BY unique1 LIMIT 2 +-- !query schema +struct +-- !query output + 51 76 ZBAAAA + 52 985 ACAAAA + + +-- !query +SELECT '' AS five, unique1, unique2, stringu1 + FROM onek WHERE unique1 > 60 + ORDER BY unique1 LIMIT 5 +-- !query schema +struct +-- !query output + 61 560 JCAAAA + 62 633 KCAAAA + 63 296 LCAAAA + 64 479 MCAAAA + 65 64 NCAAAA + + +-- !query +SELECT '' AS two, unique1, unique2, stringu1 + FROM onek WHERE unique1 > 60 AND unique1 < 63 + ORDER BY unique1 LIMIT 5 +-- !query schema +struct +-- !query output + 61 560 JCAAAA + 62 633 KCAAAA + + +-- !query +SELECT '' AS three, unique1, unique2, stringu1 + FROM onek WHERE unique1 > 100 + ORDER BY unique1 LIMIT 3 OFFSET 20 +-- !query schema +struct +-- !query output + 121 700 REAAAA + 122 519 SEAAAA + 123 777 TEAAAA + + +-- !query +SELECT '' AS zero, unique1, unique2, stringu1 + FROM onek WHERE unique1 < 50 + ORDER BY unique1 DESC LIMIT 8 OFFSET 99 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT '' AS eleven, unique1, unique2, stringu1 + FROM onek WHERE unique1 < 50 + ORDER BY unique1 DESC LIMIT 20 OFFSET 39 +-- !query schema +struct +-- !query output + 10 520 KAAAAA + 9 49 JAAAAA + 8 653 IAAAAA + 7 647 HAAAAA + 6 978 GAAAAA + 5 541 FAAAAA + 4 833 EAAAAA + 3 431 DAAAAA + 2 326 CAAAAA + 1 214 BAAAAA + 0 998 AAAAAA + + +-- !query +SELECT '' AS ten, unique1, unique2, stringu1 + FROM onek + ORDER BY unique1 OFFSET 990 +-- !query schema +struct +-- !query output + 990 369 CMAAAA + 991 426 DMAAAA + 992 363 EMAAAA + 993 661 FMAAAA + 994 695 GMAAAA + 995 144 HMAAAA + 996 258 IMAAAA + 997 21 JMAAAA + 998 549 KMAAAA + 999 152 LMAAAA + + +-- !query +SELECT '' AS five, unique1, unique2, stringu1 + FROM onek + ORDER BY unique1 LIMIT 5 OFFSET 900 +-- !query schema +struct +-- !query output + 900 913 QIAAAA + 901 931 RIAAAA + 902 702 SIAAAA + 903 641 TIAAAA + 904 793 UIAAAA + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM + (VALUES + (123, 456), + (123, 4567890123456789), + (4567890123456789, 123), + (4567890123456789, 4567890123456789), + (4567890123456789, -4567890123456789)) + AS v(q1, q2) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from int8_tbl limit (case when random() < 0.5 then bigint(null) end) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE", + "sqlState" : "42K0E", + "messageParameters" : { + "expr" : "\"CASE WHEN (_nondeterministic < 0.5) THEN NULL END\"", + "name" : "limit" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 31, + "stopIndex" : 76, + "fragment" : "case when random() < 0.5 then bigint(null) end" + } ] +} + + +-- !query +select * from int8_tbl offset (case when random() < 0.5 then bigint(null) end) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_LIMIT_LIKE_EXPRESSION.IS_UNFOLDABLE", + "sqlState" : "42K0E", + "messageParameters" : { + "expr" : "\"CASE WHEN (_nondeterministic < 0.5) THEN NULL END\"", + "name" : "offset" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 77, + "fragment" : "case when random() < 0.5 then bigint(null) end" + } ] +} + + +-- !query +DROP VIEW INT8_TBL +-- !query schema +struct<> +-- !query output + + + +-- !query +select sum(tenthous) as s1, sum(tenthous) + random()*0 as s2 + from tenk1 group by thousand order by thousand limit 3 +-- !query schema +struct +-- !query output +45000 45000.0 +45010 45010.0 +45020 45020.0 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/numeric.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/numeric.sql.out new file mode 100644 index 000000000000..7b40f0bcaf23 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/numeric.sql.out @@ -0,0 +1,5252 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE num_data (id int, val decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE num_exp_add (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE num_exp_sub (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE num_exp_div (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE num_exp_mul (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE num_exp_sqrt (id int, expected decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE num_exp_ln (id int, expected decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE num_exp_log10 (id int, expected decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE num_exp_power_10_ln (id int, expected decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE num_result (id1 int, id2 int, result decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,2,-34338492.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,2,34338492.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,2,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,2,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,3,4.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,3,-4.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,3,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,3,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,4,7799461.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,4,-7799461.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,4,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,4,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,5,16397.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,5,-16397.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,5,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,5,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,6,93901.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,6,-93901.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,6,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,6,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,7,-83028485) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,7,83028485) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,7,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,7,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,8,74881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,8,-74881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,8,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,8,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,9,-24926804.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,9,24926804.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,9,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,9,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,2,-34338492.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,2,34338492.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,2,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,2,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,3,4.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,3,-4.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,3,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,3,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,4,7799461.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,4,-7799461.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,4,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,4,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,5,16397.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,5,-16397.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,5,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,5,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,6,93901.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,6,-93901.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,6,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,6,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,7,-83028485) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,7,83028485) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,7,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,7,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,8,74881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,8,-74881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,8,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,8,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,9,-24926804.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,9,24926804.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,9,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,9,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,0,-34338492.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,0,-34338492.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,1,-34338492.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,1,-34338492.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,2,-68676984.430794094) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,2,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,2,1179132047626883.596862135856320209) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,2,1.00000000000000000000) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,3,-34338487.905397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,3,-34338496.525397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,3,-147998901.44836127257) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,3,-7967167.56737750510440835266) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,4,-26539030.803497047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,4,-42137953.627297047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,4,-267821744976817.8111137106593) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,4,-4.40267480046830116685) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,5,-34322095.176906047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,5,-34354889.253888047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,5,-563049578578.769242506736077) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,5,-2094.18866914563535496429) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,6,-34244590.637766787) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,6,-34432393.793027307) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,6,-3224438592470.18449811926184222) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,6,-365.68599891479766440940) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,7,-117366977.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,7,48689992.784602953) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,7,2851072985828710.485883795) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,7,.41357483778485235518) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,8,-34263611.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,8,-34413373.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,8,-2571300635581.146276407) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,8,-458.57416721727870888476) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,9,-59265296.260444467) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,9,-9411688.170349627) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,9,855948866655588.453741509242968740) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,9,1.37757299946438931811) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,0,4.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,0,4.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,1,4.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,1,4.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,2,-34338487.905397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,2,34338496.525397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,2,-147998901.44836127257) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,2,-.00000012551512084352) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,3,8.62) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,3,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,3,18.5761) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,3,1.00000000000000000000) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,4,7799465.7219) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,4,-7799457.1019) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,4,33615678.685289) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,4,.00000055260225961552) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,5,16401.348491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,5,-16392.728491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,5,70671.23589621) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,5,.00026285234387695504) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,6,93905.88763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,6,-93897.26763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,6,404715.7995864206) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,6,.00004589912234457595) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,7,-83028480.69) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,7,83028489.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,7,-357852770.35) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,7,-.00000005190989574240) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,8,74885.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,8,-74876.69) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,8,322737.11) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,8,.00005755799201399553) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,9,-24926799.735047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,9,24926808.355047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,9,-107434525.43415438020) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,9,-.00000017290624149854) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,0,7799461.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,0,7799461.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,1,7799461.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,1,7799461.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,2,-26539030.803497047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,2,42137953.627297047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,2,-267821744976817.8111137106593) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,2,-.22713465002993920385) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,3,7799465.7219) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,3,7799457.1019) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,3,33615678.685289) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,3,1809619.81714617169373549883) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,4,15598922.8238) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,4,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,4,60831598315717.14146161) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,4,1.00000000000000000000) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,5,7815858.450391) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,5,7783064.373409) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,5,127888068979.9935054429) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,5,475.66281046305802686061) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,6,7893362.98953026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,6,7705559.83426974) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,6,732381731243.745115764094) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,6,83.05996138436129499606) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,7,-75229023.5881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,7,90827946.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,7,-647577464846017.9715) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,7,-.09393717604145131637) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,8,7874342.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,8,7724580.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,8,584031469984.4839) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,8,104.15808298366741897143) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,9,-17127342.633147420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,9,32726265.456947420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,9,-194415646271340.1815956522980) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,9,-.31289456112403769409) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,0,16397.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,0,16397.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,1,16397.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,1,16397.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,2,-34322095.176906047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,2,34354889.253888047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,2,-563049578578.769242506736077) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,2,-.00047751189505192446) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,3,16401.348491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,3,16392.728491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,3,70671.23589621) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,3,3804.41728329466357308584) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,4,7815858.450391) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,4,-7783064.373409) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,4,127888068979.9935054429) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,4,.00210232958726897192) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,5,32794.076982) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,5,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,5,268862871.275335557081) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,5,1.00000000000000000000) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,6,110298.61612126) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,6,-77504.53913926) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,6,1539707782.76899778633766) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,6,.17461941433576102689) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,7,-83012087.961509) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,7,83044882.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,7,-1361421264394.416135) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,7,-.00019748690453643710) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,8,91278.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,8,-58483.961509) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,8,1227826639.244571) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,8,.21897461960978085228) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,9,-24910407.006556420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,9,24943201.083538420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,9,-408725765384.257043660243220) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,9,-.00065780749354660427) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,0,93901.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,0,93901.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,1,93901.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,1,93901.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,2,-34244590.637766787) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,2,34432393.793027307) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,2,-3224438592470.18449811926184222) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,2,-.00273458651128995823) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,3,93905.88763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,3,93897.26763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,3,404715.7995864206) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,3,21786.90896293735498839907) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,4,7893362.98953026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,4,-7705559.83426974) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,4,732381731243.745115764094) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,4,.01203949512295682469) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,5,110298.61612126) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,5,77504.53913926) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,5,1539707782.76899778633766) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,5,5.72674008674192359679) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,6,187803.15526052) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,6,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,6,8817506281.4517452372676676) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,6,1.00000000000000000000) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,7,-82934583.42236974) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,7,83122386.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,7,-7796505729750.37795610) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,7,-.00113095617281538980) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,8,168782.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,8,19020.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,8,7031444034.53149906) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,8,1.25401073209839612184) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,9,-24832902.467417160) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,9,25020705.622677680) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,9,-2340666225110.29929521292692920) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,9,-.00376709254265256789) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,0,-83028485) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,0,-83028485) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,1,-83028485) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,1,-83028485) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,2,-117366977.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,2,-48689992.784602953) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,2,2851072985828710.485883795) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,2,2.41794207151503385700) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,3,-83028480.69) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,3,-83028489.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,3,-357852770.35) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,3,-19264149.65197215777262180974) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,4,-75229023.5881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,4,-90827946.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,4,-647577464846017.9715) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,4,-10.64541262725136247686) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,5,-83012087.961509) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,5,-83044882.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,5,-1361421264394.416135) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,5,-5063.62688881730941836574) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,6,-82934583.42236974) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,6,-83122386.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,6,-7796505729750.37795610) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,6,-884.20756174009028770294) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,7,-166056970) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,7,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,7,6893729321395225) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,7,1.00000000000000000000) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,8,-82953604) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,8,-83103366) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,8,-6217255985285) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,8,-1108.80577182462841041118) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,9,-107955289.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,9,-58101680.954952580) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,9,2069634775752159.035758700) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,9,3.33089171198810413382) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,0,74881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,0,74881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,1,74881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,1,74881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,2,-34263611.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,2,34413373.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,2,-2571300635581.146276407) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,2,-.00218067233500788615) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,3,74885.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,3,74876.69) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,3,322737.11) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,3,17373.78190255220417633410) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,4,7874342.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,4,-7724580.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,4,584031469984.4839) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,4,.00960079113741758956) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,5,91278.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,5,58483.961509) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,5,1227826639.244571) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,5,4.56673929509287019456) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,6,168782.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,6,-19020.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,6,7031444034.53149906) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,6,.79744134113322314424) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,7,-82953604) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,7,83103366) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,7,-6217255985285) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,7,-.00090187120721280172) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,8,149762) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,8,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,8,5607164161) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,8,1.00000000000000000000) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,9,-24851923.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,9,25001685.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,9,-1866544013697.195857020) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,9,-.00300403532938582735) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,0,-24926804.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,0,-24926804.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,1,-24926804.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,1,-24926804.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,2,-59265296.260444467) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,2,9411688.170349627) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,2,855948866655588.453741509242968740) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,2,.72591434384152961526) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,3,-24926799.735047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,3,-24926808.355047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,3,-107434525.43415438020) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,3,-5783481.21694835730858468677) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,4,-17127342.633147420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,4,-32726265.456947420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,4,-194415646271340.1815956522980) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,4,-3.19596478892958416484) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,5,-24910407.006556420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,5,-24943201.083538420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,5,-408725765384.257043660243220) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,5,-1520.20159364322004505807) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,6,-24832902.467417160) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,6,-25020705.622677680) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,6,-2340666225110.29929521292692920) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,6,-265.45671195426965751280) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,7,-107955289.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,7,58101680.954952580) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,7,2069634775752159.035758700) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,7,.30021990699995814689) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,8,-24851923.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,8,-25001685.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,8,-1866544013697.195857020) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,8,-332.88556569820675471748) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,9,-49853608.090094840) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,9,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,9,621345559900192.420120630048656400) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,9,1.00000000000000000000) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (2,5859.90547836712524903505) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (3,2.07605394920266944396) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (4,2792.75158435189147418923) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (5,128.05092147657509145473) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (6,306.43364311096782703406) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (7,9111.99676251039939975230) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (8,273.64392922189960397542) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (9,4992.67503899937593364766) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (2,17.35177750493897715514) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (3,1.46093790411565641971) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (4,15.86956523951936572464) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (5,9.70485601768871834038) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (6,11.45000246622944403127) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (7,18.23469429965478772991) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (8,11.22365546576315513668) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (9,17.03145425013166006962) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (2,7.53578122160797276459) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (3,.63447727016073160075) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (4,6.89206461372691743345) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (5,4.21476541614777768626) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (6,4.97267288886207207671) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (7,7.91922711353275546914) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (8,4.87437163556421004138) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (9,7.39666659961986567059) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (2,224790267919917955.13261618583642653184) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (3,28.90266599445155957393) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (4,7405685069594999.07733999469386277636) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (5,5068226527.32127265408584640098) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (6,281839893606.99372343357047819067) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (7,1716699575118597095.42330819910640247627) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION", + "sqlState" : "22003", + "messageParameters" : { + "maxPrecision" : "38", + "precision" : "39" + } +} + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (8,167361463828.07491320069016125952) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (9,107511333880052007.04141124673540337457) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (0, 0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (1, 0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (2, -34338492.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (3, 4.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (4, 7799461.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (5, 16397.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (6, 93901.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (7, -83028485) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (8, 74881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (9, -24926804.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM num_data +-- !query schema +struct +-- !query output +0 0.0000000000 +1 0.0000000000 +2 -34338492.2153970470 +3 4.3100000000 +4 7799461.4119000000 +5 16397.0384910000 +6 93901.5776302600 +7 -83028485.0000000000 +8 74881.0000000000 +9 -24926804.0450474200 + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, t1.val + t2.val + FROM num_data t1, num_data t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_add t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query schema +struct +-- !query output + + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val + t2.val, 10) + FROM num_data t1, num_data t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 10) as expected + FROM num_result t1, num_exp_add t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 10) +-- !query schema +struct +-- !query output + + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, t1.val - t2.val + FROM num_data t1, num_data t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_sub t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query schema +struct +-- !query output + + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val - t2.val, 40) + FROM num_data t1, num_data t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 40) + FROM num_result t1, num_exp_sub t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 40) +-- !query schema +struct +-- !query output + + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, t1.val, t2.val, t1.val * t2.val + FROM num_data t1, num_data t2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", + "sqlState" : "21S01", + "messageParameters" : { + "dataColumns" : "`id`, `id`, `val`, `val`, `(val * val)`", + "tableColumns" : "`id1`, `id2`, `result`", + "tableName" : "`spark_catalog`.`default`.`num_result`" + } +} + + +-- !query +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_mul t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query schema +struct +-- !query output + + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val * t2.val, 30) + FROM num_data t1, num_data t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 30) as expected + FROM num_result t1, num_exp_mul t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 30) +-- !query schema +struct +-- !query output +2 2 1179132047626883.5968620000 1179132047626883.5968621359 +2 3 -147998901.4483610000 -147998901.4483612726 +2 4 -267821744976817.8111140000 -267821744976817.8111137107 +2 5 -563049578578.7692430000 -563049578578.7692425067 +2 6 -3224438592470.1844980000 -3224438592470.1844981193 +2 7 2851072985828710.4858840000 2851072985828710.4858837950 +2 8 -2571300635581.1462760000 -2571300635581.1462764070 +2 9 855948866655588.4537420000 855948866655588.4537415092 +3 2 -147998901.4483610000 -147998901.4483612726 +3 5 70671.2358960000 70671.2358962100 +3 6 404715.7995860000 404715.7995864206 +3 9 -107434525.4341540000 -107434525.4341543802 +4 2 -267821744976817.8111140000 -267821744976817.8111137107 +4 4 60831598315717.1414620000 60831598315717.1414616100 +4 5 127888068979.9935050000 127888068979.9935054429 +4 6 732381731243.7451160000 732381731243.7451157641 +4 9 -194415646271340.1815960000 -194415646271340.1815956523 +5 2 -563049578578.7692430000 -563049578578.7692425067 +5 3 70671.2358960000 70671.2358962100 +5 4 127888068979.9935050000 127888068979.9935054429 +5 5 268862871.2753360000 268862871.2753355571 +5 6 1539707782.7689980000 1539707782.7689977863 +5 9 -408725765384.2570440000 -408725765384.2570436602 +6 2 -3224438592470.1844980000 -3224438592470.1844981193 +6 3 404715.7995860000 404715.7995864206 +6 4 732381731243.7451160000 732381731243.7451157641 +6 5 1539707782.7689980000 1539707782.7689977863 +6 6 8817506281.4517450000 8817506281.4517452373 +6 7 -7796505729750.3779560000 -7796505729750.3779561000 +6 8 7031444034.5314990000 7031444034.5314990600 +6 9 -2340666225110.2992950000 -2340666225110.2992952129 +7 2 2851072985828710.4858840000 2851072985828710.4858837950 +7 6 -7796505729750.3779560000 -7796505729750.3779561000 +7 9 2069634775752159.0357590000 2069634775752159.0357587000 +8 2 -2571300635581.1462760000 -2571300635581.1462764070 +8 6 7031444034.5314990000 7031444034.5314990600 +8 9 -1866544013697.1958570000 -1866544013697.1958570200 +9 2 855948866655588.4537420000 855948866655588.4537415092 +9 3 -107434525.4341540000 -107434525.4341543802 +9 4 -194415646271340.1815960000 -194415646271340.1815956523 +9 5 -408725765384.2570440000 -408725765384.2570436602 +9 6 -2340666225110.2992950000 -2340666225110.2992952129 +9 7 2069634775752159.0357590000 2069634775752159.0357587000 +9 8 -1866544013697.1958570000 -1866544013697.1958570200 +9 9 621345559900192.4201210000 621345559900192.4201206300 + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, t1.val / t2.val + FROM num_data t1, num_data t2 + WHERE t2.val != '0.0' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_div t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query schema +struct +-- !query output +2 3 -7967167.5673780000 -7967167.5673775051 +2 4 -4.4026750000 -4.4026748005 +2 5 -2094.1886690000 -2094.1886691456 +2 6 -365.6859990000 -365.6859989148 +2 7 0.4135750000 0.4135748378 +2 8 -458.5741670000 -458.5741672173 +2 9 1.3775730000 1.3775729995 +3 2 0.0000000000 -0.0000001255 +3 4 0.0000010000 0.0000005526 +3 5 0.0002630000 0.0002628523 +3 6 0.0000460000 0.0000458991 +3 7 0.0000000000 -0.0000000519 +3 8 0.0000580000 0.0000575580 +3 9 0.0000000000 -0.0000001729 +4 2 -0.2271350000 -0.2271346500 +4 3 1809619.8171460000 1809619.8171461717 +4 5 475.6628100000 475.6628104631 +4 6 83.0599610000 83.0599613844 +4 7 -0.0939370000 -0.0939371760 +4 8 104.1580830000 104.1580829837 +4 9 -0.3128950000 -0.3128945611 +5 2 -0.0004780000 -0.0004775119 +5 3 3804.4172830000 3804.4172832947 +5 4 0.0021020000 0.0021023296 +5 6 0.1746190000 0.1746194143 +5 7 -0.0001970000 -0.0001974869 +5 8 0.2189750000 0.2189746196 +5 9 -0.0006580000 -0.0006578075 +6 2 -0.0027350000 -0.0027345865 +6 3 21786.9089630000 21786.9089629374 +6 4 0.0120390000 0.0120394951 +6 5 5.7267400000 5.7267400867 +6 7 -0.0011310000 -0.0011309562 +6 8 1.2540110000 1.2540107321 +6 9 -0.0037670000 -0.0037670925 +7 2 2.4179420000 2.4179420715 +7 3 -19264149.6519720000 -19264149.6519721578 +7 4 -10.6454130000 -10.6454126273 +7 5 -5063.6268890000 -5063.6268888173 +7 6 -884.2075620000 -884.2075617401 +7 8 -1108.8057720000 -1108.8057718246 +7 9 3.3308920000 3.3308917120 +8 2 -0.0021810000 -0.0021806723 +8 3 17373.7819030000 17373.7819025522 +8 4 0.0096010000 0.0096007911 +8 5 4.5667390000 4.5667392951 +8 6 0.7974410000 0.7974413411 +8 7 -0.0009020000 -0.0009018712 +8 9 -0.0030040000 -0.0030040353 +9 2 0.7259140000 0.7259143438 +9 3 -5783481.2169480000 -5783481.2169483573 +9 4 -3.1959650000 -3.1959647889 +9 5 -1520.2015940000 -1520.2015936432 +9 6 -265.4567120000 -265.4567119543 +9 7 0.3002200000 0.3002199070 +9 8 -332.8855660000 -332.8855656982 + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val / t2.val, 80) + FROM num_data t1, num_data t2 + WHERE t2.val != '0.0' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 80) as expected + FROM num_result t1, num_exp_div t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 80) +-- !query schema +struct +-- !query output +2 3 -7967167.5673780000 -7967167.5673775051 +2 4 -4.4026750000 -4.4026748005 +2 5 -2094.1886690000 -2094.1886691456 +2 6 -365.6859990000 -365.6859989148 +2 7 0.4135750000 0.4135748378 +2 8 -458.5741670000 -458.5741672173 +2 9 1.3775730000 1.3775729995 +3 2 0.0000000000 -0.0000001255 +3 4 0.0000010000 0.0000005526 +3 5 0.0002630000 0.0002628523 +3 6 0.0000460000 0.0000458991 +3 7 0.0000000000 -0.0000000519 +3 8 0.0000580000 0.0000575580 +3 9 0.0000000000 -0.0000001729 +4 2 -0.2271350000 -0.2271346500 +4 3 1809619.8171460000 1809619.8171461717 +4 5 475.6628100000 475.6628104631 +4 6 83.0599610000 83.0599613844 +4 7 -0.0939370000 -0.0939371760 +4 8 104.1580830000 104.1580829837 +4 9 -0.3128950000 -0.3128945611 +5 2 -0.0004780000 -0.0004775119 +5 3 3804.4172830000 3804.4172832947 +5 4 0.0021020000 0.0021023296 +5 6 0.1746190000 0.1746194143 +5 7 -0.0001970000 -0.0001974869 +5 8 0.2189750000 0.2189746196 +5 9 -0.0006580000 -0.0006578075 +6 2 -0.0027350000 -0.0027345865 +6 3 21786.9089630000 21786.9089629374 +6 4 0.0120390000 0.0120394951 +6 5 5.7267400000 5.7267400867 +6 7 -0.0011310000 -0.0011309562 +6 8 1.2540110000 1.2540107321 +6 9 -0.0037670000 -0.0037670925 +7 2 2.4179420000 2.4179420715 +7 3 -19264149.6519720000 -19264149.6519721578 +7 4 -10.6454130000 -10.6454126273 +7 5 -5063.6268890000 -5063.6268888173 +7 6 -884.2075620000 -884.2075617401 +7 8 -1108.8057720000 -1108.8057718246 +7 9 3.3308920000 3.3308917120 +8 2 -0.0021810000 -0.0021806723 +8 3 17373.7819030000 17373.7819025522 +8 4 0.0096010000 0.0096007911 +8 5 4.5667390000 4.5667392951 +8 6 0.7974410000 0.7974413411 +8 7 -0.0009020000 -0.0009018712 +8 9 -0.0030040000 -0.0030040353 +9 2 0.7259140000 0.7259143438 +9 3 -5783481.2169480000 -5783481.2169483573 +9 4 -3.1959650000 -3.1959647889 +9 5 -1520.2015940000 -1520.2015936432 +9 6 -265.4567120000 -265.4567119543 +9 7 0.3002200000 0.3002199070 +9 8 -332.8855660000 -332.8855656982 + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT id, 0, SQRT(ABS(val)) + FROM num_data +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_sqrt t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query schema +struct +-- !query output + + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT id, 0, LN(ABS(val)) + FROM num_data + WHERE val != '0.0' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_ln t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query schema +struct +-- !query output + + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT id, 0, LOG(cast('10' as decimal(38, 18)), ABS(val)) + FROM num_data + WHERE val != '0.0' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_log10 t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query schema +struct +-- !query output + + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT id, 0, POWER(cast('10' as decimal(38, 18)), LN(ABS(round(val,200)))) + FROM num_data + WHERE val != '0.0' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_power_10_ln t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query schema +struct +-- !query output +2 224790267919917440.0000000000 224790267919917955.1326161858 +4 7405685069595001.0000000000 7405685069594999.0773399947 +5 5068226527.3212630000 5068226527.3212726541 +6 281839893606.9936500000 281839893606.9937234336 +8 167361463828.0749000000 167361463828.0749132007 +9 107511333880051872.0000000000 107511333880052007.0414112467 + + +-- !query +SELECT AVG(val) FROM num_data +-- !query schema +struct +-- !query output +-13430913.59224232070000 + + +-- !query +CREATE TABLE fract_only (id int, val decimal(4,4)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO fract_only VALUES (1, 0.0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO fract_only VALUES (2, 0.1) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO fract_only VALUES (4, -0.9999) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO fract_only VALUES (5, 0.99994) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO fract_only VALUES (7, 0.00001) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO fract_only VALUES (8, 0.00017) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM fract_only +-- !query schema +struct +-- !query output +1 0.0000 +2 0.1000 +4 -0.9999 +5 0.9999 +7 0.0000 +8 0.0002 + + +-- !query +DROP TABLE fract_only +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT decimal(double('NaN')) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT decimal(double('Infinity')) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT decimal(double('-Infinity')) +-- !query schema +struct<-Infinity:decimal(10,0)> +-- !query output +NULL + + +-- !query +SELECT decimal(float('NaN')) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT decimal(float('Infinity')) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT decimal(float('-Infinity')) +-- !query schema +struct<-Infinity:decimal(10,0)> +-- !query output +NULL + + +-- !query +CREATE TABLE ceil_floor_round (a decimal(38, 18)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO ceil_floor_round VALUES (-5.5) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO ceil_floor_round VALUES (-5.499999) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO ceil_floor_round VALUES (9.5) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO ceil_floor_round VALUES (9.4999999) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO ceil_floor_round VALUES (0.0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO ceil_floor_round VALUES (0.0000001) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO ceil_floor_round VALUES (-0.000001) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT a, ceil(a), ceiling(a), floor(a), round(a) FROM ceil_floor_round +-- !query schema +struct +-- !query output +-0.000001000000000000 0 0 -1 0 +-5.499999000000000000 -5 -5 -6 -5 +-5.500000000000000000 -5 -5 -6 -6 +0.000000000000000000 0 0 0 0 +0.000000100000000000 1 1 0 0 +9.499999900000000000 10 10 9 9 +9.500000000000000000 10 10 9 10 + + +-- !query +DROP TABLE ceil_floor_round +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT width_bucket(5.0, 3.0, 4.0, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(5.0, 3.0, 4.0, -5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(3.5, 3.0, 3.0, 888) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(double(5.0), double(3.0), double(4.0), 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(double(5.0), double(3.0), double(4.0), -5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(double(3.5), double(3.0), double(3.0), 888) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket('NaN', 3.0, 4.0, 888) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(double(0), 'NaN', double(4.0), 888) +-- !query schema +struct +-- !query output +NULL + + +-- !query +CREATE TABLE width_bucket_test (operand_num decimal(30,15), operand_f8 double) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO width_bucket_test VALUES + (-5.2, -5.2), + (-0.0000000001, -0.0000000001), + (0.000000000001, 0.000000000001), + (1, 1), + (1.99999999999999, 1.99999999999999), + (2, 2), + (2.00000000000001, 2.00000000000001), + (3, 3), + (4, 4), + (4.5, 4.5), + (5, 5), + (5.5, 5.5), + (6, 6), + (7, 7), + (8, 8), + (9, 9), + (9.99999999999999, 9.99999999999999), + (10, 10), + (10.0000000000001, 10.0000000000001) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + operand_num, + width_bucket(operand_num, 0, 10, 5) AS wb_1, + width_bucket(operand_f8, 0, 10, 5) AS wb_1f, + width_bucket(operand_num, 10, 0, 5) AS wb_2, + width_bucket(operand_f8, 10, 0, 5) AS wb_2f, + width_bucket(operand_num, 2, 8, 4) AS wb_3, + width_bucket(operand_f8, 2, 8, 4) AS wb_3f, + width_bucket(operand_num, 5.0, 5.5, 20) AS wb_4, + width_bucket(operand_f8, 5.0, 5.5, 20) AS wb_4f, + width_bucket(operand_num, -25, 25, 10) AS wb_5, + width_bucket(operand_f8, -25, 25, 10) AS wb_5f + FROM width_bucket_test + ORDER BY operand_num ASC +-- !query schema +struct +-- !query output +-5.200000000000000 0 0 6 6 0 0 0 0 4 4 +-0.000000000100000 0 0 6 6 0 0 0 0 5 5 +0.000000000001000 1 1 5 5 0 0 0 0 6 6 +1.000000000000000 1 1 5 5 0 0 0 0 6 6 +1.999999999999990 1 1 5 5 0 0 0 0 6 6 +2.000000000000000 2 2 5 5 1 1 0 0 6 6 +2.000000000000010 2 2 4 4 1 1 0 0 6 6 +3.000000000000000 2 2 4 4 1 1 0 0 6 6 +4.000000000000000 3 3 4 4 2 2 0 0 6 6 +4.500000000000000 3 3 3 3 2 2 0 0 6 6 +5.000000000000000 3 3 3 3 3 3 1 1 7 7 +5.500000000000000 3 3 3 3 3 3 21 21 7 7 +6.000000000000000 4 4 3 3 3 3 21 21 7 7 +7.000000000000000 4 4 2 2 4 4 21 21 7 7 +8.000000000000000 5 5 2 2 5 5 21 21 7 7 +9.000000000000000 5 5 1 1 5 5 21 21 7 7 +9.999999999999990 5 5 1 1 5 5 21 21 7 7 +10.000000000000000 6 6 1 1 5 5 21 21 8 8 +10.000000000000100 6 6 0 0 5 5 21 21 8 8 + + +-- !query +SELECT width_bucket(double(0.0), double('Infinity'), 5, 10) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(double(0.0), 5, double('-Infinity'), 20) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(double('Infinity'), 1, 10, 10), + width_bucket(double('-Infinity'), 1, 10, 10) +-- !query schema +struct +-- !query output +11 0 + + +-- !query +DROP TABLE width_bucket_test +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT '' AS to_char_3, to_char(val, '9999999999999999.999999999999999PR'), val +FROM num_data +-- !query schema +struct +-- !query output + 0.000000000000000 0.0000000000 + 0.000000000000000 0.0000000000 + 4.310000000000000 4.3100000000 + 16397.038491000000000 16397.0384910000 + 74881.000000000000000 74881.0000000000 + 93901.577630260000000 93901.5776302600 + 7799461.411900000000000 7799461.4119000000 + <24926804.045047420000000> -24926804.0450474200 + <34338492.215397047000000> -34338492.2153970470 + <83028485.000000000000000> -83028485.0000000000 + + +-- !query +SELECT '' AS to_char_4, to_char(val, '9999999999999999.999999999999999S'), val +FROM num_data +-- !query schema +struct +-- !query output + 0.000000000000000+ 0.0000000000 + 0.000000000000000+ 0.0000000000 + 4.310000000000000+ 4.3100000000 + 16397.038491000000000+ 16397.0384910000 + 74881.000000000000000+ 74881.0000000000 + 93901.577630260000000+ 93901.5776302600 + 7799461.411900000000000+ 7799461.4119000000 + 24926804.045047420000000- -24926804.0450474200 + 34338492.215397047000000- -34338492.2153970470 + 83028485.000000000000000- -83028485.0000000000 + + +-- !query +SELECT '' AS to_char_5, to_char(val, 'MI9999999999999999.999999999999999'), val FROM num_data +-- !query schema +struct +-- !query output + 0.000000000000000 0.0000000000 + 0.000000000000000 0.0000000000 + 4.310000000000000 4.3100000000 + 16397.038491000000000 16397.0384910000 + 74881.000000000000000 74881.0000000000 + 93901.577630260000000 93901.5776302600 + 7799461.411900000000000 7799461.4119000000 + -24926804.045047420000000 -24926804.0450474200 + -34338492.215397047000000 -34338492.2153970470 + -83028485.000000000000000 -83028485.0000000000 + + +-- !query +SELECT '' AS to_char_9, to_char(val, '0999999999999999.999999999999999'), val FROM num_data +-- !query schema +struct +-- !query output + 0000000000000000.000000000000000 0.0000000000 + 0000000000000000.000000000000000 0.0000000000 + 0000000000000004.310000000000000 4.3100000000 + 0000000000016397.038491000000000 16397.0384910000 + 0000000000074881.000000000000000 74881.0000000000 + 0000000000093901.577630260000000 93901.5776302600 + 0000000007799461.411900000000000 7799461.4119000000 + 0000000024926804.045047420000000 -24926804.0450474200 + 0000000034338492.215397047000000 -34338492.2153970470 + 0000000083028485.000000000000000 -83028485.0000000000 + + +-- !query +SELECT '' AS to_char_10, to_char(val, 'S0999999999999999.999999999999999'), val FROM num_data +-- !query schema +struct +-- !query output + +0000000000000000.000000000000000 0.0000000000 + +0000000000000000.000000000000000 0.0000000000 + +0000000000000004.310000000000000 4.3100000000 + +0000000000016397.038491000000000 16397.0384910000 + +0000000000074881.000000000000000 74881.0000000000 + +0000000000093901.577630260000000 93901.5776302600 + +0000000007799461.411900000000000 7799461.4119000000 + -0000000024926804.045047420000000 -24926804.0450474200 + -0000000034338492.215397047000000 -34338492.2153970470 + -0000000083028485.000000000000000 -83028485.0000000000 + + +-- !query +SELECT '' AS to_number_1, to_number('-34,338,492', '99G999G999') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_FORMAT.MISMATCH_INPUT", + "sqlState" : "42601", + "messageParameters" : { + "format" : "99G999G999", + "input" : "-34,338,492", + "inputType" : "\"STRING\"" + } +} + + +-- !query +SELECT '' AS to_number_2, to_number('-34,338,492.654,878', '99G999G999D999G999') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_FORMAT.THOUSANDS_SEPS_MUST_BEFORE_DEC", + "sqlState" : "42601", + "messageParameters" : { + "format" : "'99G999G999D999G999'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 81, + "fragment" : "to_number('-34,338,492.654,878', '99G999G999D999G999')" + } ] +} + + +-- !query +SELECT '' AS to_number_4, to_number('0.00001-', '9.999999S') +-- !query schema +struct +-- !query output + -0.000010 + + +-- !query +SELECT '' AS to_number_9, to_number('.0', '99999999.99999999') +-- !query schema +struct +-- !query output + 0.00000000 + + +-- !query +SELECT '' AS to_number_10, to_number('0', '99.99') +-- !query schema +struct +-- !query output + 0.00 + + +-- !query +SELECT '' AS to_number_12, to_number('.01-', '99.99S') +-- !query schema +struct +-- !query output + -0.01 + + +-- !query +SELECT '' AS to_number_14, to_number('34,50','999,99') +-- !query schema +struct +-- !query output + 3450 + + +-- !query +SELECT '' AS to_number_15, to_number('123,000','999G') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_FORMAT.CONT_THOUSANDS_SEPS", + "sqlState" : "42601", + "messageParameters" : { + "format" : "'999G'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 54, + "fragment" : "to_number('123,000','999G')" + } ] +} + + +-- !query +SELECT '' AS to_number_16, to_number('123456','999G999') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_FORMAT.MISMATCH_INPUT", + "sqlState" : "42601", + "messageParameters" : { + "format" : "999G999", + "input" : "123456", + "inputType" : "\"STRING\"" + } +} + + +-- !query +CREATE TABLE num_input_test (n1 decimal(38, 18)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_input_test VALUES (double(trim(' 123'))) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_input_test VALUES (double(trim(' 3245874 '))) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_input_test VALUES (double(trim(' -93853'))) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_input_test VALUES (555.50) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_input_test VALUES (-555.50) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM num_input_test +-- !query schema +struct +-- !query output +-555.500000000000000000 +-93853.000000000000000000 +123.000000000000000000 +3245874.000000000000000000 +555.500000000000000000 + + +-- !query +select cast(999999999999999999999 as decimal(38, 0))/1000000000000000000000 +-- !query schema +struct<(CAST(999999999999999999999 AS DECIMAL(38,0)) / 1000000000000000000000):decimal(38,6)> +-- !query output +1.000000 + + +-- !query +select div(cast(999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query schema +struct<(CAST(999999999999999999999 AS DECIMAL(38,0)) div 1000000000000000000000):bigint> +-- !query output +0 + + +-- !query +select mod(cast(999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query schema +struct +-- !query output +999999999999999999999 + + +-- !query +select div(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query schema +struct<(CAST(-9999999999999999999999 AS DECIMAL(38,0)) div 1000000000000000000000):bigint> +-- !query output +-9 + + +-- !query +select mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query schema +struct +-- !query output +-999999999999999999999 + + +-- !query +select div(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000)*1000000000000000000000 + mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query schema +struct<(((CAST(-9999999999999999999999 AS DECIMAL(38,0)) div 1000000000000000000000) * 1000000000000000000000) + mod(CAST(-9999999999999999999999 AS DECIMAL(38,0)), 1000000000000000000000)):decimal(38,0)> +-- !query output +-9999999999999999999999 + + +-- !query +select mod (70.0,70) +-- !query schema +struct +-- !query output +0.0 + + +-- !query +select div (70.0,70) +-- !query schema +struct<(70.0 div 70):bigint> +-- !query output +1 + + +-- !query +select 70.0 / 70 +-- !query schema +struct<(70.0 / 70):decimal(8,6)> +-- !query output +1.000000 + + +-- !query +select 12345678901234567890 % 123 +-- !query schema +struct<(12345678901234567890 % 123):decimal(3,0)> +-- !query output +78 + + +-- !query +select exp(0.0) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +select exp(1.0) +-- !query schema +struct +-- !query output +2.7182818284590455 + + +-- !query +select exp(32.999) +-- !query schema +struct +-- !query output +2.1442904349215556E14 + + +-- !query +select exp(-32.999) +-- !query schema +struct +-- !query output +4.663547361468238E-15 + + +-- !query +select exp(123.456) +-- !query schema +struct +-- !query output +4.132944352778106E53 + + +-- !query +select exp(-123.456) +-- !query schema +struct +-- !query output +2.4195825412645934E-54 + + +-- !query +select exp(1234.5678) +-- !query schema +struct +-- !query output +Infinity + + +-- !query +select * from range(cast(0.0 as decimal(38, 18)), cast(4.0 as decimal(38, 18))) +-- !query schema +struct +-- !query output +0 +1 +2 +3 + + +-- !query +select * from range(cast(0.1 as decimal(38, 18)), cast(4.0 as decimal(38, 18)), cast(1.3 as decimal(38, 18))) +-- !query schema +struct +-- !query output +0 +1 +2 +3 + + +-- !query +select * from range(cast(4.0 as decimal(38, 18)), cast(-1.5 as decimal(38, 18)), cast(-2.2 as decimal(38, 18))) +-- !query schema +struct +-- !query output +0 +2 +4 + + +-- !query +select ln(1.2345678e-28) +-- !query schema +struct +-- !query output +-64.26166165451762 + + +-- !query +select ln(0.0456789) +-- !query schema +struct +-- !query output +-3.0861187944847437 + + +-- !query +select ln(0.99949452) +-- !query schema +struct +-- !query output +-5.056077980832118E-4 + + +-- !query +select ln(1.00049687395) +-- !query schema +struct +-- !query output +4.967505490136803E-4 + + +-- !query +select ln(1234.567890123456789) +-- !query schema +struct +-- !query output +7.11847630129779 + + +-- !query +select ln(5.80397490724e5) +-- !query schema +struct +-- !query output +13.271468476626518 + + +-- !query +select ln(9.342536355e34) +-- !query schema +struct +-- !query output +80.52247093552418 + + +-- !query +select log(3.4634998359873254962349856073435545) +-- !query schema +struct +-- !query output +1.2422795911259166 + + +-- !query +select log(9.999999999999999999) +-- !query schema +struct +-- !query output +2.302585092994046 + + +-- !query +select log(10.00000000000000000) +-- !query schema +struct +-- !query output +2.302585092994046 + + +-- !query +select log(10.00000000000000001) +-- !query schema +struct +-- !query output +2.302585092994046 + + +-- !query +select log(590489.45235237) +-- !query schema +struct +-- !query output +13.288707052228641 + + +-- !query +select log(0.99923, 4.58934e34) +-- !query schema +struct +-- !query output +-103611.55579543479 + + +-- !query +select log(1.000016, 8.452010e18) +-- !query schema +struct +-- !query output +2723830.287707013 + + +-- !query +SELECT SUM(decimal(9999)) FROM range(1, 100001) +-- !query schema +struct +-- !query output +999900000 + + +-- !query +SELECT SUM(decimal(-9999)) FROM range(1, 100001) +-- !query schema +struct +-- !query output +-999900000 + + +-- !query +DROP TABLE num_data +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_exp_add +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_exp_sub +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_exp_div +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_exp_mul +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_exp_sqrt +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_exp_ln +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_exp_log10 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_exp_power_10_ln +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_input_test +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/numeric.sql.out.java21 b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/numeric.sql.out.java21 new file mode 100644 index 000000000000..7f89381c18eb --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/numeric.sql.out.java21 @@ -0,0 +1,5252 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE num_data (id int, val decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE num_exp_add (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE num_exp_sub (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE num_exp_div (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE num_exp_mul (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE num_exp_sqrt (id int, expected decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE num_exp_ln (id int, expected decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE num_exp_log10 (id int, expected decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE num_exp_power_10_ln (id int, expected decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE num_result (id1 int, id2 int, result decimal(38,10)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,2,-34338492.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,2,34338492.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,2,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,2,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,3,4.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,3,-4.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,3,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,3,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,4,7799461.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,4,-7799461.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,4,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,4,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,5,16397.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,5,-16397.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,5,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,5,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,6,93901.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,6,-93901.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,6,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,6,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,7,-83028485) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,7,83028485) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,7,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,7,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,8,74881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,8,-74881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,8,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,8,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (0,9,-24926804.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (0,9,24926804.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (0,9,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (0,9,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,2,-34338492.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,2,34338492.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,2,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,2,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,3,4.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,3,-4.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,3,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,3,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,4,7799461.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,4,-7799461.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,4,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,4,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,5,16397.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,5,-16397.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,5,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,5,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,6,93901.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,6,-93901.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,6,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,6,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,7,-83028485) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,7,83028485) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,7,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,7,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,8,74881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,8,-74881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,8,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,8,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (1,9,-24926804.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (1,9,24926804.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (1,9,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (1,9,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,0,-34338492.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,0,-34338492.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,1,-34338492.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,1,-34338492.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,2,-68676984.430794094) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,2,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,2,1179132047626883.596862135856320209) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,2,1.00000000000000000000) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,3,-34338487.905397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,3,-34338496.525397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,3,-147998901.44836127257) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,3,-7967167.56737750510440835266) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,4,-26539030.803497047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,4,-42137953.627297047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,4,-267821744976817.8111137106593) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,4,-4.40267480046830116685) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,5,-34322095.176906047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,5,-34354889.253888047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,5,-563049578578.769242506736077) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,5,-2094.18866914563535496429) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,6,-34244590.637766787) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,6,-34432393.793027307) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,6,-3224438592470.18449811926184222) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,6,-365.68599891479766440940) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,7,-117366977.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,7,48689992.784602953) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,7,2851072985828710.485883795) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,7,.41357483778485235518) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,8,-34263611.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,8,-34413373.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,8,-2571300635581.146276407) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,8,-458.57416721727870888476) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (2,9,-59265296.260444467) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (2,9,-9411688.170349627) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (2,9,855948866655588.453741509242968740) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (2,9,1.37757299946438931811) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,0,4.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,0,4.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,1,4.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,1,4.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,2,-34338487.905397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,2,34338496.525397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,2,-147998901.44836127257) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,2,-.00000012551512084352) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,3,8.62) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,3,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,3,18.5761) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,3,1.00000000000000000000) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,4,7799465.7219) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,4,-7799457.1019) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,4,33615678.685289) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,4,.00000055260225961552) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,5,16401.348491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,5,-16392.728491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,5,70671.23589621) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,5,.00026285234387695504) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,6,93905.88763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,6,-93897.26763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,6,404715.7995864206) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,6,.00004589912234457595) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,7,-83028480.69) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,7,83028489.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,7,-357852770.35) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,7,-.00000005190989574240) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,8,74885.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,8,-74876.69) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,8,322737.11) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,8,.00005755799201399553) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (3,9,-24926799.735047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (3,9,24926808.355047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (3,9,-107434525.43415438020) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (3,9,-.00000017290624149854) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,0,7799461.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,0,7799461.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,1,7799461.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,1,7799461.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,2,-26539030.803497047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,2,42137953.627297047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,2,-267821744976817.8111137106593) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,2,-.22713465002993920385) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,3,7799465.7219) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,3,7799457.1019) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,3,33615678.685289) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,3,1809619.81714617169373549883) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,4,15598922.8238) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,4,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,4,60831598315717.14146161) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,4,1.00000000000000000000) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,5,7815858.450391) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,5,7783064.373409) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,5,127888068979.9935054429) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,5,475.66281046305802686061) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,6,7893362.98953026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,6,7705559.83426974) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,6,732381731243.745115764094) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,6,83.05996138436129499606) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,7,-75229023.5881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,7,90827946.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,7,-647577464846017.9715) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,7,-.09393717604145131637) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,8,7874342.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,8,7724580.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,8,584031469984.4839) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,8,104.15808298366741897143) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (4,9,-17127342.633147420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (4,9,32726265.456947420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (4,9,-194415646271340.1815956522980) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (4,9,-.31289456112403769409) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,0,16397.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,0,16397.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,1,16397.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,1,16397.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,2,-34322095.176906047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,2,34354889.253888047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,2,-563049578578.769242506736077) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,2,-.00047751189505192446) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,3,16401.348491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,3,16392.728491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,3,70671.23589621) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,3,3804.41728329466357308584) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,4,7815858.450391) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,4,-7783064.373409) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,4,127888068979.9935054429) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,4,.00210232958726897192) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,5,32794.076982) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,5,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,5,268862871.275335557081) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,5,1.00000000000000000000) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,6,110298.61612126) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,6,-77504.53913926) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,6,1539707782.76899778633766) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,6,.17461941433576102689) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,7,-83012087.961509) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,7,83044882.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,7,-1361421264394.416135) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,7,-.00019748690453643710) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,8,91278.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,8,-58483.961509) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,8,1227826639.244571) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,8,.21897461960978085228) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (5,9,-24910407.006556420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (5,9,24943201.083538420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (5,9,-408725765384.257043660243220) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (5,9,-.00065780749354660427) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,0,93901.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,0,93901.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,1,93901.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,1,93901.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,2,-34244590.637766787) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,2,34432393.793027307) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,2,-3224438592470.18449811926184222) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,2,-.00273458651128995823) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,3,93905.88763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,3,93897.26763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,3,404715.7995864206) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,3,21786.90896293735498839907) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,4,7893362.98953026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,4,-7705559.83426974) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,4,732381731243.745115764094) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,4,.01203949512295682469) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,5,110298.61612126) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,5,77504.53913926) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,5,1539707782.76899778633766) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,5,5.72674008674192359679) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,6,187803.15526052) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,6,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,6,8817506281.4517452372676676) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,6,1.00000000000000000000) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,7,-82934583.42236974) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,7,83122386.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,7,-7796505729750.37795610) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,7,-.00113095617281538980) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,8,168782.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,8,19020.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,8,7031444034.53149906) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,8,1.25401073209839612184) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (6,9,-24832902.467417160) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (6,9,25020705.622677680) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (6,9,-2340666225110.29929521292692920) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (6,9,-.00376709254265256789) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,0,-83028485) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,0,-83028485) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,1,-83028485) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,1,-83028485) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,2,-117366977.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,2,-48689992.784602953) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,2,2851072985828710.485883795) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,2,2.41794207151503385700) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,3,-83028480.69) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,3,-83028489.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,3,-357852770.35) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,3,-19264149.65197215777262180974) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,4,-75229023.5881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,4,-90827946.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,4,-647577464846017.9715) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,4,-10.64541262725136247686) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,5,-83012087.961509) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,5,-83044882.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,5,-1361421264394.416135) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,5,-5063.62688881730941836574) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,6,-82934583.42236974) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,6,-83122386.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,6,-7796505729750.37795610) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,6,-884.20756174009028770294) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,7,-166056970) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,7,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,7,6893729321395225) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,7,1.00000000000000000000) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,8,-82953604) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,8,-83103366) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,8,-6217255985285) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,8,-1108.80577182462841041118) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (7,9,-107955289.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (7,9,-58101680.954952580) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (7,9,2069634775752159.035758700) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (7,9,3.33089171198810413382) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,0,74881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,0,74881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,1,74881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,1,74881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,2,-34263611.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,2,34413373.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,2,-2571300635581.146276407) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,2,-.00218067233500788615) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,3,74885.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,3,74876.69) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,3,322737.11) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,3,17373.78190255220417633410) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,4,7874342.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,4,-7724580.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,4,584031469984.4839) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,4,.00960079113741758956) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,5,91278.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,5,58483.961509) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,5,1227826639.244571) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,5,4.56673929509287019456) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,6,168782.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,6,-19020.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,6,7031444034.53149906) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,6,.79744134113322314424) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,7,-82953604) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,7,83103366) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,7,-6217255985285) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,7,-.00090187120721280172) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,8,149762) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,8,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,8,5607164161) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,8,1.00000000000000000000) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (8,9,-24851923.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (8,9,25001685.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (8,9,-1866544013697.195857020) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (8,9,-.00300403532938582735) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,0,-24926804.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,0,-24926804.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,1,-24926804.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,1,-24926804.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,2,-59265296.260444467) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,2,9411688.170349627) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,2,855948866655588.453741509242968740) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,2,.72591434384152961526) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,3,-24926799.735047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,3,-24926808.355047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,3,-107434525.43415438020) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,3,-5783481.21694835730858468677) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,4,-17127342.633147420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,4,-32726265.456947420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,4,-194415646271340.1815956522980) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,4,-3.19596478892958416484) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,5,-24910407.006556420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,5,-24943201.083538420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,5,-408725765384.257043660243220) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,5,-1520.20159364322004505807) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,6,-24832902.467417160) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,6,-25020705.622677680) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,6,-2340666225110.29929521292692920) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,6,-265.45671195426965751280) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,7,-107955289.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,7,58101680.954952580) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,7,2069634775752159.035758700) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,7,.30021990699995814689) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,8,-24851923.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,8,-25001685.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,8,-1866544013697.195857020) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,8,-332.88556569820675471748) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_add VALUES (9,9,-49853608.090094840) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sub VALUES (9,9,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_mul VALUES (9,9,621345559900192.420120630048656400) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_div VALUES (9,9,1.00000000000000000000) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (0,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (1,0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (2,5859.90547836712524903505) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (3,2.07605394920266944396) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (4,2792.75158435189147418923) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (5,128.05092147657509145473) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (6,306.43364311096782703406) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (7,9111.99676251039939975230) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (8,273.64392922189960397542) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_sqrt VALUES (9,4992.67503899937593364766) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (2,17.35177750493897715514) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (3,1.46093790411565641971) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (4,15.86956523951936572464) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (5,9.70485601768871834038) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (6,11.45000246622944403127) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (7,18.23469429965478772991) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (8,11.22365546576315513668) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_ln VALUES (9,17.03145425013166006962) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (2,7.53578122160797276459) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (3,.63447727016073160075) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (4,6.89206461372691743345) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (5,4.21476541614777768626) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (6,4.97267288886207207671) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (7,7.91922711353275546914) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (8,4.87437163556421004138) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_log10 VALUES (9,7.39666659961986567059) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (0,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (1,double('NaN')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (2,224790267919917955.13261618583642653184) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (3,28.90266599445155957393) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (4,7405685069594999.07733999469386277636) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (5,5068226527.32127265408584640098) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (6,281839893606.99372343357047819067) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (7,1716699575118597095.42330819910640247627) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION", + "sqlState" : "22003", + "messageParameters" : { + "maxPrecision" : "38", + "precision" : "39" + } +} + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (8,167361463828.07491320069016125952) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_exp_power_10_ln VALUES (9,107511333880052007.04141124673540337457) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (0, 0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (1, 0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (2, -34338492.215397047) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (3, 4.31) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (4, 7799461.4119) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (5, 16397.038491) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (6, 93901.57763026) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (7, -83028485) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (8, 74881) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_data VALUES (9, -24926804.045047420) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM num_data +-- !query schema +struct +-- !query output +0 0.0000000000 +1 0.0000000000 +2 -34338492.2153970470 +3 4.3100000000 +4 7799461.4119000000 +5 16397.0384910000 +6 93901.5776302600 +7 -83028485.0000000000 +8 74881.0000000000 +9 -24926804.0450474200 + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, t1.val + t2.val + FROM num_data t1, num_data t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_add t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query schema +struct +-- !query output + + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val + t2.val, 10) + FROM num_data t1, num_data t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 10) as expected + FROM num_result t1, num_exp_add t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 10) +-- !query schema +struct +-- !query output + + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, t1.val - t2.val + FROM num_data t1, num_data t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_sub t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query schema +struct +-- !query output + + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val - t2.val, 40) + FROM num_data t1, num_data t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 40) + FROM num_result t1, num_exp_sub t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 40) +-- !query schema +struct +-- !query output + + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, t1.val, t2.val, t1.val * t2.val + FROM num_data t1, num_data t2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", + "sqlState" : "21S01", + "messageParameters" : { + "dataColumns" : "`id`, `id`, `val`, `val`, `(val * val)`", + "tableColumns" : "`id1`, `id2`, `result`", + "tableName" : "`spark_catalog`.`default`.`num_result`" + } +} + + +-- !query +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_mul t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query schema +struct +-- !query output + + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val * t2.val, 30) + FROM num_data t1, num_data t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 30) as expected + FROM num_result t1, num_exp_mul t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 30) +-- !query schema +struct +-- !query output +2 2 1179132047626883.5968620000 1179132047626883.5968621359 +2 3 -147998901.4483610000 -147998901.4483612726 +2 4 -267821744976817.8111140000 -267821744976817.8111137107 +2 5 -563049578578.7692430000 -563049578578.7692425067 +2 6 -3224438592470.1844980000 -3224438592470.1844981193 +2 7 2851072985828710.4858840000 2851072985828710.4858837950 +2 8 -2571300635581.1462760000 -2571300635581.1462764070 +2 9 855948866655588.4537420000 855948866655588.4537415092 +3 2 -147998901.4483610000 -147998901.4483612726 +3 5 70671.2358960000 70671.2358962100 +3 6 404715.7995860000 404715.7995864206 +3 9 -107434525.4341540000 -107434525.4341543802 +4 2 -267821744976817.8111140000 -267821744976817.8111137107 +4 4 60831598315717.1414620000 60831598315717.1414616100 +4 5 127888068979.9935050000 127888068979.9935054429 +4 6 732381731243.7451160000 732381731243.7451157641 +4 9 -194415646271340.1815960000 -194415646271340.1815956523 +5 2 -563049578578.7692430000 -563049578578.7692425067 +5 3 70671.2358960000 70671.2358962100 +5 4 127888068979.9935050000 127888068979.9935054429 +5 5 268862871.2753360000 268862871.2753355571 +5 6 1539707782.7689980000 1539707782.7689977863 +5 9 -408725765384.2570440000 -408725765384.2570436602 +6 2 -3224438592470.1844980000 -3224438592470.1844981193 +6 3 404715.7995860000 404715.7995864206 +6 4 732381731243.7451160000 732381731243.7451157641 +6 5 1539707782.7689980000 1539707782.7689977863 +6 6 8817506281.4517450000 8817506281.4517452373 +6 7 -7796505729750.3779560000 -7796505729750.3779561000 +6 8 7031444034.5314990000 7031444034.5314990600 +6 9 -2340666225110.2992950000 -2340666225110.2992952129 +7 2 2851072985828710.4858840000 2851072985828710.4858837950 +7 6 -7796505729750.3779560000 -7796505729750.3779561000 +7 9 2069634775752159.0357590000 2069634775752159.0357587000 +8 2 -2571300635581.1462760000 -2571300635581.1462764070 +8 6 7031444034.5314990000 7031444034.5314990600 +8 9 -1866544013697.1958570000 -1866544013697.1958570200 +9 2 855948866655588.4537420000 855948866655588.4537415092 +9 3 -107434525.4341540000 -107434525.4341543802 +9 4 -194415646271340.1815960000 -194415646271340.1815956523 +9 5 -408725765384.2570440000 -408725765384.2570436602 +9 6 -2340666225110.2992950000 -2340666225110.2992952129 +9 7 2069634775752159.0357590000 2069634775752159.0357587000 +9 8 -1866544013697.1958570000 -1866544013697.1958570200 +9 9 621345559900192.4201210000 621345559900192.4201206300 + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, t1.val / t2.val + FROM num_data t1, num_data t2 + WHERE t2.val != '0.0' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_div t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query schema +struct +-- !query output +2 3 -7967167.5673780000 -7967167.5673775051 +2 4 -4.4026750000 -4.4026748005 +2 5 -2094.1886690000 -2094.1886691456 +2 6 -365.6859990000 -365.6859989148 +2 7 0.4135750000 0.4135748378 +2 8 -458.5741670000 -458.5741672173 +2 9 1.3775730000 1.3775729995 +3 2 0.0000000000 -0.0000001255 +3 4 0.0000010000 0.0000005526 +3 5 0.0002630000 0.0002628523 +3 6 0.0000460000 0.0000458991 +3 7 0.0000000000 -0.0000000519 +3 8 0.0000580000 0.0000575580 +3 9 0.0000000000 -0.0000001729 +4 2 -0.2271350000 -0.2271346500 +4 3 1809619.8171460000 1809619.8171461717 +4 5 475.6628100000 475.6628104631 +4 6 83.0599610000 83.0599613844 +4 7 -0.0939370000 -0.0939371760 +4 8 104.1580830000 104.1580829837 +4 9 -0.3128950000 -0.3128945611 +5 2 -0.0004780000 -0.0004775119 +5 3 3804.4172830000 3804.4172832947 +5 4 0.0021020000 0.0021023296 +5 6 0.1746190000 0.1746194143 +5 7 -0.0001970000 -0.0001974869 +5 8 0.2189750000 0.2189746196 +5 9 -0.0006580000 -0.0006578075 +6 2 -0.0027350000 -0.0027345865 +6 3 21786.9089630000 21786.9089629374 +6 4 0.0120390000 0.0120394951 +6 5 5.7267400000 5.7267400867 +6 7 -0.0011310000 -0.0011309562 +6 8 1.2540110000 1.2540107321 +6 9 -0.0037670000 -0.0037670925 +7 2 2.4179420000 2.4179420715 +7 3 -19264149.6519720000 -19264149.6519721578 +7 4 -10.6454130000 -10.6454126273 +7 5 -5063.6268890000 -5063.6268888173 +7 6 -884.2075620000 -884.2075617401 +7 8 -1108.8057720000 -1108.8057718246 +7 9 3.3308920000 3.3308917120 +8 2 -0.0021810000 -0.0021806723 +8 3 17373.7819030000 17373.7819025522 +8 4 0.0096010000 0.0096007911 +8 5 4.5667390000 4.5667392951 +8 6 0.7974410000 0.7974413411 +8 7 -0.0009020000 -0.0009018712 +8 9 -0.0030040000 -0.0030040353 +9 2 0.7259140000 0.7259143438 +9 3 -5783481.2169480000 -5783481.2169483573 +9 4 -3.1959650000 -3.1959647889 +9 5 -1520.2015940000 -1520.2015936432 +9 6 -265.4567120000 -265.4567119543 +9 7 0.3002200000 0.3002199070 +9 8 -332.8855660000 -332.8855656982 + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val / t2.val, 80) + FROM num_data t1, num_data t2 + WHERE t2.val != '0.0' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 80) as expected + FROM num_result t1, num_exp_div t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 80) +-- !query schema +struct +-- !query output +2 3 -7967167.5673780000 -7967167.5673775051 +2 4 -4.4026750000 -4.4026748005 +2 5 -2094.1886690000 -2094.1886691456 +2 6 -365.6859990000 -365.6859989148 +2 7 0.4135750000 0.4135748378 +2 8 -458.5741670000 -458.5741672173 +2 9 1.3775730000 1.3775729995 +3 2 0.0000000000 -0.0000001255 +3 4 0.0000010000 0.0000005526 +3 5 0.0002630000 0.0002628523 +3 6 0.0000460000 0.0000458991 +3 7 0.0000000000 -0.0000000519 +3 8 0.0000580000 0.0000575580 +3 9 0.0000000000 -0.0000001729 +4 2 -0.2271350000 -0.2271346500 +4 3 1809619.8171460000 1809619.8171461717 +4 5 475.6628100000 475.6628104631 +4 6 83.0599610000 83.0599613844 +4 7 -0.0939370000 -0.0939371760 +4 8 104.1580830000 104.1580829837 +4 9 -0.3128950000 -0.3128945611 +5 2 -0.0004780000 -0.0004775119 +5 3 3804.4172830000 3804.4172832947 +5 4 0.0021020000 0.0021023296 +5 6 0.1746190000 0.1746194143 +5 7 -0.0001970000 -0.0001974869 +5 8 0.2189750000 0.2189746196 +5 9 -0.0006580000 -0.0006578075 +6 2 -0.0027350000 -0.0027345865 +6 3 21786.9089630000 21786.9089629374 +6 4 0.0120390000 0.0120394951 +6 5 5.7267400000 5.7267400867 +6 7 -0.0011310000 -0.0011309562 +6 8 1.2540110000 1.2540107321 +6 9 -0.0037670000 -0.0037670925 +7 2 2.4179420000 2.4179420715 +7 3 -19264149.6519720000 -19264149.6519721578 +7 4 -10.6454130000 -10.6454126273 +7 5 -5063.6268890000 -5063.6268888173 +7 6 -884.2075620000 -884.2075617401 +7 8 -1108.8057720000 -1108.8057718246 +7 9 3.3308920000 3.3308917120 +8 2 -0.0021810000 -0.0021806723 +8 3 17373.7819030000 17373.7819025522 +8 4 0.0096010000 0.0096007911 +8 5 4.5667390000 4.5667392951 +8 6 0.7974410000 0.7974413411 +8 7 -0.0009020000 -0.0009018712 +8 9 -0.0030040000 -0.0030040353 +9 2 0.7259140000 0.7259143438 +9 3 -5783481.2169480000 -5783481.2169483573 +9 4 -3.1959650000 -3.1959647889 +9 5 -1520.2015940000 -1520.2015936432 +9 6 -265.4567120000 -265.4567119543 +9 7 0.3002200000 0.3002199070 +9 8 -332.8855660000 -332.8855656982 + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT id, 0, SQRT(ABS(val)) + FROM num_data +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_sqrt t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query schema +struct +-- !query output + + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT id, 0, LN(ABS(val)) + FROM num_data + WHERE val != '0.0' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_ln t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query schema +struct +-- !query output + + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT id, 0, LOG(cast('10' as decimal(38, 18)), ABS(val)) + FROM num_data + WHERE val != '0.0' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_log10 t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query schema +struct +-- !query output + + + +-- !query +TRUNCATE TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_result SELECT id, 0, POWER(cast('10' as decimal(38, 18)), LN(ABS(round(val,200)))) + FROM num_data + WHERE val != '0.0' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_power_10_ln t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query schema +struct +-- !query output +2 224790267919917440.0000000000 224790267919917955.1326161858 +4 7405685069595001.0000000000 7405685069594999.0773399947 +5 5068226527.3212630000 5068226527.3212726541 +6 281839893606.9936500000 281839893606.9937234336 +8 167361463828.0749000000 167361463828.0749132007 +9 107511333880051870.0000000000 107511333880052007.0414112467 + + +-- !query +SELECT AVG(val) FROM num_data +-- !query schema +struct +-- !query output +-13430913.59224232070000 + + +-- !query +CREATE TABLE fract_only (id int, val decimal(4,4)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO fract_only VALUES (1, 0.0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO fract_only VALUES (2, 0.1) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO fract_only VALUES (4, -0.9999) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO fract_only VALUES (5, 0.99994) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO fract_only VALUES (7, 0.00001) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO fract_only VALUES (8, 0.00017) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM fract_only +-- !query schema +struct +-- !query output +1 0.0000 +2 0.1000 +4 -0.9999 +5 0.9999 +7 0.0000 +8 0.0002 + + +-- !query +DROP TABLE fract_only +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT decimal(double('NaN')) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT decimal(double('Infinity')) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT decimal(double('-Infinity')) +-- !query schema +struct<-Infinity:decimal(10,0)> +-- !query output +NULL + + +-- !query +SELECT decimal(float('NaN')) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT decimal(float('Infinity')) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT decimal(float('-Infinity')) +-- !query schema +struct<-Infinity:decimal(10,0)> +-- !query output +NULL + + +-- !query +CREATE TABLE ceil_floor_round (a decimal(38, 18)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO ceil_floor_round VALUES (-5.5) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO ceil_floor_round VALUES (-5.499999) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO ceil_floor_round VALUES (9.5) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO ceil_floor_round VALUES (9.4999999) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO ceil_floor_round VALUES (0.0) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO ceil_floor_round VALUES (0.0000001) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO ceil_floor_round VALUES (-0.000001) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT a, ceil(a), ceiling(a), floor(a), round(a) FROM ceil_floor_round +-- !query schema +struct +-- !query output +-0.000001000000000000 0 0 -1 0 +-5.499999000000000000 -5 -5 -6 -5 +-5.500000000000000000 -5 -5 -6 -6 +0.000000000000000000 0 0 0 0 +0.000000100000000000 1 1 0 0 +9.499999900000000000 10 10 9 9 +9.500000000000000000 10 10 9 10 + + +-- !query +DROP TABLE ceil_floor_round +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT width_bucket(5.0, 3.0, 4.0, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(5.0, 3.0, 4.0, -5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(3.5, 3.0, 3.0, 888) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(double(5.0), double(3.0), double(4.0), 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(double(5.0), double(3.0), double(4.0), -5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(double(3.5), double(3.0), double(3.0), 888) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket('NaN', 3.0, 4.0, 888) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(double(0), 'NaN', double(4.0), 888) +-- !query schema +struct +-- !query output +NULL + + +-- !query +CREATE TABLE width_bucket_test (operand_num decimal(30,15), operand_f8 double) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO width_bucket_test VALUES + (-5.2, -5.2), + (-0.0000000001, -0.0000000001), + (0.000000000001, 0.000000000001), + (1, 1), + (1.99999999999999, 1.99999999999999), + (2, 2), + (2.00000000000001, 2.00000000000001), + (3, 3), + (4, 4), + (4.5, 4.5), + (5, 5), + (5.5, 5.5), + (6, 6), + (7, 7), + (8, 8), + (9, 9), + (9.99999999999999, 9.99999999999999), + (10, 10), + (10.0000000000001, 10.0000000000001) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + operand_num, + width_bucket(operand_num, 0, 10, 5) AS wb_1, + width_bucket(operand_f8, 0, 10, 5) AS wb_1f, + width_bucket(operand_num, 10, 0, 5) AS wb_2, + width_bucket(operand_f8, 10, 0, 5) AS wb_2f, + width_bucket(operand_num, 2, 8, 4) AS wb_3, + width_bucket(operand_f8, 2, 8, 4) AS wb_3f, + width_bucket(operand_num, 5.0, 5.5, 20) AS wb_4, + width_bucket(operand_f8, 5.0, 5.5, 20) AS wb_4f, + width_bucket(operand_num, -25, 25, 10) AS wb_5, + width_bucket(operand_f8, -25, 25, 10) AS wb_5f + FROM width_bucket_test + ORDER BY operand_num ASC +-- !query schema +struct +-- !query output +-5.200000000000000 0 0 6 6 0 0 0 0 4 4 +-0.000000000100000 0 0 6 6 0 0 0 0 5 5 +0.000000000001000 1 1 5 5 0 0 0 0 6 6 +1.000000000000000 1 1 5 5 0 0 0 0 6 6 +1.999999999999990 1 1 5 5 0 0 0 0 6 6 +2.000000000000000 2 2 5 5 1 1 0 0 6 6 +2.000000000000010 2 2 4 4 1 1 0 0 6 6 +3.000000000000000 2 2 4 4 1 1 0 0 6 6 +4.000000000000000 3 3 4 4 2 2 0 0 6 6 +4.500000000000000 3 3 3 3 2 2 0 0 6 6 +5.000000000000000 3 3 3 3 3 3 1 1 7 7 +5.500000000000000 3 3 3 3 3 3 21 21 7 7 +6.000000000000000 4 4 3 3 3 3 21 21 7 7 +7.000000000000000 4 4 2 2 4 4 21 21 7 7 +8.000000000000000 5 5 2 2 5 5 21 21 7 7 +9.000000000000000 5 5 1 1 5 5 21 21 7 7 +9.999999999999990 5 5 1 1 5 5 21 21 7 7 +10.000000000000000 6 6 1 1 5 5 21 21 8 8 +10.000000000000100 6 6 0 0 5 5 21 21 8 8 + + +-- !query +SELECT width_bucket(double(0.0), double('Infinity'), 5, 10) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(double(0.0), 5, double('-Infinity'), 20) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT width_bucket(double('Infinity'), 1, 10, 10), + width_bucket(double('-Infinity'), 1, 10, 10) +-- !query schema +struct +-- !query output +11 0 + + +-- !query +DROP TABLE width_bucket_test +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT '' AS to_char_3, to_char(val, '9999999999999999.999999999999999PR'), val +FROM num_data +-- !query schema +struct +-- !query output + 0.000000000000000 0.0000000000 + 0.000000000000000 0.0000000000 + 4.310000000000000 4.3100000000 + 16397.038491000000000 16397.0384910000 + 74881.000000000000000 74881.0000000000 + 93901.577630260000000 93901.5776302600 + 7799461.411900000000000 7799461.4119000000 + <24926804.045047420000000> -24926804.0450474200 + <34338492.215397047000000> -34338492.2153970470 + <83028485.000000000000000> -83028485.0000000000 + + +-- !query +SELECT '' AS to_char_4, to_char(val, '9999999999999999.999999999999999S'), val +FROM num_data +-- !query schema +struct +-- !query output + 0.000000000000000+ 0.0000000000 + 0.000000000000000+ 0.0000000000 + 4.310000000000000+ 4.3100000000 + 16397.038491000000000+ 16397.0384910000 + 74881.000000000000000+ 74881.0000000000 + 93901.577630260000000+ 93901.5776302600 + 7799461.411900000000000+ 7799461.4119000000 + 24926804.045047420000000- -24926804.0450474200 + 34338492.215397047000000- -34338492.2153970470 + 83028485.000000000000000- -83028485.0000000000 + + +-- !query +SELECT '' AS to_char_5, to_char(val, 'MI9999999999999999.999999999999999'), val FROM num_data +-- !query schema +struct +-- !query output + 0.000000000000000 0.0000000000 + 0.000000000000000 0.0000000000 + 4.310000000000000 4.3100000000 + 16397.038491000000000 16397.0384910000 + 74881.000000000000000 74881.0000000000 + 93901.577630260000000 93901.5776302600 + 7799461.411900000000000 7799461.4119000000 + -24926804.045047420000000 -24926804.0450474200 + -34338492.215397047000000 -34338492.2153970470 + -83028485.000000000000000 -83028485.0000000000 + + +-- !query +SELECT '' AS to_char_9, to_char(val, '0999999999999999.999999999999999'), val FROM num_data +-- !query schema +struct +-- !query output + 0000000000000000.000000000000000 0.0000000000 + 0000000000000000.000000000000000 0.0000000000 + 0000000000000004.310000000000000 4.3100000000 + 0000000000016397.038491000000000 16397.0384910000 + 0000000000074881.000000000000000 74881.0000000000 + 0000000000093901.577630260000000 93901.5776302600 + 0000000007799461.411900000000000 7799461.4119000000 + 0000000024926804.045047420000000 -24926804.0450474200 + 0000000034338492.215397047000000 -34338492.2153970470 + 0000000083028485.000000000000000 -83028485.0000000000 + + +-- !query +SELECT '' AS to_char_10, to_char(val, 'S0999999999999999.999999999999999'), val FROM num_data +-- !query schema +struct +-- !query output + +0000000000000000.000000000000000 0.0000000000 + +0000000000000000.000000000000000 0.0000000000 + +0000000000000004.310000000000000 4.3100000000 + +0000000000016397.038491000000000 16397.0384910000 + +0000000000074881.000000000000000 74881.0000000000 + +0000000000093901.577630260000000 93901.5776302600 + +0000000007799461.411900000000000 7799461.4119000000 + -0000000024926804.045047420000000 -24926804.0450474200 + -0000000034338492.215397047000000 -34338492.2153970470 + -0000000083028485.000000000000000 -83028485.0000000000 + + +-- !query +SELECT '' AS to_number_1, to_number('-34,338,492', '99G999G999') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_FORMAT.MISMATCH_INPUT", + "sqlState" : "42601", + "messageParameters" : { + "format" : "99G999G999", + "input" : "-34,338,492", + "inputType" : "\"STRING\"" + } +} + + +-- !query +SELECT '' AS to_number_2, to_number('-34,338,492.654,878', '99G999G999D999G999') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_FORMAT.THOUSANDS_SEPS_MUST_BEFORE_DEC", + "sqlState" : "42601", + "messageParameters" : { + "format" : "'99G999G999D999G999'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 81, + "fragment" : "to_number('-34,338,492.654,878', '99G999G999D999G999')" + } ] +} + + +-- !query +SELECT '' AS to_number_4, to_number('0.00001-', '9.999999S') +-- !query schema +struct +-- !query output + -0.000010 + + +-- !query +SELECT '' AS to_number_9, to_number('.0', '99999999.99999999') +-- !query schema +struct +-- !query output + 0.00000000 + + +-- !query +SELECT '' AS to_number_10, to_number('0', '99.99') +-- !query schema +struct +-- !query output + 0.00 + + +-- !query +SELECT '' AS to_number_12, to_number('.01-', '99.99S') +-- !query schema +struct +-- !query output + -0.01 + + +-- !query +SELECT '' AS to_number_14, to_number('34,50','999,99') +-- !query schema +struct +-- !query output + 3450 + + +-- !query +SELECT '' AS to_number_15, to_number('123,000','999G') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_FORMAT.CONT_THOUSANDS_SEPS", + "sqlState" : "42601", + "messageParameters" : { + "format" : "'999G'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 54, + "fragment" : "to_number('123,000','999G')" + } ] +} + + +-- !query +SELECT '' AS to_number_16, to_number('123456','999G999') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_FORMAT.MISMATCH_INPUT", + "sqlState" : "42601", + "messageParameters" : { + "format" : "999G999", + "input" : "123456", + "inputType" : "\"STRING\"" + } +} + + +-- !query +CREATE TABLE num_input_test (n1 decimal(38, 18)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_input_test VALUES (double(trim(' 123'))) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_input_test VALUES (double(trim(' 3245874 '))) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_input_test VALUES (double(trim(' -93853'))) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_input_test VALUES (555.50) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO num_input_test VALUES (-555.50) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM num_input_test +-- !query schema +struct +-- !query output +-555.500000000000000000 +-93853.000000000000000000 +123.000000000000000000 +3245874.000000000000000000 +555.500000000000000000 + + +-- !query +select cast(999999999999999999999 as decimal(38, 0))/1000000000000000000000 +-- !query schema +struct<(CAST(999999999999999999999 AS DECIMAL(38,0)) / 1000000000000000000000):decimal(38,6)> +-- !query output +1.000000 + + +-- !query +select div(cast(999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query schema +struct<(CAST(999999999999999999999 AS DECIMAL(38,0)) div 1000000000000000000000):bigint> +-- !query output +0 + + +-- !query +select mod(cast(999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query schema +struct +-- !query output +999999999999999999999 + + +-- !query +select div(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query schema +struct<(CAST(-9999999999999999999999 AS DECIMAL(38,0)) div 1000000000000000000000):bigint> +-- !query output +-9 + + +-- !query +select mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query schema +struct +-- !query output +-999999999999999999999 + + +-- !query +select div(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000)*1000000000000000000000 + mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query schema +struct<(((CAST(-9999999999999999999999 AS DECIMAL(38,0)) div 1000000000000000000000) * 1000000000000000000000) + mod(CAST(-9999999999999999999999 AS DECIMAL(38,0)), 1000000000000000000000)):decimal(38,0)> +-- !query output +-9999999999999999999999 + + +-- !query +select mod (70.0,70) +-- !query schema +struct +-- !query output +0.0 + + +-- !query +select div (70.0,70) +-- !query schema +struct<(70.0 div 70):bigint> +-- !query output +1 + + +-- !query +select 70.0 / 70 +-- !query schema +struct<(70.0 / 70):decimal(8,6)> +-- !query output +1.000000 + + +-- !query +select 12345678901234567890 % 123 +-- !query schema +struct<(12345678901234567890 % 123):decimal(3,0)> +-- !query output +78 + + +-- !query +select exp(0.0) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +select exp(1.0) +-- !query schema +struct +-- !query output +2.7182818284590455 + + +-- !query +select exp(32.999) +-- !query schema +struct +-- !query output +2.1442904349215556E14 + + +-- !query +select exp(-32.999) +-- !query schema +struct +-- !query output +4.663547361468238E-15 + + +-- !query +select exp(123.456) +-- !query schema +struct +-- !query output +4.132944352778106E53 + + +-- !query +select exp(-123.456) +-- !query schema +struct +-- !query output +2.4195825412645934E-54 + + +-- !query +select exp(1234.5678) +-- !query schema +struct +-- !query output +Infinity + + +-- !query +select * from range(cast(0.0 as decimal(38, 18)), cast(4.0 as decimal(38, 18))) +-- !query schema +struct +-- !query output +0 +1 +2 +3 + + +-- !query +select * from range(cast(0.1 as decimal(38, 18)), cast(4.0 as decimal(38, 18)), cast(1.3 as decimal(38, 18))) +-- !query schema +struct +-- !query output +0 +1 +2 +3 + + +-- !query +select * from range(cast(4.0 as decimal(38, 18)), cast(-1.5 as decimal(38, 18)), cast(-2.2 as decimal(38, 18))) +-- !query schema +struct +-- !query output +0 +2 +4 + + +-- !query +select ln(1.2345678e-28) +-- !query schema +struct +-- !query output +-64.26166165451762 + + +-- !query +select ln(0.0456789) +-- !query schema +struct +-- !query output +-3.0861187944847437 + + +-- !query +select ln(0.99949452) +-- !query schema +struct +-- !query output +-5.056077980832118E-4 + + +-- !query +select ln(1.00049687395) +-- !query schema +struct +-- !query output +4.967505490136803E-4 + + +-- !query +select ln(1234.567890123456789) +-- !query schema +struct +-- !query output +7.11847630129779 + + +-- !query +select ln(5.80397490724e5) +-- !query schema +struct +-- !query output +13.271468476626518 + + +-- !query +select ln(9.342536355e34) +-- !query schema +struct +-- !query output +80.52247093552418 + + +-- !query +select log(3.4634998359873254962349856073435545) +-- !query schema +struct +-- !query output +1.2422795911259166 + + +-- !query +select log(9.999999999999999999) +-- !query schema +struct +-- !query output +2.302585092994046 + + +-- !query +select log(10.00000000000000000) +-- !query schema +struct +-- !query output +2.302585092994046 + + +-- !query +select log(10.00000000000000001) +-- !query schema +struct +-- !query output +2.302585092994046 + + +-- !query +select log(590489.45235237) +-- !query schema +struct +-- !query output +13.288707052228641 + + +-- !query +select log(0.99923, 4.58934e34) +-- !query schema +struct +-- !query output +-103611.55579543479 + + +-- !query +select log(1.000016, 8.452010e18) +-- !query schema +struct +-- !query output +2723830.287707013 + + +-- !query +SELECT SUM(decimal(9999)) FROM range(1, 100001) +-- !query schema +struct +-- !query output +999900000 + + +-- !query +SELECT SUM(decimal(-9999)) FROM range(1, 100001) +-- !query schema +struct +-- !query output +-999900000 + + +-- !query +DROP TABLE num_data +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_exp_add +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_exp_sub +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_exp_div +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_exp_mul +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_exp_sqrt +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_exp_ln +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_exp_log10 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_exp_power_10_ln +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_result +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE num_input_test +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/select.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/select.sql.out new file mode 100644 index 000000000000..8e76755992c8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/select.sql.out @@ -0,0 +1,539 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create or replace temporary view onek2 as select * from onek +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view INT8_TBL as select * from values + (cast(trim(' 123 ') as bigint), cast(trim(' 456') as bigint)), + (cast(trim('123 ') as bigint),cast('4567890123456789' as bigint)), + (cast('4567890123456789' as bigint),cast('123' as bigint)), + (cast(+4567890123456789 as bigint),cast('4567890123456789' as bigint)), + (cast('+4567890123456789' as bigint),cast('-4567890123456789' as bigint)) + as INT8_TBL(q1, q2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM onek + WHERE onek.unique1 < 10 + ORDER BY onek.unique1 +-- !query schema +struct +-- !query output +0 998 0 0 0 0 0 0 0 0 0 0 1 AAAAAA KMBAAA OOOOxx +1 214 1 1 1 1 1 1 1 1 1 2 3 BAAAAA GIAAAA OOOOxx +2 326 0 2 2 2 2 2 2 2 2 4 5 CAAAAA OMAAAA OOOOxx +3 431 1 3 3 3 3 3 3 3 3 6 7 DAAAAA PQAAAA VVVVxx +4 833 0 0 4 4 4 4 4 4 4 8 9 EAAAAA BGBAAA HHHHxx +5 541 1 1 5 5 5 5 5 5 5 10 11 FAAAAA VUAAAA HHHHxx +6 978 0 2 6 6 6 6 6 6 6 12 13 GAAAAA QLBAAA OOOOxx +7 647 1 3 7 7 7 7 7 7 7 14 15 HAAAAA XYAAAA VVVVxx +8 653 0 0 8 8 8 8 8 8 8 16 17 IAAAAA DZAAAA HHHHxx +9 49 1 1 9 9 9 9 9 9 9 18 19 JAAAAA XBAAAA HHHHxx + + +-- !query +SELECT onek.unique1, onek.stringu1 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 DESC +-- !query schema +struct +-- !query output +19 TAAAAA +18 SAAAAA +17 RAAAAA +16 QAAAAA +15 PAAAAA +14 OAAAAA +13 NAAAAA +12 MAAAAA +11 LAAAAA +10 KAAAAA +9 JAAAAA +8 IAAAAA +7 HAAAAA +6 GAAAAA +5 FAAAAA +4 EAAAAA +3 DAAAAA +2 CAAAAA +1 BAAAAA +0 AAAAAA + + +-- !query +SELECT onek.unique1, onek.stringu1 FROM onek + WHERE onek.unique1 > 980 + ORDER BY stringu1 ASC +-- !query schema +struct +-- !query output +988 AMAAAA +989 BMAAAA +990 CMAAAA +991 DMAAAA +992 EMAAAA +993 FMAAAA +994 GMAAAA +995 HMAAAA +996 IMAAAA +997 JMAAAA +998 KMAAAA +999 LMAAAA +981 TLAAAA +982 ULAAAA +983 VLAAAA +984 WLAAAA +985 XLAAAA +986 YLAAAA +987 ZLAAAA + + +-- !query +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 > 980 + ORDER BY string4 ASC, unique1 DESC +-- !query schema +struct +-- !query output +999 AAAAxx +995 AAAAxx +983 AAAAxx +982 AAAAxx +981 AAAAxx +998 HHHHxx +997 HHHHxx +993 HHHHxx +990 HHHHxx +986 HHHHxx +996 OOOOxx +991 OOOOxx +988 OOOOxx +987 OOOOxx +985 OOOOxx +994 VVVVxx +992 VVVVxx +989 VVVVxx +984 VVVVxx + + +-- !query +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 > 980 + ORDER BY string4 DESC, unique1 ASC +-- !query schema +struct +-- !query output +984 VVVVxx +989 VVVVxx +992 VVVVxx +994 VVVVxx +985 OOOOxx +987 OOOOxx +988 OOOOxx +991 OOOOxx +996 OOOOxx +986 HHHHxx +990 HHHHxx +993 HHHHxx +997 HHHHxx +998 HHHHxx +981 AAAAxx +982 AAAAxx +983 AAAAxx +995 AAAAxx +999 AAAAxx + + +-- !query +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 DESC, string4 ASC +-- !query schema +struct +-- !query output +19 OOOOxx +18 VVVVxx +17 HHHHxx +16 OOOOxx +15 VVVVxx +14 AAAAxx +13 OOOOxx +12 AAAAxx +11 OOOOxx +10 AAAAxx +9 HHHHxx +8 HHHHxx +7 VVVVxx +6 OOOOxx +5 HHHHxx +4 HHHHxx +3 VVVVxx +2 OOOOxx +1 OOOOxx +0 OOOOxx + + +-- !query +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 ASC, string4 DESC +-- !query schema +struct +-- !query output +0 OOOOxx +1 OOOOxx +2 OOOOxx +3 VVVVxx +4 HHHHxx +5 HHHHxx +6 OOOOxx +7 VVVVxx +8 HHHHxx +9 HHHHxx +10 AAAAxx +11 OOOOxx +12 AAAAxx +13 OOOOxx +14 AAAAxx +15 VVVVxx +16 OOOOxx +17 HHHHxx +18 VVVVxx +19 OOOOxx + + +-- !query +SELECT onek2.* FROM onek2 WHERE onek2.unique1 < 10 +-- !query schema +struct +-- !query output +0 998 0 0 0 0 0 0 0 0 0 0 1 AAAAAA KMBAAA OOOOxx +1 214 1 1 1 1 1 1 1 1 1 2 3 BAAAAA GIAAAA OOOOxx +2 326 0 2 2 2 2 2 2 2 2 4 5 CAAAAA OMAAAA OOOOxx +3 431 1 3 3 3 3 3 3 3 3 6 7 DAAAAA PQAAAA VVVVxx +4 833 0 0 4 4 4 4 4 4 4 8 9 EAAAAA BGBAAA HHHHxx +5 541 1 1 5 5 5 5 5 5 5 10 11 FAAAAA VUAAAA HHHHxx +6 978 0 2 6 6 6 6 6 6 6 12 13 GAAAAA QLBAAA OOOOxx +7 647 1 3 7 7 7 7 7 7 7 14 15 HAAAAA XYAAAA VVVVxx +8 653 0 0 8 8 8 8 8 8 8 16 17 IAAAAA DZAAAA HHHHxx +9 49 1 1 9 9 9 9 9 9 9 18 19 JAAAAA XBAAAA HHHHxx + + +-- !query +SELECT onek2.unique1, onek2.stringu1 FROM onek2 + WHERE onek2.unique1 < 20 + ORDER BY unique1 DESC +-- !query schema +struct +-- !query output +19 TAAAAA +18 SAAAAA +17 RAAAAA +16 QAAAAA +15 PAAAAA +14 OAAAAA +13 NAAAAA +12 MAAAAA +11 LAAAAA +10 KAAAAA +9 JAAAAA +8 IAAAAA +7 HAAAAA +6 GAAAAA +5 FAAAAA +4 EAAAAA +3 DAAAAA +2 CAAAAA +1 BAAAAA +0 AAAAAA + + +-- !query +SELECT onek2.unique1, onek2.stringu1 FROM onek2 + WHERE onek2.unique1 > 980 +-- !query schema +struct +-- !query output +981 TLAAAA +982 ULAAAA +983 VLAAAA +984 WLAAAA +985 XLAAAA +986 YLAAAA +987 ZLAAAA +988 AMAAAA +989 BMAAAA +990 CMAAAA +991 DMAAAA +992 EMAAAA +993 FMAAAA +994 GMAAAA +995 HMAAAA +996 IMAAAA +997 JMAAAA +998 KMAAAA +999 LMAAAA + + +-- !query +CREATE TABLE tmp USING parquet AS +SELECT two, stringu1, ten, string4 +FROM onek +-- !query schema +struct<> +-- !query output + + + +-- !query +select foo.* from (select 1) as foo +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +select foo.* from (select null) as foo +-- !query schema +struct +-- !query output +NULL + + +-- !query +select foo.* from (select 'xyzzy',1,null) as foo +-- !query schema +struct +-- !query output +xyzzy 1 NULL + + +-- !query +select * from onek, values(147, 'RFAAAA'), (931, 'VJAAAA') as v (i, j) + WHERE onek.unique1 = v.i and onek.stringu1 = v.j +-- !query schema +struct +-- !query output +147 0 1 3 7 7 7 47 147 147 147 14 15 RFAAAA AAAAAA AAAAxx 147 RFAAAA +931 1 1 3 1 11 1 31 131 431 931 2 3 VJAAAA BAAAAA HHHHxx 931 VJAAAA + + +-- !query +VALUES (1,2), (3,4+4), (7,77.7) +-- !query schema +struct +-- !query output +1 2.0 +3 8.0 +7 77.7 + + +-- !query +VALUES (1,2), (3,4+4), (7,77.7) +UNION ALL +SELECT 2+2, 57 +UNION ALL +TABLE int8_tbl +-- !query schema +struct +-- !query output +1 2.0 +123 456.0 +123 4567890123456789.0 +3 8.0 +4 57.0 +4567890123456789 -4567890123456789.0 +4567890123456789 123.0 +4567890123456789 4567890123456789.0 +7 77.7 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW foo AS +SELECT * FROM (values(42),(3),(10),(7),(null),(null),(1)) as foo (f1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM foo ORDER BY f1 +-- !query schema +struct +-- !query output +NULL +NULL +1 +3 +7 +10 +42 + + +-- !query +SELECT * FROM foo ORDER BY f1 ASC +-- !query schema +struct +-- !query output +NULL +NULL +1 +3 +7 +10 +42 + + +-- !query +SELECT * FROM foo ORDER BY f1 NULLS FIRST +-- !query schema +struct +-- !query output +NULL +NULL +1 +3 +7 +10 +42 + + +-- !query +SELECT * FROM foo ORDER BY f1 DESC +-- !query schema +struct +-- !query output +42 +10 +7 +3 +1 +NULL +NULL + + +-- !query +SELECT * FROM foo ORDER BY f1 DESC NULLS LAST +-- !query schema +struct +-- !query output +42 +10 +7 +3 +1 +NULL +NULL + + +-- !query +select * from onek2 where unique2 = 11 and stringu1 = 'ATAAAA' +-- !query schema +struct +-- !query output +494 11 0 2 4 14 4 94 94 494 494 8 9 ATAAAA LAAAAA VVVVxx + + +-- !query +select unique2 from onek2 where unique2 = 11 and stringu1 = 'ATAAAA' +-- !query schema +struct +-- !query output +11 + + +-- !query +select * from onek2 where unique2 = 11 and stringu1 < 'B' +-- !query schema +struct +-- !query output +494 11 0 2 4 14 4 94 94 494 494 8 9 ATAAAA LAAAAA VVVVxx + + +-- !query +select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' +-- !query schema +struct +-- !query output +11 + + +-- !query +select unique2 from onek2 where unique2 = 11 and stringu1 < 'C' +-- !query schema +struct +-- !query output +11 + + +-- !query +select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' +-- !query schema +struct +-- !query output +11 + + +-- !query +select unique1, unique2 from onek2 + where (unique2 = 11 or unique1 = 0) and stringu1 < 'B' +-- !query schema +struct +-- !query output +0 998 +494 11 + + +-- !query +select unique1, unique2 from onek2 + where (unique2 = 11 and stringu1 < 'B') or unique1 = 0 +-- !query schema +struct +-- !query output +0 998 +494 11 + + +-- !query +SELECT 1 AS x ORDER BY x +-- !query schema +struct +-- !query output +1 + + +-- !query +select * from (values (2),(null),(1)) v(k) where k = k order by k +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +select * from (values (2),(null),(1)) v(k) where k = k +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +drop table tmp +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/select_distinct.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/select_distinct.sql.out new file mode 100644 index 000000000000..9bf31ae1a94d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/select_distinct.sql.out @@ -0,0 +1,222 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW tmp AS +SELECT two, stringu1, ten, string4 +FROM onek +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT DISTINCT two FROM tmp ORDER BY 1 +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +SELECT DISTINCT ten FROM tmp ORDER BY 1 +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +SELECT DISTINCT string4 FROM tmp ORDER BY 1 +-- !query schema +struct +-- !query output +AAAAxx +HHHHxx +OOOOxx +VVVVxx + + +-- !query +SELECT DISTINCT two, string4, ten + FROM tmp + ORDER BY two ASC, string4 ASC, ten ASC +-- !query schema +struct +-- !query output +0 AAAAxx 0 +0 AAAAxx 2 +0 AAAAxx 4 +0 AAAAxx 6 +0 AAAAxx 8 +0 HHHHxx 0 +0 HHHHxx 2 +0 HHHHxx 4 +0 HHHHxx 6 +0 HHHHxx 8 +0 OOOOxx 0 +0 OOOOxx 2 +0 OOOOxx 4 +0 OOOOxx 6 +0 OOOOxx 8 +0 VVVVxx 0 +0 VVVVxx 2 +0 VVVVxx 4 +0 VVVVxx 6 +0 VVVVxx 8 +1 AAAAxx 1 +1 AAAAxx 3 +1 AAAAxx 5 +1 AAAAxx 7 +1 AAAAxx 9 +1 HHHHxx 1 +1 HHHHxx 3 +1 HHHHxx 5 +1 HHHHxx 7 +1 HHHHxx 9 +1 OOOOxx 1 +1 OOOOxx 3 +1 OOOOxx 5 +1 OOOOxx 7 +1 OOOOxx 9 +1 VVVVxx 1 +1 VVVVxx 3 +1 VVVVxx 5 +1 VVVVxx 7 +1 VVVVxx 9 + + +-- !query +SELECT count(*) FROM + (SELECT DISTINCT two, four, two FROM tenk1) ss +-- !query schema +struct +-- !query output +4 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW disttable AS SELECT * FROM + (VALUES (1), (2), (3), (NULL)) + AS v(f1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT f1, f1 IS DISTINCT FROM 2 as `not 2` FROM disttable +-- !query schema +struct +-- !query output +1 true +2 false +3 true +NULL true + + +-- !query +SELECT f1, f1 IS DISTINCT FROM NULL as `not null` FROM disttable +-- !query schema +struct +-- !query output +1 true +2 true +3 true +NULL false + + +-- !query +SELECT f1, f1 IS DISTINCT FROM f1 as `false` FROM disttable +-- !query schema +struct +-- !query output +1 false +2 false +3 false +NULL false + + +-- !query +SELECT f1, f1 IS DISTINCT FROM f1+1 as `not null` FROM disttable +-- !query schema +struct +-- !query output +1 true +2 true +3 true +NULL false + + +-- !query +SELECT 1 IS DISTINCT FROM 2 as `yes` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 2 IS DISTINCT FROM 2 as `no` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 2 IS DISTINCT FROM null as `yes` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT null IS DISTINCT FROM null as `no` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 1 IS NOT DISTINCT FROM 2 as `no` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 2 IS NOT DISTINCT FROM 2 as `yes` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 2 IS NOT DISTINCT FROM null as `no` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT null IS NOT DISTINCT FROM null as `yes` +-- !query schema +struct +-- !query output +true diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/select_having.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/select_having.sql.out new file mode 100644 index 000000000000..dc295b0578b0 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/select_having.sql.out @@ -0,0 +1,222 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE test_having (a int, b int, c string, d string) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (0, 1, 'XXXX', 'A') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (1, 2, 'AAAA', 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (2, 2, 'AAAA', 'c') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (3, 3, 'BBBB', 'D') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (4, 3, 'BBBB', 'e') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (5, 3, 'bbbb', 'F') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (6, 4, 'cccc', 'g') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (7, 4, 'cccc', 'h') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (8, 4, 'CCCC', 'I') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (9, 4, 'CCCC', 'j') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT b, c FROM test_having + GROUP BY b, c HAVING count(*) = 1 ORDER BY b, c +-- !query schema +struct +-- !query output +1 XXXX +3 bbbb + + +-- !query +SELECT b, c FROM test_having + GROUP BY b, c HAVING b = 3 ORDER BY b, c +-- !query schema +struct +-- !query output +3 BBBB +3 bbbb + + +-- !query +SELECT c, max(a) FROM test_having + GROUP BY c HAVING count(*) > 2 OR min(a) = max(a) + ORDER BY c +-- !query schema +struct +-- !query output +XXXX 0 +bbbb 5 + + +-- !query +SELECT min(a), max(a) FROM test_having HAVING min(a) = max(a) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT min(a), max(a) FROM test_having HAVING min(a) < max(a) +-- !query schema +struct +-- !query output +0 9 + + +-- !query +SELECT a FROM test_having HAVING min(a) < max(a) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 48, + "fragment" : "SELECT a FROM test_having HAVING min(a) < max(a)" + } ] +} + + +-- !query +SELECT 1 AS one FROM test_having HAVING a > 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`", + "proposal" : "`one`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 41, + "stopIndex" : 41, + "fragment" : "a" + } ] +} + + +-- !query +SELECT 1 AS one FROM test_having HAVING 1 > 2 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 1 AS one FROM test_having HAVING 1 < 2 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT 1 AS one FROM test_having WHERE 1/a = 1 HAVING 1 < 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 40, + "stopIndex" : 42, + "fragment" : "1/a" + } ] +} + + +-- !query +DROP TABLE test_having +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/select_implicit.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/select_implicit.sql.out new file mode 100755 index 000000000000..f0c283cb4036 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/select_implicit.sql.out @@ -0,0 +1,506 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE test_missing_target (a int, b int, c string, d string) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (0, 1, 'XXXX', 'A') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (1, 2, 'ABAB', 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (2, 2, 'ABAB', 'c') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (3, 3, 'BBBB', 'D') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (4, 3, 'BBBB', 'e') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (5, 3, 'bbbb', 'F') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (6, 4, 'cccc', 'g') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (7, 4, 'cccc', 'h') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (8, 4, 'CCCC', 'I') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (9, 4, 'CCCC', 'j') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT c, count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c +-- !query schema +struct +-- !query output +ABAB 2 +BBBB 2 +CCCC 2 +XXXX 1 +bbbb 1 +cccc 2 + + +-- !query +SELECT count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c +-- !query schema +struct +-- !query output +2 +2 +2 +1 +1 +2 + + +-- !query +SELECT count(*) FROM test_missing_target GROUP BY a ORDER BY b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`b`", + "proposal" : "`count(1)`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 62, + "stopIndex" : 62, + "fragment" : "b" + } ] +} + + +-- !query +SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b +-- !query schema +struct +-- !query output +1 +2 +3 +4 + + +-- !query +SELECT test_missing_target.b, count(*) + FROM test_missing_target GROUP BY b ORDER BY b +-- !query schema +struct +-- !query output +1 1 +2 2 +3 3 +4 4 + + +-- !query +SELECT c FROM test_missing_target ORDER BY a +-- !query schema +struct +-- !query output +XXXX +ABAB +ABAB +BBBB +BBBB +bbbb +cccc +cccc +CCCC +CCCC + + +-- !query +SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b desc +-- !query schema +struct +-- !query output +4 +3 +2 +1 + + +-- !query +SELECT count(*) FROM test_missing_target ORDER BY 1 desc +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT c, count(*) FROM test_missing_target GROUP BY 1 ORDER BY 1 +-- !query schema +struct +-- !query output +ABAB 2 +BBBB 2 +CCCC 2 +XXXX 1 +bbbb 1 +cccc 2 + + +-- !query +SELECT c, count(*) FROM test_missing_target GROUP BY 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "3", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 54, + "stopIndex" : 54, + "fragment" : "3" + } ] +} + + +-- !query +SELECT count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY b ORDER BY b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`b`", + "referenceNames" : "[`x`.`b`, `y`.`b`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 94, + "stopIndex" : 94, + "fragment" : "b" + } ] +} + + +-- !query +SELECT a, a FROM test_missing_target + ORDER BY a +-- !query schema +struct +-- !query output +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 + + +-- !query +SELECT a/2, a/2 FROM test_missing_target + ORDER BY a/2 +-- !query schema +struct<(a / 2):double,(a / 2):double> +-- !query output +0.0 0.0 +0.5 0.5 +1.0 1.0 +1.5 1.5 +2.0 2.0 +2.5 2.5 +3.0 3.0 +3.5 3.5 +4.0 4.0 +4.5 4.5 + + +-- !query +SELECT a/2, a/2 FROM test_missing_target + GROUP BY a/2 ORDER BY a/2 +-- !query schema +struct<(a / 2):double,(a / 2):double> +-- !query output +0.0 0.0 +0.5 0.5 +1.0 1.0 +1.5 1.5 +2.0 2.0 +2.5 2.5 +3.0 3.0 +3.5 3.5 +4.0 4.0 +4.5 4.5 + + +-- !query +SELECT x.b, count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b ORDER BY x.b +-- !query schema +struct +-- !query output +1 1 +2 2 +3 3 +4 4 + + +-- !query +SELECT count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b ORDER BY x.b +-- !query schema +struct +-- !query output +1 +2 +3 +4 + + +-- !query +SELECT a%2, count(b) FROM test_missing_target +GROUP BY test_missing_target.a%2 +ORDER BY test_missing_target.a%2 +-- !query schema +struct<(a % 2):int,count(b):bigint> +-- !query output +0 5 +1 5 + + +-- !query +SELECT count(c) FROM test_missing_target +GROUP BY lower(test_missing_target.c) +ORDER BY lower(test_missing_target.c) +-- !query schema +struct +-- !query output +2 +3 +4 +1 + + +-- !query +SELECT count(a) FROM test_missing_target GROUP BY a ORDER BY b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`b`", + "proposal" : "`count(a)`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 62, + "stopIndex" : 62, + "fragment" : "b" + } ] +} + + +-- !query +SELECT count(b) FROM test_missing_target GROUP BY b/2 ORDER BY b/2 +-- !query schema +struct +-- !query output +1 +2 +3 +4 + + +-- !query +SELECT lower(test_missing_target.c), count(c) + FROM test_missing_target GROUP BY lower(c) ORDER BY lower(c) +-- !query schema +struct +-- !query output +abab 2 +bbbb 3 +cccc 4 +xxxx 1 + + +-- !query +SELECT a FROM test_missing_target ORDER BY upper(d) +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +SELECT count(b) FROM test_missing_target + GROUP BY (b + 1) / 2 ORDER BY (b + 1) / 2 desc +-- !query schema +struct +-- !query output +4 +3 +2 +1 + + +-- !query +SELECT count(x.a) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY b/2 ORDER BY b/2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`b`", + "referenceNames" : "[`x`.`b`, `y`.`b`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 96, + "stopIndex" : 96, + "fragment" : "b" + } ] +} + + +-- !query +SELECT x.b/2, count(x.b) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b/2 ORDER BY x.b/2 +-- !query schema +struct<(b / 2):double,count(b):bigint> +-- !query output +0.5 1 +1.0 2 +1.5 3 +2.0 4 + + +-- !query +SELECT count(b) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b/2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`b`", + "referenceNames" : "[`x`.`b`, `y`.`b`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 14, + "stopIndex" : 14, + "fragment" : "b" + } ] +} + + +-- !query +DROP TABLE test_missing_target +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/strings.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/strings.sql.out new file mode 100644 index 000000000000..b4442a767079 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/strings.sql.out @@ -0,0 +1,1067 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT 'first line' +' - next line' + ' - third line' + AS `Three lines to one` +-- !query schema +struct +-- !query output +first line - next line - third line + + +-- !query +SELECT 'first line' +' - next line' /* this comment is not allowed here */ +' - third line' + AS `Illegal comment within continuation` +-- !query schema +struct +-- !query output +first line - next line - third line + + +-- !query +SELECT binary('\\xDeAdBeEf') +-- !query schema +struct<\xDeAdBeEf:binary> +-- !query output +\xDeAdBeEf + + +-- !query +SELECT binary('\\x De Ad Be Ef ') +-- !query schema +struct<\x De Ad Be Ef :binary> +-- !query output +\x De Ad Be Ef + + +-- !query +SELECT binary('\\xDe00BeEf') +-- !query schema +struct<\xDe00BeEf:binary> +-- !query output +\xDe00BeEf + + +-- !query +SELECT binary('DeAdBeEf') +-- !query schema +struct +-- !query output +DeAdBeEf + + +-- !query +SELECT binary('De\\000dBeEf') +-- !query schema +struct +-- !query output +De\000dBeEf + + +-- !query +SELECT binary('De\\123dBeEf') +-- !query schema +struct +-- !query output +De\123dBeEf + + +-- !query +SELECT TRIM(BOTH FROM ' bunch o blanks ') = 'bunch o blanks' AS `bunch o blanks` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT TRIM(LEADING FROM ' bunch o blanks ') = 'bunch o blanks ' AS `bunch o blanks ` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT TRIM(TRAILING FROM ' bunch o blanks ') = ' bunch o blanks' AS ` bunch o blanks` +-- !query schema +struct< bunch o blanks:boolean> +-- !query output +true + + +-- !query +SELECT TRIM(BOTH 'x' FROM 'xxxxxsome Xsxxxxx') = 'some Xs' AS `some Xs` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT SUBSTRING('1234567890' FROM 3) = '34567890' AS `34567890` +-- !query schema +struct<34567890:boolean> +-- !query output +true + + +-- !query +SELECT SUBSTRING('1234567890' FROM 4 FOR 3) = '456' AS `456` +-- !query schema +struct<456:boolean> +-- !query output +true + + +-- !query +SELECT POSITION('4' IN '1234567890') = '4' AS `4` +-- !query schema +struct<4:boolean> +-- !query output +true + + +-- !query +SELECT POSITION('5' IN '1234567890') = '5' AS `5` +-- !query schema +struct<5:boolean> +-- !query output +true + + +-- !query +SELECT OVERLAY('abcdef' PLACING '45' FROM 4) AS `abc45f` +-- !query schema +struct +-- !query output +abc45f + + +-- !query +SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5) AS `yabadaba` +-- !query schema +struct +-- !query output +yabadaba + + +-- !query +SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5 FOR 0) AS `yabadabadoo` +-- !query schema +struct +-- !query output +yabadabadoo + + +-- !query +SELECT OVERLAY('babosa' PLACING 'ubb' FROM 2 FOR 4) AS `bubba` +-- !query schema +struct +-- !query output +bubba + + +-- !query +SELECT 'hawkeye' LIKE 'h%' AS `true` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'hawkeye' NOT LIKE 'h%' AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 'hawkeye' LIKE 'H%' AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 'hawkeye' NOT LIKE 'H%' AS `true` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'hawkeye' LIKE 'indio%' AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 'hawkeye' NOT LIKE 'indio%' AS `true` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'hawkeye' LIKE 'h%eye' AS `true` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'hawkeye' NOT LIKE 'h%eye' AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 'indio' LIKE '_ndio' AS `true` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'indio' NOT LIKE '_ndio' AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 'indio' LIKE 'in__o' AS `true` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'indio' NOT LIKE 'in__o' AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 'indio' LIKE 'in_o' AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 'indio' NOT LIKE 'in_o' AS `true` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'hawkeye' LIKE 'h%' ESCAPE '#' AS `true` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'hawkeye' NOT LIKE 'h%' ESCAPE '#' AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 'indio' LIKE 'ind_o' ESCAPE '$' AS `true` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'indio' NOT LIKE 'ind_o' ESCAPE '$' AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 'h%' LIKE 'h#%' ESCAPE '#' AS `true` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'h%' NOT LIKE 'h#%' ESCAPE '#' AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 'h%wkeye' LIKE 'h#%' ESCAPE '#' AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 'h%wkeye' NOT LIKE 'h#%' ESCAPE '#' AS `true` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'h%wkeye' LIKE 'h#%%' ESCAPE '#' AS `true` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'h%wkeye' NOT LIKE 'h#%%' ESCAPE '#' AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 'h%awkeye' LIKE 'h#%a%k%e' ESCAPE '#' AS `true` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'h%awkeye' NOT LIKE 'h#%a%k%e' ESCAPE '#' AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 'indio' LIKE '_ndio' ESCAPE '$' AS `true` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'indio' NOT LIKE '_ndio' ESCAPE '$' AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 'i_dio' LIKE 'i$_d_o' ESCAPE '$' AS `true` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'i_dio' NOT LIKE 'i$_d_o' ESCAPE '$' AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 'i_dio' LIKE 'i$_nd_o' ESCAPE '$' AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 'i_dio' NOT LIKE 'i$_nd_o' ESCAPE '$' AS `true` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'i_dio' LIKE 'i$_d%o' ESCAPE '$' AS `true` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'i_dio' NOT LIKE 'i$_d%o' ESCAPE '$' AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 'maca' LIKE 'm%aca' ESCAPE '%' AS `true` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_FORMAT.ESC_IN_THE_MIDDLE", + "sqlState" : "42601", + "messageParameters" : { + "char" : "'a'", + "format" : "'m%aca'" + } +} + + +-- !query +SELECT 'maca' NOT LIKE 'm%aca' ESCAPE '%' AS `false` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_FORMAT.ESC_IN_THE_MIDDLE", + "sqlState" : "42601", + "messageParameters" : { + "char" : "'a'", + "format" : "'m%aca'" + } +} + + +-- !query +SELECT 'ma%a' LIKE 'm%a%%a' ESCAPE '%' AS `true` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_FORMAT.ESC_IN_THE_MIDDLE", + "sqlState" : "42601", + "messageParameters" : { + "char" : "'a'", + "format" : "'m%a%%a'" + } +} + + +-- !query +SELECT 'ma%a' NOT LIKE 'm%a%%a' ESCAPE '%' AS `false` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_FORMAT.ESC_IN_THE_MIDDLE", + "sqlState" : "42601", + "messageParameters" : { + "char" : "'a'", + "format" : "'m%a%%a'" + } +} + + +-- !query +SELECT 'bear' LIKE 'b_ear' ESCAPE '_' AS `true` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_FORMAT.ESC_IN_THE_MIDDLE", + "sqlState" : "42601", + "messageParameters" : { + "char" : "'e'", + "format" : "'b_ear'" + } +} + + +-- !query +SELECT 'bear' NOT LIKE 'b_ear' ESCAPE '_' AS `false` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_FORMAT.ESC_IN_THE_MIDDLE", + "sqlState" : "42601", + "messageParameters" : { + "char" : "'e'", + "format" : "'b_ear'" + } +} + + +-- !query +SELECT 'be_r' LIKE 'b_e__r' ESCAPE '_' AS `true` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_FORMAT.ESC_IN_THE_MIDDLE", + "sqlState" : "42601", + "messageParameters" : { + "char" : "'e'", + "format" : "'b_e__r'" + } +} + + +-- !query +SELECT 'be_r' NOT LIKE 'b_e__r' ESCAPE '_' AS `false` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_FORMAT.ESC_IN_THE_MIDDLE", + "sqlState" : "42601", + "messageParameters" : { + "char" : "'e'", + "format" : "'b_e__r'" + } +} + + +-- !query +SELECT 'be_r' LIKE '__e__r' ESCAPE '_' AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT 'be_r' NOT LIKE '__e__r' ESCAPE '_' AS `true` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'foo' LIKE '_%' as t, 'f' LIKE '_%' as t, '' LIKE '_%' as f +-- !query schema +struct +-- !query output +true true false + + +-- !query +SELECT 'foo' LIKE '%_' as t, 'f' LIKE '%_' as t, '' LIKE '%_' as f +-- !query schema +struct +-- !query output +true true false + + +-- !query +SELECT 'foo' LIKE '__%' as t, 'foo' LIKE '___%' as t, 'foo' LIKE '____%' as f +-- !query schema +struct +-- !query output +true true false + + +-- !query +SELECT 'foo' LIKE '%__' as t, 'foo' LIKE '%___' as t, 'foo' LIKE '%____' as f +-- !query schema +struct +-- !query output +true true false + + +-- !query +SELECT 'jack' LIKE '%____%' AS t +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT 'unknown' || ' and unknown' AS `Concat unknown types` +-- !query schema +struct +-- !query output +unknown and unknown + + +-- !query +SELECT string('text') || ' and unknown' AS `Concat text to unknown type` +-- !query schema +struct +-- !query output +text and unknown + + +-- !query +CREATE TABLE toasttest(f1 string) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into toasttest values(repeat('1234567890',10000)) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into toasttest values(repeat('1234567890',10000)) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into toasttest values(repeat('1234567890',10000)) +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into toasttest values(repeat('1234567890',10000)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT substr(f1, 99995) from toasttest +-- !query schema +struct +-- !query output +567890 +567890 +567890 +567890 + + +-- !query +SELECT substr(f1, 99995, 10) from toasttest +-- !query schema +struct +-- !query output +567890 +567890 +567890 +567890 + + +-- !query +SELECT length('abcdef') AS `length_6` +-- !query schema +struct +-- !query output +6 + + +-- !query +SELECT position('cd', 'abcdef') AS `pos_3` +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT position('xy', 'abcdef') AS `pos_0` +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT replace('abcdef', 'de', '45') AS `abc45f` +-- !query schema +struct +-- !query output +abc45f + + +-- !query +SELECT replace('yabadabadoo', 'ba', '123') AS `ya123da123doo` +-- !query schema +struct +-- !query output +ya123da123doo + + +-- !query +SELECT replace('yabadoo', 'bad', '') AS `yaoo` +-- !query schema +struct +-- !query output +yaoo + + +-- !query +select hex(256*256*256 - 1) AS `ffffff` +-- !query schema +struct +-- !query output +FFFFFF + + +-- !query +select hex(bigint(bigint(bigint(bigint(256)*256)*256)*256) - 1) AS `ffffffff` +-- !query schema +struct +-- !query output +FFFFFFFF + + +-- !query +select md5('') = 'd41d8cd98f00b204e9800998ecf8427e' AS `TRUE` +-- !query schema +struct +-- !query output +true + + +-- !query +select md5('a') = '0cc175b9c0f1b6a831c399e269772661' AS `TRUE` +-- !query schema +struct +-- !query output +true + + +-- !query +select md5('abc') = '900150983cd24fb0d6963f7d28e17f72' AS `TRUE` +-- !query schema +struct +-- !query output +true + + +-- !query +select md5('message digest') = 'f96b697d7cb7938d525a2f31aaf161d0' AS `TRUE` +-- !query schema +struct +-- !query output +true + + +-- !query +select md5('abcdefghijklmnopqrstuvwxyz') = 'c3fcd3d76192e4007dfb496cca67e13b' AS `TRUE` +-- !query schema +struct +-- !query output +true + + +-- !query +select md5('ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789') = 'd174ab98d277d9f5a5611c2c9f419d9f' AS `TRUE` +-- !query schema +struct +-- !query output +true + + +-- !query +select md5('12345678901234567890123456789012345678901234567890123456789012345678901234567890') = '57edf4a22be3c955ac49da2e2107b67a' AS `TRUE` +-- !query schema +struct +-- !query output +true + + +-- !query +select md5(binary('')) = 'd41d8cd98f00b204e9800998ecf8427e' AS `TRUE` +-- !query schema +struct +-- !query output +true + + +-- !query +select md5(binary('a')) = '0cc175b9c0f1b6a831c399e269772661' AS `TRUE` +-- !query schema +struct +-- !query output +true + + +-- !query +select md5(binary('abc')) = '900150983cd24fb0d6963f7d28e17f72' AS `TRUE` +-- !query schema +struct +-- !query output +true + + +-- !query +select md5(binary('message digest')) = 'f96b697d7cb7938d525a2f31aaf161d0' AS `TRUE` +-- !query schema +struct +-- !query output +true + + +-- !query +select md5(binary('abcdefghijklmnopqrstuvwxyz')) = 'c3fcd3d76192e4007dfb496cca67e13b' AS `TRUE` +-- !query schema +struct +-- !query output +true + + +-- !query +select md5(binary('ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789')) = 'd174ab98d277d9f5a5611c2c9f419d9f' AS `TRUE` +-- !query schema +struct +-- !query output +true + + +-- !query +select md5(binary('12345678901234567890123456789012345678901234567890123456789012345678901234567890')) = '57edf4a22be3c955ac49da2e2107b67a' AS `TRUE` +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT initcap('hi THOMAS') +-- !query schema +struct +-- !query output +Hi Thomas + + +-- !query +SELECT lpad('hi', 5, 'xy') +-- !query schema +struct +-- !query output +xyxhi + + +-- !query +SELECT lpad('hi', 5) +-- !query schema +struct +-- !query output + hi + + +-- !query +SELECT lpad('hi', -5, 'xy') +-- !query schema +struct +-- !query output + + + +-- !query +SELECT lpad('hello', 2) +-- !query schema +struct +-- !query output +he + + +-- !query +SELECT lpad('hi', 5, '') +-- !query schema +struct +-- !query output +hi + + +-- !query +SELECT rpad('hi', 5, 'xy') +-- !query schema +struct +-- !query output +hixyx + + +-- !query +SELECT rpad('hi', 5) +-- !query schema +struct +-- !query output +hi + + +-- !query +SELECT rpad('hi', -5, 'xy') +-- !query schema +struct +-- !query output + + + +-- !query +SELECT rpad('hello', 2) +-- !query schema +struct +-- !query output +he + + +-- !query +SELECT rpad('hi', 5, '') +-- !query schema +struct +-- !query output +hi + + +-- !query +SELECT translate('', '14', 'ax') +-- !query schema +struct +-- !query output + + + +-- !query +SELECT translate('12345', '14', 'ax') +-- !query schema +struct +-- !query output +a23x5 + + +-- !query +SELECT ascii('x') +-- !query schema +struct +-- !query output +120 + + +-- !query +SELECT ascii('') +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT chr(65) +-- !query schema +struct +-- !query output +A + + +-- !query +SELECT chr(0) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT repeat('Pg', 4) +-- !query schema +struct +-- !query output +PgPgPgPg + + +-- !query +SELECT repeat('Pg', -4) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT trim(binary('\\000') from binary('\\000Tom\\000')) +-- !query schema +struct +-- !query output +Tom + + +-- !query +SELECT btrim(binary('\\000trim\\000'), binary('\\000')) +-- !query schema +struct +-- !query output +trim + + +-- !query +SELECT btrim(binary(''), binary('\\000')) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT btrim(binary('\\000trim\\000'), binary('')) +-- !query schema +struct +-- !query output +\000trim\000 + + +-- !query +DROP TABLE toasttest +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/text.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/text.sql.out new file mode 100755 index 000000000000..37b8a3e8fd19 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/text.sql.out @@ -0,0 +1,405 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT string('this is a text string') = string('this is a text string') AS true +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT string('this is a text string') = string('this is a text strin') AS `false` +-- !query schema +struct +-- !query output +false + + +-- !query +CREATE TABLE TEXT_TBL (f1 string) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO TEXT_TBL VALUES ('doh!') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO TEXT_TBL VALUES ('hi de ho neighbor') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT '' AS two, * FROM TEXT_TBL +-- !query schema +struct +-- !query output + doh! + hi de ho neighbor + + +-- !query +select length(42) +-- !query schema +struct +-- !query output +2 + + +-- !query +select string('four: ') || 2+2 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'four: 2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "string('four: ') || 2+2" + } ] +} + + +-- !query +select 'four: ' || 2+2 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'four: 2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "'four: ' || 2+2" + } ] +} + + +-- !query +select 3 || 4.0 +-- !query schema +struct +-- !query output +34.0 + + +-- !query +/* + * various string functions + */ +select concat('one') +-- !query schema +struct +-- !query output +one + + +-- !query +select concat(1,2,3,'hello',true, false, to_date('20100309','yyyyMMdd')) +-- !query schema +struct +-- !query output +123hellotruefalse2010-03-09 + + +-- !query +select concat_ws('#','one') +-- !query schema +struct +-- !query output +one + + +-- !query +select concat_ws('#',1,2,3,'hello',true, false, to_date('20100309','yyyyMMdd')) +-- !query schema +struct +-- !query output +1#x#x#hello#true#false#x-03-09 + + +-- !query +select concat_ws(',',10,20,null,30) +-- !query schema +struct +-- !query output +10,20,30 + + +-- !query +select concat_ws('',10,20,null,30) +-- !query schema +struct +-- !query output +102030 + + +-- !query +select concat_ws(NULL,10,20,null,30) is null +-- !query schema +struct<(concat_ws(NULL, 10, 20, NULL, 30) IS NULL):boolean> +-- !query output +true + + +-- !query +select reverse('abcde') +-- !query schema +struct +-- !query output +edcba + + +-- !query +select i, left('ahoj', i), right('ahoj', i) from range(-5, 6) t(i) order by i +-- !query schema +struct +-- !query output +-5 +-4 +-3 +-2 +-1 +0 +1 a j +2 ah oj +3 aho hoj +4 ahoj ahoj +5 ahoj ahoj + + +-- !query +/* + * format + */ +select format_string(NULL) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select format_string('Hello') +-- !query schema +struct +-- !query output +Hello + + +-- !query +select format_string('Hello %s', 'World') +-- !query schema +struct +-- !query output +Hello World + + +-- !query +select format_string('Hello %%') +-- !query schema +struct +-- !query output +Hello % + + +-- !query +select format_string('Hello %%%%') +-- !query schema +struct +-- !query output +Hello %% + + +-- !query +select format_string('Hello %s %s', 'World') +-- !query schema +struct<> +-- !query output +java.util.MissingFormatArgumentException +Format specifier '%s' + + +-- !query +select format_string('Hello %s') +-- !query schema +struct<> +-- !query output +java.util.MissingFormatArgumentException +Format specifier '%s' + + +-- !query +select format_string('Hello %x', 20) +-- !query schema +struct +-- !query output +Hello 14 + + +-- !query +select format_string('%1$s %3$s', 1, 2, 3) +-- !query schema +struct +-- !query output +1 3 + + +-- !query +select format_string('%1$s %12$s', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12) +-- !query schema +struct +-- !query output +1 12 + + +-- !query +select format_string('%1$s %4$s', 1, 2, 3) +-- !query schema +struct<> +-- !query output +java.util.MissingFormatArgumentException +Format specifier '%4$s' + + +-- !query +select format_string('%1$s %13$s', 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12) +-- !query schema +struct<> +-- !query output +java.util.MissingFormatArgumentException +Format specifier '%13$s' + + +-- !query +select format_string('%0$s', 'Hello') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.ZERO_INDEX", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`format_string`", + "parameter" : "`strfmt`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "format_string('%0$s', 'Hello')" + } ] +} + + +-- !query +select format_string('Hello %s %1$s %s', 'World', 'Hello again') +-- !query schema +struct +-- !query output +Hello World World Hello again + + +-- !query +select format_string('Hello %s %s, %2$s %2$s', 'World', 'Hello again') +-- !query schema +struct +-- !query output +Hello World Hello again, Hello again Hello again + + +-- !query +select format_string('>>%10s<<', 'Hello') +-- !query schema +struct>%10s<<, Hello):string> +-- !query output +>> Hello<< + + +-- !query +select format_string('>>%10s<<', NULL) +-- !query schema +struct>%10s<<, NULL):string> +-- !query output +>> null<< + + +-- !query +select format_string('>>%10s<<', '') +-- !query schema +struct>%10s<<, ):string> +-- !query output +>> << + + +-- !query +select format_string('>>%-10s<<', '') +-- !query schema +struct>%-10s<<, ):string> +-- !query output +>> << + + +-- !query +select format_string('>>%-10s<<', 'Hello') +-- !query schema +struct>%-10s<<, Hello):string> +-- !query output +>>Hello << + + +-- !query +select format_string('>>%-10s<<', NULL) +-- !query schema +struct>%-10s<<, NULL):string> +-- !query output +>>null << + + +-- !query +select format_string('>>%1$10s<<', 'Hello') +-- !query schema +struct>%1$10s<<, Hello):string> +-- !query output +>> Hello<< + + +-- !query +DROP TABLE TEXT_TBL +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/timestamp.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/timestamp.sql.out new file mode 100644 index 000000000000..d936fba96d8f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/timestamp.sql.out @@ -0,0 +1,278 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE TIMESTAMP_TBL (d1 timestamp) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'now') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'now') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'today') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'yesterday') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'tomorrow') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'tomorrow EST') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'tomorrow Zulu') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'today' +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT count(*) AS Three FROM TIMESTAMP_TBL WHERE d1 = timestamp 'tomorrow' +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'yesterday' +-- !query schema +struct +-- !query output +1 + + +-- !query +TRUNCATE TABLE TIMESTAMP_TBL +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp'epoch') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-01-02')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-01-02 03:04:05')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('1997-02-10 17:32:01-08')) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO TIMESTAMP_TBL VALUES (timestamp('2001-09-22T18:19:20')) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT '' AS `64`, d1 FROM TIMESTAMP_TBL +-- !query schema +struct<64:string,d1:timestamp> +-- !query output + 1969-12-31 16:00:00 + 1997-01-02 00:00:00 + 1997-01-02 03:04:05 + 1997-02-10 17:32:01 + 2001-09-22 18:19:20 + + +-- !query +SELECT '' AS `48`, d1 FROM TIMESTAMP_TBL + WHERE d1 > timestamp '1997-01-02' +-- !query schema +struct<48:string,d1:timestamp> +-- !query output + 1997-01-02 03:04:05 + 1997-02-10 17:32:01 + 2001-09-22 18:19:20 + + +-- !query +SELECT '' AS `15`, d1 FROM TIMESTAMP_TBL + WHERE d1 < timestamp '1997-01-02' +-- !query schema +struct<15:string,d1:timestamp> +-- !query output + 1969-12-31 16:00:00 + + +-- !query +SELECT '' AS one, d1 FROM TIMESTAMP_TBL + WHERE d1 = timestamp '1997-01-02' +-- !query schema +struct +-- !query output + 1997-01-02 00:00:00 + + +-- !query +SELECT '' AS `63`, d1 FROM TIMESTAMP_TBL + WHERE d1 != timestamp '1997-01-02' +-- !query schema +struct<63:string,d1:timestamp> +-- !query output + 1969-12-31 16:00:00 + 1997-01-02 03:04:05 + 1997-02-10 17:32:01 + 2001-09-22 18:19:20 + + +-- !query +SELECT '' AS `16`, d1 FROM TIMESTAMP_TBL + WHERE d1 <= timestamp '1997-01-02' +-- !query schema +struct<16:string,d1:timestamp> +-- !query output + 1969-12-31 16:00:00 + 1997-01-02 00:00:00 + + +-- !query +SELECT '' AS `49`, d1 FROM TIMESTAMP_TBL + WHERE d1 >= timestamp '1997-01-02' +-- !query schema +struct<49:string,d1:timestamp> +-- !query output + 1997-01-02 00:00:00 + 1997-01-02 03:04:05 + 1997-02-10 17:32:01 + 2001-09-22 18:19:20 + + +-- !query +SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff + FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01' +-- !query schema +struct<54:string,diff:interval> +-- !query output + -236720 hours + 0 seconds + 3 hours 4 minutes 5 seconds + 41393 hours 19 minutes 20 seconds + 953 hours 32 minutes 1 seconds + + +-- !query +SELECT '' AS date_trunc_week, date_trunc( 'week', timestamp '2004-02-29 15:44:17.71393' ) AS week_trunc +-- !query schema +struct +-- !query output + 2004-02-23 00:00:00 + + +-- !query +SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff + FROM TIMESTAMP_TBL + WHERE d1 BETWEEN timestamp '1902-01-01' + AND timestamp '2038-01-01' +-- !query schema +struct<54:string,diff:interval> +-- !query output + -236720 hours + 0 seconds + 3 hours 4 minutes 5 seconds + 41393 hours 19 minutes 20 seconds + 953 hours 32 minutes 1 seconds + + +-- !query +SELECT '' AS `54`, d1 as `timestamp`, + date_part( 'year', d1) AS `year`, date_part( 'month', d1) AS `month`, + date_part( 'day', d1) AS `day`, date_part( 'hour', d1) AS `hour`, + date_part( 'minute', d1) AS `minute`, date_part( 'second', d1) AS `second` + FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01' +-- !query schema +struct<54:string,timestamp:timestamp,year:int,month:int,day:int,hour:int,minute:int,second:decimal(8,6)> +-- !query output + 1969-12-31 16:00:00 1969 12 31 16 0 0.000000 + 1997-01-02 00:00:00 1997 1 2 0 0 0.000000 + 1997-01-02 03:04:05 1997 1 2 3 4 5.000000 + 1997-02-10 17:32:01 1997 2 10 17 32 1.000000 + 2001-09-22 18:19:20 2001 9 22 18 19 20.000000 + + +-- !query +SELECT make_timestamp(2014,12,28,6,30,45.887) +-- !query schema +struct +-- !query output +2014-12-28 06:30:45.887 + + +-- !query +DROP TABLE TIMESTAMP_TBL +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/union.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/union.sql.out new file mode 100644 index 000000000000..7c920bbd32b3 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/union.sql.out @@ -0,0 +1,798 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM + (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) + AS v(f1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM + (VALUES + (123, 456), + (123, 4567890123456789), + (4567890123456789, 123), + (4567890123456789, 4567890123456789), + (4567890123456789, -4567890123456789)) + AS v(q1, q2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW FLOAT8_TBL AS SELECT * FROM + (VALUES (0.0), (-34.84), (-1004.30), + (CAST('-1.2345678901234e+200' AS DOUBLE)), (CAST('-1.2345678901234e-200' AS DOUBLE))) + AS v(f1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 1 AS two UNION SELECT 2 ORDER BY 1 +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT 1 AS one UNION SELECT 1 ORDER BY 1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT 1 AS two UNION ALL SELECT 2 +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT 1 AS two UNION ALL SELECT 1 +-- !query schema +struct +-- !query output +1 +1 + + +-- !query +SELECT 1 AS three UNION SELECT 2 UNION SELECT 3 ORDER BY 1 +-- !query schema +struct +-- !query output +1 +2 +3 + + +-- !query +SELECT 1 AS two UNION SELECT 2 UNION SELECT 2 ORDER BY 1 +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT 1 AS three UNION SELECT 2 UNION ALL SELECT 2 ORDER BY 1 +-- !query schema +struct +-- !query output +1 +2 +2 + + +-- !query +SELECT 1.1 AS two UNION SELECT 2.2 ORDER BY 1 +-- !query schema +struct +-- !query output +1.1 +2.2 + + +-- !query +SELECT 1.1 AS two UNION SELECT 2 ORDER BY 1 +-- !query schema +struct +-- !query output +1.1 +2.0 + + +-- !query +SELECT 1 AS two UNION SELECT 2.2 ORDER BY 1 +-- !query schema +struct +-- !query output +1.0 +2.2 + + +-- !query +SELECT 1 AS one UNION SELECT double(1.0) ORDER BY 1 +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT 1.1 AS two UNION ALL SELECT 2 ORDER BY 1 +-- !query schema +struct +-- !query output +1.1 +2.0 + + +-- !query +SELECT double(1.0) AS two UNION ALL SELECT 1 ORDER BY 1 +-- !query schema +struct +-- !query output +1.0 +1.0 + + +-- !query +SELECT 1.1 AS three UNION SELECT 2 UNION SELECT 3 ORDER BY 1 +-- !query schema +struct +-- !query output +1.1 +2.0 +3.0 + + +-- !query +SELECT double(1.1) AS two UNION SELECT 2 UNION SELECT double(2.0) ORDER BY 1 +-- !query schema +struct +-- !query output +1.1 +2.0 + + +-- !query +SELECT 1.1 AS three UNION SELECT 2 UNION ALL SELECT 2 ORDER BY 1 +-- !query schema +struct +-- !query output +1.1 +2.0 +2.0 + + +-- !query +SELECT 1.1 AS two UNION (SELECT 2 UNION ALL SELECT 2) ORDER BY 1 +-- !query schema +struct +-- !query output +1.1 +2.0 + + +-- !query +SELECT f1 AS five FROM FLOAT8_TBL +UNION +SELECT f1 FROM FLOAT8_TBL +ORDER BY 1 +-- !query schema +struct +-- !query output +-1.2345678901234E200 +-1004.3 +-34.84 +-1.2345678901234E-200 +0.0 + + +-- !query +SELECT f1 AS ten FROM FLOAT8_TBL +UNION ALL +SELECT f1 FROM FLOAT8_TBL +-- !query schema +struct +-- !query output +-1.2345678901234E-200 +-1.2345678901234E-200 +-1.2345678901234E200 +-1.2345678901234E200 +-1004.3 +-1004.3 +-34.84 +-34.84 +0.0 +0.0 + + +-- !query +SELECT f1 AS nine FROM FLOAT8_TBL +UNION +SELECT f1 FROM INT4_TBL +ORDER BY 1 +-- !query schema +struct +-- !query output +-1.2345678901234E200 +-2.147483647E9 +-123456.0 +-1004.3 +-34.84 +-1.2345678901234E-200 +0.0 +123456.0 +2.147483647E9 + + +-- !query +SELECT f1 AS ten FROM FLOAT8_TBL +UNION ALL +SELECT f1 FROM INT4_TBL +-- !query schema +struct +-- !query output +-1.2345678901234E-200 +-1.2345678901234E200 +-1004.3 +-123456.0 +-2.147483647E9 +-34.84 +0.0 +0.0 +123456.0 +2.147483647E9 + + +-- !query +SELECT f1 AS five FROM FLOAT8_TBL + WHERE f1 BETWEEN -1e6 AND 1e6 +UNION +SELECT f1 FROM INT4_TBL + WHERE f1 BETWEEN 0 AND 1000000 +ORDER BY 1 +-- !query schema +struct +-- !query output +-1004.3 +-34.84 +-1.2345678901234E-200 +0.0 +123456.0 + + +-- !query +SELECT q2 FROM int8_tbl INTERSECT SELECT q1 FROM int8_tbl ORDER BY 1 +-- !query schema +struct +-- !query output +123 +4567890123456789 + + +-- !query +SELECT q2 FROM int8_tbl INTERSECT ALL SELECT q1 FROM int8_tbl ORDER BY 1 +-- !query schema +struct +-- !query output +123 +4567890123456789 +4567890123456789 + + +-- !query +SELECT q2 FROM int8_tbl EXCEPT SELECT q1 FROM int8_tbl ORDER BY 1 +-- !query schema +struct +-- !query output +-4567890123456789 +456 + + +-- !query +SELECT q2 FROM int8_tbl EXCEPT ALL SELECT q1 FROM int8_tbl ORDER BY 1 +-- !query schema +struct +-- !query output +-4567890123456789 +456 + + +-- !query +SELECT q2 FROM int8_tbl EXCEPT ALL SELECT DISTINCT q1 FROM int8_tbl ORDER BY 1 +-- !query schema +struct +-- !query output +-4567890123456789 +456 +4567890123456789 + + +-- !query +SELECT q1 FROM int8_tbl EXCEPT SELECT q2 FROM int8_tbl ORDER BY 1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT q1 FROM int8_tbl EXCEPT ALL SELECT q2 FROM int8_tbl ORDER BY 1 +-- !query schema +struct +-- !query output +123 +4567890123456789 + + +-- !query +SELECT q1 FROM int8_tbl EXCEPT ALL SELECT DISTINCT q2 FROM int8_tbl ORDER BY 1 +-- !query schema +struct +-- !query output +123 +4567890123456789 +4567890123456789 + + +-- !query +(SELECT 1,2,3 UNION SELECT 4,5,6) INTERSECT SELECT 4,5,6 +-- !query schema +struct<1:int,2:int,3:int> +-- !query output +4 5 6 + + +-- !query +(SELECT 1,2,3 UNION SELECT 4,5,6 ORDER BY 1,2) INTERSECT SELECT 4,5,6 +-- !query schema +struct<1:int,2:int,3:int> +-- !query output +4 5 6 + + +-- !query +(SELECT 1,2,3 UNION SELECT 4,5,6) EXCEPT SELECT 4,5,6 +-- !query schema +struct<1:int,2:int,3:int> +-- !query output +1 2 3 + + +-- !query +(SELECT 1,2,3 UNION SELECT 4,5,6 ORDER BY 1,2) EXCEPT SELECT 4,5,6 +-- !query schema +struct<1:int,2:int,3:int> +-- !query output +1 2 3 + + +-- !query +select count(*) from + ( select unique1 from tenk1 intersect select fivethous from tenk1 ) ss +-- !query schema +struct +-- !query output +5000 + + +-- !query +select unique1 from tenk1 except select unique2 from tenk1 where unique2 != 10 +-- !query schema +struct +-- !query output +10 + + +-- !query +select count(*) from + ( select unique1 from tenk1 intersect select fivethous from tenk1 ) ss +-- !query schema +struct +-- !query output +5000 + + +-- !query +select unique1 from tenk1 except select unique2 from tenk1 where unique2 != 10 +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT f1 FROM float8_tbl INTERSECT SELECT f1 FROM int4_tbl ORDER BY 1 +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT f1 FROM float8_tbl EXCEPT SELECT f1 FROM int4_tbl ORDER BY 1 +-- !query schema +struct +-- !query output +-1.2345678901234E200 +-1004.3 +-34.84 +-1.2345678901234E-200 + + +-- !query +SELECT q1 FROM int8_tbl INTERSECT SELECT q2 FROM int8_tbl UNION ALL SELECT q2 FROM int8_tbl ORDER BY 1 +-- !query schema +struct +-- !query output +-4567890123456789 +123 +123 +456 +4567890123456789 +4567890123456789 +4567890123456789 + + +-- !query +SELECT q1 FROM int8_tbl INTERSECT (((SELECT q2 FROM int8_tbl UNION ALL SELECT q2 FROM int8_tbl))) ORDER BY 1 +-- !query schema +struct +-- !query output +123 +4567890123456789 + + +-- !query +(((SELECT q1 FROM int8_tbl INTERSECT SELECT q2 FROM int8_tbl ORDER BY 1))) UNION ALL SELECT q2 FROM int8_tbl +-- !query schema +struct +-- !query output +123 +4567890123456789 +456 +4567890123456789 +123 +4567890123456789 +-4567890123456789 + + +-- !query +SELECT q1 FROM int8_tbl UNION ALL SELECT q2 FROM int8_tbl EXCEPT SELECT q1 FROM int8_tbl ORDER BY 1 +-- !query schema +struct +-- !query output +-4567890123456789 +456 + + +-- !query +SELECT q1 FROM int8_tbl UNION ALL (((SELECT q2 FROM int8_tbl EXCEPT SELECT q1 FROM int8_tbl ORDER BY 1))) +-- !query schema +struct +-- !query output +123 +123 +4567890123456789 +4567890123456789 +4567890123456789 +-4567890123456789 +456 + + +-- !query +(((SELECT q1 FROM int8_tbl UNION ALL SELECT q2 FROM int8_tbl))) EXCEPT SELECT q1 FROM int8_tbl ORDER BY 1 +-- !query schema +struct +-- !query output +-4567890123456789 +456 + + +-- !query +SELECT q1,q2 FROM int8_tbl EXCEPT SELECT q2,q1 FROM int8_tbl +ORDER BY q2,q1 +-- !query schema +struct +-- !query output +4567890123456789 -4567890123456789 +123 456 + + +-- !query +SELECT q1 FROM int8_tbl EXCEPT SELECT q2 FROM int8_tbl ORDER BY q2 LIMIT 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`q2`", + "proposal" : "`q1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 65, + "stopIndex" : 66, + "fragment" : "q2" + } ] +} + + +-- !query +SELECT q1 FROM int8_tbl EXCEPT (((SELECT q2 FROM int8_tbl ORDER BY q2 LIMIT 1))) ORDER BY 1 +-- !query schema +struct +-- !query output +123 +4567890123456789 + + +-- !query +(((((select * from int8_tbl))))) +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query +select * from range(1,5) union select * from range(1,3) +-- !query schema +struct +-- !query output +1 +2 +3 +4 + + +-- !query +select * from range(1,6) union all select * from range(1,4) +-- !query schema +struct +-- !query output +1 +1 +2 +2 +3 +3 +4 +5 + + +-- !query +select * from range(1,6) intersect select * from range(1,4) +-- !query schema +struct +-- !query output +1 +2 +3 + + +-- !query +select * from range(1,6) intersect all select * from range(1,4) +-- !query schema +struct +-- !query output +1 +2 +3 + + +-- !query +select * from range(1,6) except select * from range(1,4) +-- !query schema +struct +-- !query output +4 +5 + + +-- !query +select * from range(1,6) except all select * from range(1,4) +-- !query schema +struct +-- !query output +4 +5 + + +-- !query +select * from range(1,6) union select * from range(1,4) +-- !query schema +struct +-- !query output +1 +2 +3 +4 +5 + + +-- !query +select * from range(1,6) union all select * from range(1,4) +-- !query schema +struct +-- !query output +1 +1 +2 +2 +3 +3 +4 +5 + + +-- !query +select * from range(1,6) intersect select * from range(1,4) +-- !query schema +struct +-- !query output +1 +2 +3 + + +-- !query +select * from range(1,6) intersect all select * from range(1,4) +-- !query schema +struct +-- !query output +1 +2 +3 + + +-- !query +select * from range(1,6) except select * from range(1,4) +-- !query schema +struct +-- !query output +4 +5 + + +-- !query +select * from range(1,6) except all select * from range(1,4) +-- !query schema +struct +-- !query output +4 +5 + + +-- !query +SELECT cast('3.4' as decimal(38, 18)) UNION SELECT 'foo' +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'foo'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 56, + "fragment" : "SELECT cast('3.4' as decimal(38, 18)) UNION SELECT 'foo'" + } ] +} + + +-- !query +SELECT * FROM + (SELECT 1 AS t, 2 AS x + UNION + SELECT 2 AS t, 4 AS x) ss +WHERE x < 4 +ORDER BY x +-- !query schema +struct +-- !query output +1 2 + + +-- !query +SELECT * FROM + (SELECT 1 AS t, id as x from range(1,11) + UNION + SELECT 2 AS t, 4 AS x) ss +WHERE x < 4 +ORDER BY x +-- !query schema +struct +-- !query output +1 1 +1 2 +1 3 + + +-- !query +SELECT * FROM + (SELECT 1 AS t, int((random()*3)) AS x + UNION + SELECT 2 AS t, 4 AS x) ss +WHERE x > 3 +ORDER BY x +-- !query schema +struct +-- !query output +2 4 + + +-- !query +select distinct q1 from + (select distinct * from int8_tbl i81 + union all + select distinct * from int8_tbl i82) ss +where q2 = q2 +-- !query schema +struct +-- !query output +123 +4567890123456789 + + +-- !query +select distinct q1 from + (select distinct * from int8_tbl i81 + union all + select distinct * from int8_tbl i82) ss +where -q1 = q2 +-- !query schema +struct +-- !query output +4567890123456789 + + +-- !query +select * from + (select *, 0 as x from int8_tbl a + union all + select *, 1 as x from int8_tbl b) ss +where (x = 0) or (q1 >= q2 and q1 <= q2) +-- !query schema +struct +-- !query output +123 456 0 +123 4567890123456789 0 +4567890123456789 -4567890123456789 0 +4567890123456789 123 0 +4567890123456789 4567890123456789 0 +4567890123456789 4567890123456789 1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part1.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part1.sql.out new file mode 100755 index 000000000000..8b05d28dfcf4 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part1.sql.out @@ -0,0 +1,756 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT four, ten, SUM(SUM(four)) OVER (PARTITION BY four), AVG(ten) FROM tenk1 +GROUP BY four, ten ORDER BY four, ten +-- !query schema +struct +-- !query output +0 0 0 0.0 +0 2 0 2.0 +0 4 0 4.0 +0 6 0 6.0 +0 8 0 8.0 +1 1 2500 1.0 +1 3 2500 3.0 +1 5 2500 5.0 +1 7 2500 7.0 +1 9 2500 9.0 +2 0 5000 0.0 +2 2 5000 2.0 +2 4 5000 4.0 +2 6 5000 6.0 +2 8 5000 8.0 +3 1 7500 1.0 +3 3 7500 3.0 +3 5 7500 5.0 +3 7 7500 7.0 +3 9 7500 9.0 + + +-- !query +SELECT COUNT(*) OVER () FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 + + +-- !query +SELECT COUNT(*) OVER w FROM tenk1 WHERE unique2 < 10 WINDOW w AS () +-- !query schema +struct +-- !query output +10 +10 +10 +10 +10 +10 +10 +10 +10 +10 + + +-- !query +SELECT four FROM tenk1 WHERE FALSE WINDOW w AS (PARTITION BY ten) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT sum(four) OVER (PARTITION BY ten ORDER BY unique2) AS sum_1, ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +0 0 0 +0 0 0 +0 4 0 +1 7 1 +1 9 1 +2 0 2 +3 1 3 +3 3 3 +4 1 1 +5 1 1 + + +-- !query +SELECT row_number() OVER (ORDER BY unique2) FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +1 +10 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +SELECT rank() OVER (PARTITION BY four ORDER BY ten) AS rank_1, ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +1 0 0 +1 0 0 +1 0 2 +1 1 1 +1 1 1 +1 1 3 +2 3 3 +3 4 0 +3 7 1 +4 9 1 + + +-- !query +SELECT dense_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +1 0 0 +1 0 0 +1 0 2 +1 1 1 +1 1 1 +1 1 3 +2 3 3 +2 4 0 +2 7 1 +3 9 1 + + +-- !query +SELECT percent_rank() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +0.0 0 0 +0.0 0 0 +0.0 0 2 +0.0 1 1 +0.0 1 1 +0.0 1 3 +0.6666666666666666 7 1 +1.0 3 3 +1.0 4 0 +1.0 9 1 + + +-- !query +SELECT cume_dist() OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +0.5 1 1 +0.5 1 1 +0.5 1 3 +0.6666666666666666 0 0 +0.6666666666666666 0 0 +0.75 7 1 +1.0 0 2 +1.0 3 3 +1.0 4 0 +1.0 9 1 + + +-- !query +SELECT ntile(3) OVER (ORDER BY ten, four), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +1 0 0 +1 0 0 +1 0 2 +1 1 1 +2 1 1 +2 1 3 +2 3 3 +3 4 0 +3 7 1 +3 9 1 + + +-- !query +SELECT lag(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +0 0 0 +0 4 0 +1 1 1 +1 3 3 +1 7 1 +7 9 1 +NULL 0 0 +NULL 0 2 +NULL 1 1 +NULL 1 3 + + +-- !query +SELECT lead(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +0 0 0 +1 1 1 +3 1 3 +4 0 0 +7 1 1 +9 7 1 +NULL 0 2 +NULL 3 3 +NULL 4 0 +NULL 9 1 + + +-- !query +SELECT lead(ten * 2, 1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +0 0 0 +14 1 1 +18 7 1 +2 1 1 +6 1 3 +8 0 0 +NULL 0 2 +NULL 3 3 +NULL 4 0 +NULL 9 1 + + +-- !query +SELECT lead(ten * 2, 1, -1) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +-1 0 2 +-1 3 3 +-1 4 0 +-1 9 1 +0 0 0 +14 1 1 +18 7 1 +2 1 1 +6 1 3 +8 0 0 + + +-- !query +SELECT first(ten) OVER (PARTITION BY four ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +0 0 0 +0 0 0 +0 0 2 +0 4 0 +1 1 1 +1 1 1 +1 1 3 +1 3 3 +1 7 1 +1 9 1 + + +-- !query +SELECT last(four) OVER (ORDER BY ten), ten, four FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +0 4 0 +1 1 1 +1 1 1 +1 1 3 +1 7 1 +1 9 1 +2 0 0 +2 0 0 +2 0 2 +3 3 3 + + +-- !query +SELECT last(ten) OVER (PARTITION BY four), ten, four FROM +(SELECT * FROM tenk1 WHERE unique2 < 10 ORDER BY four, ten)s +ORDER BY four, ten +-- !query schema +struct +-- !query output +4 0 0 +4 0 0 +4 4 0 +9 1 1 +9 1 1 +9 7 1 +9 9 1 +0 0 2 +3 1 3 +3 3 3 + + +-- !query +SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER (PARTITION BY two ORDER BY ten) AS wsum +FROM tenk1 GROUP BY ten, two +-- !query schema +struct +-- !query output +0 0 45000 45000 +1 1 46000 46000 +2 0 47000 92000 +3 1 48000 94000 +4 0 49000 141000 +5 1 50000 144000 +6 0 51000 192000 +7 1 52000 196000 +8 0 53000 245000 +9 1 54000 250000 + + +-- !query +SELECT count(*) OVER (PARTITION BY four), four FROM (SELECT * FROM tenk1 WHERE two = 1)s WHERE unique2 < 10 +-- !query schema +struct +-- !query output +2 3 +2 3 +4 1 +4 1 +4 1 +4 1 + + +-- !query +SELECT (count(*) OVER (PARTITION BY four ORDER BY ten) + + sum(hundred) OVER (PARTITION BY four ORDER BY ten)) AS cntsum + FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +136 +22 +22 +24 +24 +51 +82 +87 +92 +92 + + +-- !query +SELECT * FROM( + SELECT count(*) OVER (PARTITION BY four ORDER BY ten) + + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS total, + count(*) OVER (PARTITION BY four ORDER BY ten) AS fourcount, + sum(hundred) OVER (PARTITION BY two ORDER BY ten) AS twosum + FROM tenk1 +)sub WHERE total <> fourcount + twosum +-- !query schema +struct +-- !query output + + + +-- !query +SELECT avg(four) OVER (PARTITION BY four ORDER BY thousand / 100) FROM tenk1 WHERE unique2 < 10 +-- !query schema +struct +-- !query output +0.0 +0.0 +0.0 +1.0 +1.0 +1.0 +1.0 +2.0 +3.0 +3.0 + + +-- !query +SELECT ten, two, sum(hundred) AS gsum, sum(sum(hundred)) OVER win AS wsum +FROM tenk1 GROUP BY ten, two WINDOW win AS (PARTITION BY two ORDER BY ten) +-- !query schema +struct +-- !query output +0 0 45000 45000 +1 1 46000 46000 +2 0 47000 92000 +3 1 48000 94000 +4 0 49000 141000 +5 1 50000 144000 +6 0 51000 192000 +7 1 52000 196000 +8 0 53000 245000 +9 1 54000 250000 + + +-- !query +SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 WHERE FALSE)s +-- !query schema +struct +-- !query output + + + +-- !query +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT SUM(COUNT(f1)) OVER () FROM int4_tbl WHERE f1=42 +-- !query schema +struct +-- !query output +0 + + +-- !query +select ten, + sum(unique1) + sum(unique2) as res, + rank() over (order by sum(unique1) + sum(unique2)) as rank +from tenk1 +group by ten order by ten +-- !query schema +struct +-- !query output +0 9976146 4 +1 10114187 9 +2 10059554 8 +3 9878541 1 +4 9881005 2 +5 9981670 5 +6 9947099 3 +7 10120309 10 +8 9991305 6 +9 10040184 7 + + +-- !query +SELECT four, ten, +sum(ten) over (partition by four order by ten), +last(ten) over (partition by four order by ten) +FROM (select distinct ten, four from tenk1) ss +-- !query schema +struct +-- !query output +0 0 0 0 +0 2 2 2 +0 4 6 4 +0 6 12 6 +0 8 20 8 +1 1 1 1 +1 3 4 3 +1 5 9 5 +1 7 16 7 +1 9 25 9 +2 0 0 0 +2 2 2 2 +2 4 6 4 +2 6 12 6 +2 8 20 8 +3 1 1 1 +3 3 4 3 +3 5 9 5 +3 7 16 7 +3 9 25 9 + + +-- !query +SELECT four, ten, +sum(ten) over (partition by four order by ten range between unbounded preceding and current row), +last(ten) over (partition by four order by ten range between unbounded preceding and current row) +FROM (select distinct ten, four from tenk1) ss +-- !query schema +struct +-- !query output +0 0 0 0 +0 2 2 2 +0 4 6 4 +0 6 12 6 +0 8 20 8 +1 1 1 1 +1 3 4 3 +1 5 9 5 +1 7 16 7 +1 9 25 9 +2 0 0 0 +2 2 2 2 +2 4 6 4 +2 6 12 6 +2 8 20 8 +3 1 1 1 +3 3 4 3 +3 5 9 5 +3 7 16 7 +3 9 25 9 + + +-- !query +SELECT four, ten, +sum(ten) over (partition by four order by ten range between unbounded preceding and unbounded following), +last(ten) over (partition by four order by ten range between unbounded preceding and unbounded following) +FROM (select distinct ten, four from tenk1) ss +-- !query schema +struct +-- !query output +0 0 20 8 +0 2 20 8 +0 4 20 8 +0 6 20 8 +0 8 20 8 +1 1 25 9 +1 3 25 9 +1 5 25 9 +1 7 25 9 +1 9 25 9 +2 0 20 8 +2 2 20 8 +2 4 20 8 +2 6 20 8 +2 8 20 8 +3 1 25 9 +3 3 25 9 +3 5 25 9 +3 7 25 9 +3 9 25 9 + + +-- !query +SELECT sum(unique1) over (order by four range between current row and unbounded following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query schema +struct +-- !query output +10 3 3 +10 7 3 +18 2 2 +18 6 2 +33 1 1 +33 5 1 +33 9 1 +45 0 0 +45 4 0 +45 8 0 + + +-- !query +SELECT sum(unique1) over (rows between current row and unbounded following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query schema +struct +-- !query output +0 0 0 +10 3 3 +15 5 1 +23 8 0 +32 9 1 +38 6 2 +39 1 1 +41 2 2 +45 4 0 +7 7 3 +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + + + +-- !query +SELECT sum(unique1) over (rows between 2 preceding and 2 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query schema +struct +-- !query output +10 0 0 +13 2 2 +15 7 3 +22 1 1 +23 3 3 +26 6 2 +29 9 1 +31 8 0 +32 5 1 +7 4 0 + + +-- !query +SELECT sum(unique1) over (rows between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query schema +struct +-- !query output +10 0 0 +13 3 3 +15 8 0 +17 5 1 +3 6 2 +4 2 2 +6 1 1 +7 9 1 +8 7 3 +NULL 4 0 +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + + + +-- !query +SELECT sum(unique1) over (rows between 1 following and 3 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query schema +struct +-- !query output +0 7 3 +10 5 1 +15 8 0 +16 2 2 +16 9 1 +22 6 2 +23 1 1 +7 3 3 +9 4 0 +NULL 0 0 + + +-- !query +SELECT sum(unique1) over (rows between unbounded preceding and 1 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query schema +struct +-- !query output +13 1 1 +22 6 2 +30 9 1 +35 8 0 +38 5 1 +45 0 0 +45 3 3 +45 7 3 +6 4 0 +7 2 2 + + +-- !query +CREATE TEMP VIEW v_window AS +SELECT i.id, sum(i.id) over (order by i.id rows between 1 preceding and 1 following) as sum_rows +FROM range(1, 11) i +-- !query schema +struct<> +-- !query output + + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query + +SELECT * FROM v_window +-- !query schema +struct +-- !query output +1 3 +10 19 +2 6 +3 9 +4 12 +5 15 +6 18 +7 21 +8 24 +9 27 + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query + +DROP VIEW v_window +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW tenk2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW int4_tbl +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part2.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part2.sql.out new file mode 100644 index 000000000000..6cf5e69758d2 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part2.sql.out @@ -0,0 +1,520 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE empsalary ( + depname string, + empno integer, + salary int, + enroll_date date +) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO empsalary VALUES + ('develop', 10, 5200, date '2007-08-01'), + ('sales', 1, 5000, date '2006-10-01'), + ('personnel', 5, 3500, date '2007-12-10'), + ('sales', 4, 4800, date '2007-08-08'), + ('personnel', 2, 3900, date '2006-12-23'), + ('develop', 7, 4200, date '2008-01-01'), + ('develop', 9, 4500, date '2008-01-01'), + ('sales', 3, 4800, date '2007-08-01'), + ('develop', 8, 6000, date '2006-10-01'), + ('develop', 11, 5200, date '2007-08-15') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT sum(unique1) over (order by four range between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query schema +struct +-- !query output +12 1 1 +12 5 1 +12 9 1 +23 3 3 +23 7 3 +27 2 2 +27 6 2 +NULL 0 0 +NULL 4 0 +NULL 8 0 + + +-- !query +SELECT sum(unique1) over (order by four desc range between 2 preceding and 1 preceding), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query schema +struct +-- !query output +10 2 2 +10 6 2 +18 1 1 +18 5 1 +18 9 1 +23 0 0 +23 4 0 +23 8 0 +NULL 3 3 +NULL 7 3 + + +-- !query +SELECT sum(unique1) over (partition by four order by unique1 range between 5 preceding and 6 following), +unique1, four +FROM tenk1 WHERE unique1 < 10 +-- !query schema +struct +-- !query output +10 3 3 +10 7 3 +12 4 0 +12 8 0 +14 9 1 +15 5 1 +4 0 0 +6 1 1 +8 2 2 +8 6 2 + + +-- !query +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id asc nulls first range between 2 preceding and 2 following) +-- !query schema +struct +-- !query output +1 1 1 3 +2 2 1 4 +3 3 1 5 +4 4 2 5 +5 5 3 5 +NULL 42 42 43 +NULL 43 42 43 + + +-- !query +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id asc nulls last range between 2 preceding and 2 following) +-- !query schema +struct +-- !query output +1 1 1 3 +2 2 1 4 +3 3 1 5 +4 4 2 5 +5 5 3 5 +NULL 42 42 43 +NULL 43 42 43 + + +-- !query +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id desc nulls first range between 2 preceding and 2 following) +-- !query schema +struct +-- !query output +1 1 3 1 +2 2 4 1 +3 3 5 1 +4 4 5 2 +5 5 5 3 +NULL 42 42 43 +NULL 43 42 43 + + +-- !query +select ss.id, ss.y, + first(ss.y) over w, + last(ss.y) over w +from + (select x.id, x.id as y from range(1,6) as x + union all select null, 42 + union all select null, 43) ss +window w as + (order by ss.id desc nulls last range between 2 preceding and 2 following) +-- !query schema +struct +-- !query output +1 1 3 1 +2 2 4 1 +3 3 5 1 +4 4 5 2 +5 5 5 3 +NULL 42 42 43 +NULL 43 42 43 + + +-- !query +select x.id, last(x.id) over (order by x.id range between current row and 2147450884 following) +from range(32764, 32767) x +-- !query schema +struct +-- !query output +32764 32766 +32765 32766 +32766 32766 + + +-- !query +select x.id, last(x.id) over (order by x.id desc range between current row and 2147450885 following) +from range(-32766, -32765) x +-- !query schema +struct +-- !query output +-32766 -32766 + + +-- !query +select x.id, last(x.id) over (order by x.id range between current row and 4 following) +from range(2147483644, 2147483647) x +-- !query schema +struct +-- !query output +2147483644 2147483646 +2147483645 2147483646 +2147483646 2147483646 + + +-- !query +select x.id, last(x.id) over (order by x.id desc range between current row and 5 following) +from range(-2147483646, -2147483645) x +-- !query schema +struct +-- !query output +-2147483646 -2147483646 + + +-- !query +select x.id, last(x.id) over (order by x.id range between current row and 4 following) +from range(9223372036854775804, 9223372036854775807) x +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "fragment" : "" + } ] +} + + +-- !query +select x.id, last(x.id) over (order by x.id desc range between current row and 5 following) +from range(-9223372036854775806, -9223372036854775805) x +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "fragment" : "" + } ] +} + + +-- !query +create table numerics ( + id int, + f_float4 float, + f_float8 float, + f_numeric int +) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into numerics values +(1, -3, -3, -3), +(2, -1, -1, -1), +(3, 0, 0, 0), +(4, 1.1, 1.1, 1.1), +(5, 1.12, 1.12, 1.12), +(6, 2, 2, 2), +(7, 100, 100, 100) +-- !query schema +struct<> +-- !query output + + + +-- !query +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1 preceding and 1 following) +-- !query schema +struct +-- !query output +1 -3.0 1 1 +2 -1.0 2 3 +3 0.0 2 3 +4 1.1 4 6 +5 1.12 4 6 +6 2.0 4 6 +7 100.0 7 7 + + +-- !query +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1 preceding and 1.1 following) +-- !query schema +struct +-- !query output +1 -3.0 1 1 +2 -1.0 2 3 +3 0.0 2 4 +4 1.1 4 6 +5 1.12 4 6 +6 2.0 4 6 +7 100.0 7 7 + + +-- !query +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 'inf' preceding and 'inf' following) +-- !query schema +struct +-- !query output +1 -3.0 1 7 +2 -1.0 1 7 +3 0.0 1 7 +4 1.1 1 7 +5 1.12 1 7 +6 2.0 1 7 +7 100.0 1 7 + + +-- !query +select id, f_float4, first(id) over w, last(id) over w +from numerics +window w as (order by f_float4 range between + 1.1 preceding and 'NaN' following) +-- !query schema +struct +-- !query output +1 -3.0 1 7 +2 -1.0 2 7 +3 0.0 2 7 +4 1.1 3 7 +5 1.12 4 7 +6 2.0 4 7 +7 100.0 7 7 + + +-- !query +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1 preceding and 1 following) +-- !query schema +struct +-- !query output +1 -3.0 1 1 +2 -1.0 2 3 +3 0.0 2 3 +4 1.1 4 6 +5 1.12 4 6 +6 2.0 4 6 +7 100.0 7 7 + + +-- !query +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1 preceding and 1.1 following) +-- !query schema +struct +-- !query output +1 -3.0 1 1 +2 -1.0 2 3 +3 0.0 2 4 +4 1.1 4 6 +5 1.12 4 6 +6 2.0 4 6 +7 100.0 7 7 + + +-- !query +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 'inf' preceding and 'inf' following) +-- !query schema +struct +-- !query output +1 -3.0 1 7 +2 -1.0 1 7 +3 0.0 1 7 +4 1.1 1 7 +5 1.12 1 7 +6 2.0 1 7 +7 100.0 1 7 + + +-- !query +select id, f_float8, first(id) over w, last(id) over w +from numerics +window w as (order by f_float8 range between + 1.1 preceding and 'NaN' following) +-- !query schema +struct +-- !query output +1 -3.0 1 7 +2 -1.0 2 7 +3 0.0 2 7 +4 1.1 3 7 +5 1.12 4 7 +6 2.0 4 7 +7 100.0 7 7 + + +-- !query +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1 following) +-- !query schema +struct +-- !query output +1 -3 1 1 +2 -1 2 3 +3 0 2 5 +4 1 3 6 +5 1 3 6 +6 2 4 6 +7 100 7 7 + + +-- !query +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1.1 following) +-- !query schema +struct +-- !query output +1 -3 1 1 +2 -1 2 3 +3 0 2 5 +4 1 3 6 +5 1 3 6 +6 2 4 6 +7 100 7 7 + + +-- !query +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1 preceding and 1.1 following) +-- !query schema +struct +-- !query output +1 -3 1 1 +2 -1 2 3 +3 0 2 5 +4 1 3 6 +5 1 3 6 +6 2 4 6 +7 100 7 7 + + +-- !query +select id, f_numeric, first(id) over w, last(id) over w +from numerics +window w as (order by f_numeric range between + 1.1 preceding and 'NaN' following) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'NaN'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 83, + "stopIndex" : 163, + "fragment" : "(order by f_numeric range between\n 1.1 preceding and 'NaN' following)" + } ] +} + + +-- !query +drop table empsalary +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table numerics +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part3.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part3.sql.out new file mode 100644 index 000000000000..ac4768a26c4e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part3.sql.out @@ -0,0 +1,563 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE empsalary ( + depname string, + empno integer, + salary int, + enroll_date date +) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO empsalary VALUES + ('develop', 10, 5200, date '2007-08-01'), + ('sales', 1, 5000, date '2006-10-01'), + ('personnel', 5, 3500, date '2007-12-10'), + ('sales', 4, 4800, date '2007-08-08'), + ('personnel', 2, 3900, date '2006-12-23'), + ('develop', 7, 4200, date '2008-01-01'), + ('develop', 9, 4500, date '2008-01-01'), + ('sales', 3, 4800, date '2007-08-01'), + ('develop', 8, 6000, date '2006-10-01'), + ('develop', 11, 5200, date '2007-08-15') +-- !query schema +struct<> +-- !query output + + + +-- !query +create table datetimes ( + id int, + f_time timestamp, + f_timetz timestamp, + f_interval timestamp, + f_timestamptz timestamp, + f_timestamp timestamp +) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into datetimes values +(1, timestamp '11:00', cast ('11:00 BST' as timestamp), cast ('1 year' as timestamp), cast ('2000-10-19 10:23:54+01' as timestamp), timestamp '2000-10-19 10:23:54'), +(2, timestamp '12:00', cast ('12:00 BST' as timestamp), cast ('2 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'), +(3, timestamp '13:00', cast ('13:00 BST' as timestamp), cast ('3 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'), +(4, timestamp '14:00', cast ('14:00 BST' as timestamp), cast ('4 years' as timestamp), cast ('2002-10-19 10:23:54+01' as timestamp), timestamp '2002-10-19 10:23:54'), +(5, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2003-10-19 10:23:54+01' as timestamp), timestamp '2003-10-19 10:23:54'), +(6, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2004-10-19 10:23:54+01' as timestamp), timestamp '2004-10-19 10:23:54'), +(7, timestamp '17:00', cast ('17:00 BST' as timestamp), cast ('7 years' as timestamp), cast ('2005-10-19 10:23:54+01' as timestamp), timestamp '2005-10-19 10:23:54'), +(8, timestamp '18:00', cast ('18:00 BST' as timestamp), cast ('8 years' as timestamp), cast ('2006-10-19 10:23:54+01' as timestamp), timestamp '2006-10-19 10:23:54'), +(9, timestamp '19:00', cast ('19:00 BST' as timestamp), cast ('9 years' as timestamp), cast ('2007-10-19 10:23:54+01' as timestamp), timestamp '2007-10-19 10:23:54'), +(10, timestamp '20:00', cast ('20:00 BST' as timestamp), cast ('10 years' as timestamp), cast ('2008-10-19 10:23:54+01' as timestamp), timestamp '2008-10-19 10:23:54') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_INLINE_TABLE.FAILED_SQL_EXPRESSION_EVALUATION", + "sqlState" : "42000", + "messageParameters" : { + "sqlExpr" : "\"CAST(11:00 BST AS TIMESTAMP)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 1698, + "fragment" : "values\n(1, timestamp '11:00', cast ('11:00 BST' as timestamp), cast ('1 year' as timestamp), cast ('2000-10-19 10:23:54+01' as timestamp), timestamp '2000-10-19 10:23:54'),\n(2, timestamp '12:00', cast ('12:00 BST' as timestamp), cast ('2 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'),\n(3, timestamp '13:00', cast ('13:00 BST' as timestamp), cast ('3 years' as timestamp), cast ('2001-10-19 10:23:54+01' as timestamp), timestamp '2001-10-19 10:23:54'),\n(4, timestamp '14:00', cast ('14:00 BST' as timestamp), cast ('4 years' as timestamp), cast ('2002-10-19 10:23:54+01' as timestamp), timestamp '2002-10-19 10:23:54'),\n(5, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2003-10-19 10:23:54+01' as timestamp), timestamp '2003-10-19 10:23:54'),\n(6, timestamp '15:00', cast ('15:00 BST' as timestamp), cast ('5 years' as timestamp), cast ('2004-10-19 10:23:54+01' as timestamp), timestamp '2004-10-19 10:23:54'),\n(7, timestamp '17:00', cast ('17:00 BST' as timestamp), cast ('7 years' as timestamp), cast ('2005-10-19 10:23:54+01' as timestamp), timestamp '2005-10-19 10:23:54'),\n(8, timestamp '18:00', cast ('18:00 BST' as timestamp), cast ('8 years' as timestamp), cast ('2006-10-19 10:23:54+01' as timestamp), timestamp '2006-10-19 10:23:54'),\n(9, timestamp '19:00', cast ('19:00 BST' as timestamp), cast ('9 years' as timestamp), cast ('2007-10-19 10:23:54+01' as timestamp), timestamp '2007-10-19 10:23:54'),\n(10, timestamp '20:00', cast ('20:00 BST' as timestamp), cast ('10 years' as timestamp), cast ('2008-10-19 10:23:54+01' as timestamp), timestamp '2008-10-19 10:23:54')" + } ] +} + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query + +WITH cte (x) AS ( + SELECT * FROM range(1, 36, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following) +-- !query schema +struct +-- !query output +1 4 +11 33 +13 39 +15 45 +17 51 +19 57 +21 63 +23 69 +25 75 +27 81 +29 87 +3 9 +31 93 +33 99 +35 68 +5 15 +7 21 +9 27 + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query + +WITH cte (x) AS ( + SELECT * FROM range(1, 36, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x range between 1 preceding and 1 following) +-- !query schema +struct +-- !query output +1 1 +11 11 +13 13 +15 15 +17 17 +19 19 +21 21 +23 23 +25 25 +27 27 +29 29 +3 3 +31 31 +33 33 +35 35 +5 5 +7 7 +9 9 + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query + +WITH cte (x) AS ( + select 1 union all select 1 union all select 1 union all + SELECT * FROM range(5, 50, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following) +-- !query schema +struct +-- !query output +1 2 +1 3 +1 7 +11 33 +13 39 +15 45 +17 51 +19 57 +21 63 +23 69 +25 75 +27 81 +29 87 +31 93 +33 99 +35 105 +37 111 +39 117 +41 123 +43 129 +45 135 +47 141 +49 96 +5 13 +7 21 +9 27 + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query + +WITH cte (x) AS ( + select 1 union all select 1 union all select 1 union all + SELECT * FROM range(5, 50, 2) +) +SELECT x, (sum(x) over w) +FROM cte +WINDOW w AS (ORDER BY x range between 1 preceding and 1 following) +-- !query schema +struct +-- !query output +1 3 +1 3 +1 3 +11 11 +13 13 +15 15 +17 17 +19 19 +21 21 +23 23 +25 25 +27 27 +29 29 +31 31 +33 33 +35 35 +37 37 +39 39 +41 41 +43 43 +45 45 +47 47 +49 49 +5 5 +7 7 +9 9 + + +-- !query +SELECT count(*) OVER (PARTITION BY four) FROM (SELECT * FROM tenk1 UNION ALL SELECT * FROM tenk2)s LIMIT 0 +-- !query schema +struct +-- !query output + + + +-- !query +create table t1 (f1 int, f2 int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t1 values (1,1),(1,2),(2,2) +-- !query schema +struct<> +-- !query output + + + +-- !query +select f1, sum(f1) over (partition by f1 + range between 1 preceding and 1 following) +from t1 where f1 = f2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_WITHOUT_ORDER", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"(PARTITION BY f1 RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 108, + "fragment" : "(partition by f1\n range between 1 preceding and 1 following)" + } ] +} + + +-- !query +select f1, sum(f1) over (partition by f1 order by f2 +range between 1 preceding and 1 following) +from t1 where f1 = f2 +-- !query schema +struct +-- !query output +1 1 +2 2 + + +-- !query +select f1, sum(f1) over (partition by f1, f1 order by f2 +range between 2 preceding and 1 preceding) +from t1 where f1 = f2 +-- !query schema +struct +-- !query output +1 NULL +2 NULL + + +-- !query +select f1, sum(f1) over (partition by f1, f2 order by f2 +range between 1 following and 2 following) +from t1 where f1 = f2 +-- !query schema +struct +-- !query output +1 NULL +2 NULL + + +-- !query +SELECT rank() OVER (ORDER BY length('abc')) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM empsalary WHERE row_number() OVER (ORDER BY salary) < 10 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1034", + "messageParameters" : { + "clauseName" : "WHERE" + } +} + + +-- !query +SELECT * FROM empsalary INNER JOIN tenk1 ON row_number() OVER (ORDER BY salary) < 10 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_EXPR_FOR_OPERATOR", + "sqlState" : "42K0E", + "messageParameters" : { + "invalidExprSqls" : "\"row_number() OVER (ORDER BY salary ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 84, + "fragment" : "INNER JOIN tenk1 ON row_number() OVER (ORDER BY salary) < 10" + } ] +} + + +-- !query +SELECT rank() OVER (ORDER BY 1), count(*) FROM empsalary GROUP BY 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_EXPR_FOR_OPERATOR", + "sqlState" : "42K0E", + "messageParameters" : { + "invalidExprSqls" : "\"RANK() OVER (ORDER BY 1 ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 58, + "stopIndex" : 67, + "fragment" : "GROUP BY 1" + } ] +} + + +-- !query +SELECT * FROM rank() OVER (ORDER BY random()) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'BY'", + "hint" : "" + } +} + + +-- !query +SELECT * FROM empsalary WHERE (rank() OVER (ORDER BY random())) > 10 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1034", + "messageParameters" : { + "clauseName" : "WHERE" + } +} + + +-- !query +SELECT * FROM empsalary WHERE rank() OVER (ORDER BY random()) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1034", + "messageParameters" : { + "clauseName" : "WHERE" + } +} + + +-- !query +select rank() OVER (PARTITION BY four, ORDER BY ten) FROM tenk1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'BY'", + "hint" : ": extra input 'BY'" + } +} + + +-- !query +SELECT range(1, 100) OVER () FROM empsalary +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`range`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "range(1, 100) OVER ()" + } ] +} + + +-- !query +SELECT ntile(0) OVER (ORDER BY ten), ten, four FROM tenk1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", + "sqlState" : "42K09", + "messageParameters" : { + "currentValue" : "0", + "exprName" : "buckets", + "sqlExpr" : "\"ntile(0)\"", + "valueRange" : "(0, 2147483647]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "ntile(0) OVER (ORDER BY ten)" + } ] +} + + +-- !query +SELECT nth_value(four, 0) OVER (ORDER BY ten), ten, four FROM tenk1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", + "sqlState" : "42K09", + "messageParameters" : { + "currentValue" : "0L", + "exprName" : "offset", + "sqlExpr" : "\"nth_value(four, 0)\"", + "valueRange" : "(0, 9223372036854775807]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "nth_value(four, 0) OVER (ORDER BY ten)" + } ] +} + + +-- !query +DROP TABLE empsalary +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE datetimes +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE t1 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part4.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part4.sql.out new file mode 100644 index 000000000000..ab8dc4de9ace --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/window_part4.sql.out @@ -0,0 +1,531 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 1.5 +2 2.0 +3 NULL +4 NULL + + +-- !query +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 1.5 +2 2.0 +3 NULL +4 NULL + + +-- !query +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 1.5 +2 2.0 +3 NULL +4 NULL + + +-- !query +SELECT i,AVG(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.5),(2,2.5),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 2.00000 +2 2.50000 +3 NULL +4 NULL + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 3 +2 2 +3 NULL +4 NULL + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 3 +2 2 +3 NULL +4 NULL + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 3 +2 2 +3 NULL +4 NULL + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.1),(2,2.2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 3.3 +2 2.2 +3 NULL +4 NULL + + +-- !query +SELECT SUM(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1.01),(2,2),(3,3)) v(i,n) +-- !query schema +struct +-- !query output +3.00 +5.00 +6.01 + + +-- !query +SELECT i,COUNT(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 2 +2 1 +3 0 +4 0 + + +-- !query +SELECT i,COUNT(*) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 4 +2 3 +3 2 +4 1 + + +-- !query +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +0.0 +11266.666666666666 +13868.750000000002 +21703.999999999996 +4225.0 + + +-- !query +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +0.0 +11266.666666666666 +13868.750000000002 +21703.999999999996 +4225.0 + + +-- !query +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +0.0 +11266.666666666666 +13868.750000000002 +21703.999999999996 +4225.0 + + +-- !query +SELECT VAR_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +0.0 +11266.666666666666 +13868.750000000002 +21703.999999999996 +4225.0 + + +-- !query +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NULL + + +-- !query +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NULL + + +-- !query +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NULL + + +-- !query +SELECT VAR_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NULL + + +-- !query +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NULL + + +-- !query +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NULL + + +-- !query +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NULL + + +-- !query +SELECT VARIANCE(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +16900.0 +18491.666666666668 +27129.999999999996 +8450.0 +NULL + + +-- !query +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query schema +struct +-- !query output +0.0 +106.14455552060438 +117.76565713313879 +147.32277488562315 +147.32277488562315 +65.0 + + +-- !query +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query schema +struct +-- !query output +0.0 +106.14455552060438 +117.76565713313879 +147.32277488562315 +147.32277488562315 +65.0 + + +-- !query +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query schema +struct +-- !query output +0.0 +106.14455552060438 +117.76565713313879 +147.32277488562315 +147.32277488562315 +65.0 + + +-- !query +SELECT STDDEV_POP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query schema +struct +-- !query output +0.0 +106.14455552060438 +117.76565713313879 +147.32277488562315 +147.32277488562315 +65.0 + + +-- !query +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query schema +struct +-- !query output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NULL + + +-- !query +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query schema +struct +-- !query output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NULL + + +-- !query +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query schema +struct +-- !query output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NULL + + +-- !query +SELECT STDDEV_SAMP(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(1,NULL),(2,600),(3,470),(4,170),(5,430),(6,300)) r(i,n) +-- !query schema +struct +-- !query output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NULL + + +-- !query +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NULL + + +-- !query +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NULL + + +-- !query +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NULL + + +-- !query +SELECT STDDEV(n) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) + FROM (VALUES(0,NULL),(1,600),(2,470),(3,170),(4,430),(5,300)) r(i,n) +-- !query schema +struct +-- !query output +130.0 +135.9840676942217 +164.7118696390761 +164.7118696390761 +91.92388155425118 +NULL + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND CURRENT ROW) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 1 +2 2 +3 NULL +4 NULL + + +-- !query +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,NULL),(4,NULL)) t(i,v) +-- !query schema +struct +-- !query output +1 3 +2 2 +3 NULL +4 NULL + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query + +SELECT i,SUM(v) OVER (ORDER BY i ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) + FROM (VALUES(1,1),(2,2),(3,3),(4,4)) t(i,v) +-- !query schema +struct +-- !query output +1 3 +2 6 +3 9 +4 7 + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query +SELECT a, b, + SUM(b) OVER(ORDER BY A ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) +FROM (VALUES(1,1),(2,2),(3,(cast('nan' as int))),(4,3),(5,4)) t(a,b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INLINE_TABLE.FAILED_SQL_EXPRESSION_EVALUATION", + "sqlState" : "42000", + "messageParameters" : { + "sqlExpr" : "\"CAST(nan AS INT)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 92, + "stopIndex" : 145, + "fragment" : "VALUES(1,1),(2,2),(3,(cast('nan' as int))),(4,3),(5,4)" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/with.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/with.sql.out new file mode 100644 index 000000000000..af1c33023a91 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/postgreSQL/with.sql.out @@ -0,0 +1,507 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +WITH q1(x,y) AS (SELECT 1,2) +SELECT * FROM q1, q1 AS q2 +-- !query schema +struct +-- !query output +1 2 1 2 + + +-- !query +SELECT count(*) FROM ( + WITH q1(x) AS (SELECT rand() FROM (SELECT EXPLODE(SEQUENCE(1, 5)))) + SELECT * FROM q1 + UNION + SELECT * FROM q1 +) ss +-- !query schema +struct +-- !query output +5 + + +-- !query +CREATE TABLE department ( + id INTEGER, -- department ID + parent_department INTEGER, -- upper department ID + name string -- department name +) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO department VALUES (0, NULL, 'ROOT') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO department VALUES (1, 0, 'A') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO department VALUES (2, 1, 'B') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO department VALUES (3, 2, 'C') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO department VALUES (4, 2, 'D') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO department VALUES (5, 0, 'E') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO department VALUES (6, 4, 'F') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO department VALUES (7, 5, 'G') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tree( + id INTEGER, + parent_id INTEGER +) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO tree +VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3), + (9,4), (10,4), (11,7), (12,7), (13,7), (14, 9), (15,11), (16,11) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table graph( f int, t int, label string ) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into graph values + (1, 2, 'arc 1 -> 2'), + (1, 3, 'arc 1 -> 3'), + (2, 3, 'arc 2 -> 3'), + (1, 4, 'arc 1 -> 4'), + (4, 5, 'arc 4 -> 5'), + (5, 1, 'arc 5 -> 1') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE y (a INTEGER) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE y +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE y (a INTEGER) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) +-- !query schema +struct<> +-- !query output + + + +-- !query +with cte(foo) as ( select 42 ) select * from ((select foo from cte)) q +-- !query schema +struct +-- !query output +42 + + +-- !query +WITH outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM innermost + UNION SELECT 3) +) +SELECT * FROM outermost ORDER BY 1 +-- !query schema +struct +-- !query output +1 +2 +3 + + +-- !query +WITH outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM outermost -- fail + UNION SELECT * FROM innermost) +) +SELECT * FROM outermost ORDER BY 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`outermost`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 96, + "stopIndex" : 104, + "fragment" : "outermost" + } ] +} + + +-- !query +CREATE TABLE withz USING parquet AS SELECT i AS k, CAST(i AS string) || ' v' AS v FROM (SELECT EXPLODE(SEQUENCE(1, 16, 3)) i) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM withz ORDER BY k +-- !query schema +struct +-- !query output +1 1 v +4 4 v +7 7 v +10 10 v +13 13 v +16 16 v + + +-- !query +DROP TABLE withz +-- !query schema +struct<> +-- !query output + + + +-- !query +TRUNCATE TABLE y +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 3)) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE yy (a INTEGER) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM y +-- !query schema +struct +-- !query output +1 +2 +3 + + +-- !query +SELECT * FROM yy +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM y +-- !query schema +struct +-- !query output +1 +2 +3 + + +-- !query +SELECT * FROM yy +-- !query schema +struct +-- !query output + + + +-- !query +CREATE TABLE parent ( id int, val string ) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO parent VALUES ( 1, 'p1' ) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM parent +-- !query schema +struct +-- !query output +1 p1 + + +-- !query +SELECT * FROM parent +-- !query schema +struct +-- !query output +1 p1 + + +-- !query +create table foo (with baz) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_DATATYPE", + "sqlState" : "0A000", + "messageParameters" : { + "typeName" : "\"BAZ\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 26, + "fragment" : "baz" + } ] +} + + +-- !query +create table foo (with ordinality) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_DATATYPE", + "sqlState" : "0A000", + "messageParameters" : { + "typeName" : "\"ORDINALITY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 33, + "fragment" : "ordinality" + } ] +} + + +-- !query +with ordinality as (select 1 as x) select * from ordinality +-- !query schema +struct +-- !query output +1 + + +-- !query +WITH test AS (SELECT 42) INSERT INTO test VALUES (1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`test`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 38, + "stopIndex" : 41, + "fragment" : "test" + } ] +} + + +-- !query +create table test (i int) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +with test as (select 42) insert into test select * from test +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from test +-- !query schema +struct +-- !query output +42 + + +-- !query +drop table test +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE department +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE tree +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE graph +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE y +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE yy +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE parent +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/pred-pushdown.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/pred-pushdown.sql.out new file mode 100644 index 000000000000..da4f1637637f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/pred-pushdown.sql.out @@ -0,0 +1,37 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW tbl_a AS VALUES (1, 1), (2, 1), (3, 6) AS T(c1, c2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW tbl_b AS VALUES 1 AS T(c1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * +FROM tbl_a + LEFT ANTI JOIN tbl_b ON ((tbl_a.c1 = tbl_a.c2) IS NULL OR tbl_a.c1 = tbl_a.c2) +-- !query schema +struct +-- !query output +2 1 +3 6 + + +-- !query +SELECT l.c1, l.c2 +FROM tbl_a l +WHERE EXISTS (SELECT 1 FROM tbl_b r WHERE l.c1 = l.c2) OR l.c2 < 2 +-- !query schema +struct +-- !query output +1 1 +2 1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/predicate-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/predicate-functions.sql.out new file mode 100644 index 000000000000..10ff268a5606 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/predicate-functions.sql.out @@ -0,0 +1,896 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select not true +-- !query schema +struct<(NOT true):boolean> +-- !query output +false + + +-- !query +select ! true +-- !query schema +struct<(NOT true):boolean> +-- !query output +false + + +-- !query +select not null::boolean +-- !query schema +struct<(NOT CAST(NULL AS BOOLEAN)):boolean> +-- !query output +NULL + + +-- !query +select true and true +-- !query schema +struct<(true AND true):boolean> +-- !query output +true + + +-- !query +select true and false +-- !query schema +struct<(true AND false):boolean> +-- !query output +false + + +-- !query +select false and true +-- !query schema +struct<(false AND true):boolean> +-- !query output +false + + +-- !query +select false and false +-- !query schema +struct<(false AND false):boolean> +-- !query output +false + + +-- !query +select true and null::boolean +-- !query schema +struct<(true AND CAST(NULL AS BOOLEAN)):boolean> +-- !query output +NULL + + +-- !query +select false and null::boolean +-- !query schema +struct<(false AND CAST(NULL AS BOOLEAN)):boolean> +-- !query output +false + + +-- !query +select null::boolean and true +-- !query schema +struct<(CAST(NULL AS BOOLEAN) AND true):boolean> +-- !query output +NULL + + +-- !query +select null::boolean and false +-- !query schema +struct<(CAST(NULL AS BOOLEAN) AND false):boolean> +-- !query output +false + + +-- !query +select null::boolean and null::boolean +-- !query schema +struct<(CAST(NULL AS BOOLEAN) AND CAST(NULL AS BOOLEAN)):boolean> +-- !query output +NULL + + +-- !query +select true or true +-- !query schema +struct<(true OR true):boolean> +-- !query output +true + + +-- !query +select true or false +-- !query schema +struct<(true OR false):boolean> +-- !query output +true + + +-- !query +select false or true +-- !query schema +struct<(false OR true):boolean> +-- !query output +true + + +-- !query +select false or false +-- !query schema +struct<(false OR false):boolean> +-- !query output +false + + +-- !query +select true or null::boolean +-- !query schema +struct<(true OR CAST(NULL AS BOOLEAN)):boolean> +-- !query output +true + + +-- !query +select false or null::boolean +-- !query schema +struct<(false OR CAST(NULL AS BOOLEAN)):boolean> +-- !query output +NULL + + +-- !query +select null::boolean or true +-- !query schema +struct<(CAST(NULL AS BOOLEAN) OR true):boolean> +-- !query output +true + + +-- !query +select null::boolean or false +-- !query schema +struct<(CAST(NULL AS BOOLEAN) OR false):boolean> +-- !query output +NULL + + +-- !query +select null::boolean or null::boolean +-- !query schema +struct<(CAST(NULL AS BOOLEAN) OR CAST(NULL AS BOOLEAN)):boolean> +-- !query output +NULL + + +-- !query +select 1 = 1 +-- !query schema +struct<(1 = 1):boolean> +-- !query output +true + + +-- !query +select 1 = '1' +-- !query schema +struct<(1 = 1):boolean> +-- !query output +true + + +-- !query +select 1.0 = '1' +-- !query schema +struct<(1.0 = 1):boolean> +-- !query output +true + + +-- !query +select 1.5 = '1.51' +-- !query schema +struct<(1.5 = 1.51):boolean> +-- !query output +false + + +-- !query +select 1 > '1' +-- !query schema +struct<(1 > 1):boolean> +-- !query output +false + + +-- !query +select 2 > '1.0' +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1.0'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "2 > '1.0'" + } ] +} + + +-- !query +select 2 > '2.0' +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'2.0'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "2 > '2.0'" + } ] +} + + +-- !query +select 2 > '2.2' +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'2.2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "2 > '2.2'" + } ] +} + + +-- !query +select '1.5' > 0.5 +-- !query schema +struct<(1.5 > 0.5):boolean> +-- !query output +true + + +-- !query +select to_date('2009-07-30 04:17:52') > to_date('2009-07-30 04:17:52') +-- !query schema +struct<(to_date(2009-07-30 04:17:52) > to_date(2009-07-30 04:17:52)):boolean> +-- !query output +false + + +-- !query +select to_date('2009-07-30 04:17:52') > '2009-07-30 04:17:52' +-- !query schema +struct<(to_date(2009-07-30 04:17:52) > 2009-07-30 04:17:52):boolean> +-- !query output +false + + +-- !query +select 1 >= '1' +-- !query schema +struct<(1 >= 1):boolean> +-- !query output +true + + +-- !query +select 2 >= '1.0' +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1.0'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "2 >= '1.0'" + } ] +} + + +-- !query +select 2 >= '2.0' +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'2.0'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "2 >= '2.0'" + } ] +} + + +-- !query +select 2.0 >= '2.2' +-- !query schema +struct<(2.0 >= 2.2):boolean> +-- !query output +false + + +-- !query +select '1.5' >= 0.5 +-- !query schema +struct<(1.5 >= 0.5):boolean> +-- !query output +true + + +-- !query +select to_date('2009-07-30 04:17:52') >= to_date('2009-07-30 04:17:52') +-- !query schema +struct<(to_date(2009-07-30 04:17:52) >= to_date(2009-07-30 04:17:52)):boolean> +-- !query output +true + + +-- !query +select to_date('2009-07-30 04:17:52') >= '2009-07-30 04:17:52' +-- !query schema +struct<(to_date(2009-07-30 04:17:52) >= 2009-07-30 04:17:52):boolean> +-- !query output +true + + +-- !query +select 1 < '1' +-- !query schema +struct<(1 < 1):boolean> +-- !query output +false + + +-- !query +select 2 < '1.0' +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1.0'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "2 < '1.0'" + } ] +} + + +-- !query +select 2 < '2.0' +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'2.0'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "2 < '2.0'" + } ] +} + + +-- !query +select 2.0 < '2.2' +-- !query schema +struct<(2.0 < 2.2):boolean> +-- !query output +true + + +-- !query +select 0.5 < '1.5' +-- !query schema +struct<(0.5 < 1.5):boolean> +-- !query output +true + + +-- !query +select to_date('2009-07-30 04:17:52') < to_date('2009-07-30 04:17:52') +-- !query schema +struct<(to_date(2009-07-30 04:17:52) < to_date(2009-07-30 04:17:52)):boolean> +-- !query output +false + + +-- !query +select to_date('2009-07-30 04:17:52') < '2009-07-30 04:17:52' +-- !query schema +struct<(to_date(2009-07-30 04:17:52) < 2009-07-30 04:17:52):boolean> +-- !query output +false + + +-- !query +select 1 <= '1' +-- !query schema +struct<(1 <= 1):boolean> +-- !query output +true + + +-- !query +select 2 <= '1.0' +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1.0'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "2 <= '1.0'" + } ] +} + + +-- !query +select 2 <= '2.0' +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'2.0'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "2 <= '2.0'" + } ] +} + + +-- !query +select 2.0 <= '2.2' +-- !query schema +struct<(2.0 <= 2.2):boolean> +-- !query output +true + + +-- !query +select 0.5 <= '1.5' +-- !query schema +struct<(0.5 <= 1.5):boolean> +-- !query output +true + + +-- !query +select to_date('2009-07-30 04:17:52') <= to_date('2009-07-30 04:17:52') +-- !query schema +struct<(to_date(2009-07-30 04:17:52) <= to_date(2009-07-30 04:17:52)):boolean> +-- !query output +true + + +-- !query +select to_date('2009-07-30 04:17:52') <= '2009-07-30 04:17:52' +-- !query schema +struct<(to_date(2009-07-30 04:17:52) <= 2009-07-30 04:17:52):boolean> +-- !query output +true + + +-- !query +select to_date('2017-03-01') = to_timestamp('2017-03-01 00:00:00') +-- !query schema +struct<(to_date(2017-03-01) = to_timestamp(2017-03-01 00:00:00)):boolean> +-- !query output +true + + +-- !query +select to_timestamp('2017-03-01 00:00:01') > to_date('2017-03-01') +-- !query schema +struct<(to_timestamp(2017-03-01 00:00:01) > to_date(2017-03-01)):boolean> +-- !query output +true + + +-- !query +select to_timestamp('2017-03-01 00:00:01') >= to_date('2017-03-01') +-- !query schema +struct<(to_timestamp(2017-03-01 00:00:01) >= to_date(2017-03-01)):boolean> +-- !query output +true + + +-- !query +select to_date('2017-03-01') < to_timestamp('2017-03-01 00:00:01') +-- !query schema +struct<(to_date(2017-03-01) < to_timestamp(2017-03-01 00:00:01)):boolean> +-- !query output +true + + +-- !query +select to_date('2017-03-01') <= to_timestamp('2017-03-01 00:00:01') +-- !query schema +struct<(to_date(2017-03-01) <= to_timestamp(2017-03-01 00:00:01)):boolean> +-- !query output +true + + +-- !query +select 1 in (1, 2, 3) +-- !query schema +struct<(1 IN (1, 2, 3)):boolean> +-- !query output +true + + +-- !query +select 1 in (1, 2, 3, null) +-- !query schema +struct<(1 IN (1, 2, 3, NULL)):boolean> +-- !query output +true + + +-- !query +select 1 in (1.0, 2.0, 3.0) +-- !query schema +struct<(1 IN (1.0, 2.0, 3.0)):boolean> +-- !query output +true + + +-- !query +select 1 in (1.0, 2.0, 3.0, null) +-- !query schema +struct<(1 IN (1.0, 2.0, 3.0, NULL)):boolean> +-- !query output +true + + +-- !query +select 1 in ('2', '3', '4') +-- !query schema +struct<(1 IN (2, 3, 4)):boolean> +-- !query output +false + + +-- !query +select 1 in ('2', '3', '4', null) +-- !query schema +struct<(1 IN (2, 3, 4, NULL)):boolean> +-- !query output +NULL + + +-- !query +select null in (1, 2, 3) +-- !query schema +struct<(NULL IN (1, 2, 3)):boolean> +-- !query output +NULL + + +-- !query +select null in (1, 2, null) +-- !query schema +struct<(NULL IN (1, 2, NULL)):boolean> +-- !query output +NULL + + +-- !query +select 1 not in (1, 2, 3) +-- !query schema +struct<(NOT (1 IN (1, 2, 3))):boolean> +-- !query output +false + + +-- !query +select 1 not in (1, 2, 3, null) +-- !query schema +struct<(NOT (1 IN (1, 2, 3, NULL))):boolean> +-- !query output +false + + +-- !query +select 1 not in (1.0, 2.0, 3.0) +-- !query schema +struct<(NOT (1 IN (1.0, 2.0, 3.0))):boolean> +-- !query output +false + + +-- !query +select 1 not in (1.0, 2.0, 3.0, null) +-- !query schema +struct<(NOT (1 IN (1.0, 2.0, 3.0, NULL))):boolean> +-- !query output +false + + +-- !query +select 1 not in ('2', '3', '4') +-- !query schema +struct<(NOT (1 IN (2, 3, 4))):boolean> +-- !query output +true + + +-- !query +select 1 not in ('2', '3', '4', null) +-- !query schema +struct<(NOT (1 IN (2, 3, 4, NULL))):boolean> +-- !query output +NULL + + +-- !query +select null not in (1, 2, 3) +-- !query schema +struct<(NOT (NULL IN (1, 2, 3))):boolean> +-- !query output +NULL + + +-- !query +select null not in (1, 2, null) +-- !query schema +struct<(NOT (NULL IN (1, 2, NULL))):boolean> +-- !query output +NULL + + +-- !query +select 1 between 0 and 2 +-- !query schema +struct +-- !query output +true + + +-- !query +select 0.5 between 0 and 1 +-- !query schema +struct +-- !query output +true + + +-- !query +select 2.0 between '1.0' and '3.0' +-- !query schema +struct +-- !query output +true + + +-- !query +select 'b' between 'a' and 'c' +-- !query schema +struct +-- !query output +true + + +-- !query +select to_timestamp('2022-12-26 00:00:01') between to_date('2022-03-01') and to_date('2022-12-31') +-- !query schema +struct +-- !query output +true + + +-- !query +select rand(123) between 0.1 AND 0.2 +-- !query schema +struct +-- !query output +true + + +-- !query +select 1 not between 0 and 2 +-- !query schema +struct<(NOT between(1, 0, 2)):boolean> +-- !query output +false + + +-- !query +select 0.5 not between 0 and 1 +-- !query schema +struct<(NOT between(0.5, 0, 1)):boolean> +-- !query output +false + + +-- !query +select 2.0 not between '1.0' and '3.0' +-- !query schema +struct<(NOT between(2.0, 1.0, 3.0)):boolean> +-- !query output +false + + +-- !query +select 'b' not between 'a' and 'c' +-- !query schema +struct<(NOT between(b, a, c)):boolean> +-- !query output +false + + +-- !query +select to_timestamp('2022-12-26 00:00:01') not between to_date('2022-03-01') and to_date('2022-12-31') +-- !query schema +struct<(NOT between(to_timestamp(2022-12-26 00:00:01), to_date(2022-03-01), to_date(2022-12-31))):boolean> +-- !query output +false + + +-- !query +select rand(123) not between 0.1 AND 0.2 +-- !query schema +struct<(NOT between(rand(123), 0.1, 0.2)):boolean> +-- !query output +false + + +-- !query +set spark.sql.legacy.bangEqualsNot=true +-- !query schema +struct +-- !query output +spark.sql.legacy.bangEqualsNot true + + +-- !query +select 1 ! between 0 and 2 +-- !query schema +struct<(NOT between(1, 0, 2)):boolean> +-- !query output +false + + +-- !query +select 1 ! in (3, 4) +-- !query schema +struct<(NOT (1 IN (3, 4))):boolean> +-- !query output +true + + +-- !query +select 'hello' ! like 'world' +-- !query schema +struct<(NOT hello LIKE world):boolean> +-- !query output +true + + +-- !query +select 1 is ! null +-- !query schema +struct<(1 IS NOT NULL):boolean> +-- !query output +true + + +-- !query +select false is ! true +-- !query schema +struct<(NOT (false <=> true)):boolean> +-- !query output +true + + +-- !query +set spark.sql.legacy.bangEqualsNot=false +-- !query schema +struct +-- !query output +spark.sql.legacy.bangEqualsNot false diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/query_regex_column.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/query_regex_column.sql.out new file mode 100644 index 000000000000..8df97c723e10 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/query_regex_column.sql.out @@ -0,0 +1,422 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +set spark.sql.parser.quotedRegexColumnNames=false +-- !query schema +struct +-- !query output +spark.sql.parser.quotedRegexColumnNames false + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, "1", "11"), (2, "2", "22"), (3, "3", "33"), (4, "4", "44"), (5, "5", "55"), (6, "6", "66") +AS testData(key, value1, value2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData2 AS SELECT * FROM VALUES +(1, 1, 1, 2), (1, 2, 1, 2), (2, 1, 2, 3), (2, 2, 2, 3), (3, 1, 3, 4), (3, 2, 3, 4) +AS testData2(A, B, c, d) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT `(a)?+.+` FROM testData2 WHERE a = 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`(a)?+.+`", + "proposal" : "`A`, `B`, `c`, `d`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "`(a)?+.+`" + } ] +} + + +-- !query +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t`.`(a)?+.+`", + "proposal" : "`t`.`A`, `t`.`B`, `t`.`c`, `t`.`d`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "t.`(a)?+.+`" + } ] +} + + +-- !query +SELECT `(a|b)` FROM testData2 WHERE a = 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`(a|b)`", + "proposal" : "`A`, `B`, `c`, `d`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 14, + "fragment" : "`(a|b)`" + } ] +} + + +-- !query +SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`(a|b)?+.+`", + "proposal" : "`A`, `B`, `c`, `d`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "`(a|b)?+.+`" + } ] +} + + +-- !query +SELECT SUM(`(a|b)?+.+`) FROM testData2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`(a|b)?+.+`", + "proposal" : "`A`, `B`, `c`, `d`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 22, + "fragment" : "`(a|b)?+.+`" + } ] +} + + +-- !query +SELECT SUM(`(a)`) FROM testData2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`(a)`", + "proposal" : "`A`, `B`, `c`, `d`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 16, + "fragment" : "`(a)`" + } ] +} + + +-- !query +set spark.sql.parser.quotedRegexColumnNames=true +-- !query schema +struct +-- !query output +spark.sql.parser.quotedRegexColumnNames true + + +-- !query +SELECT `(a)?+.+` FROM testData2 WHERE a = 1 +-- !query schema +struct +-- !query output +1 1 2 +2 1 2 + + +-- !query +SELECT `(A)?+.+` FROM testData2 WHERE a = 1 +-- !query schema +struct +-- !query output +1 1 2 +2 1 2 + + +-- !query +SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1 +-- !query schema +struct +-- !query output +1 1 2 +2 1 2 + + +-- !query +SELECT t.`(A)?+.+` FROM testData2 t WHERE a = 1 +-- !query schema +struct +-- !query output +1 1 2 +2 1 2 + + +-- !query +SELECT `(a|B)` FROM testData2 WHERE a = 2 +-- !query schema +struct +-- !query output +2 1 +2 2 + + +-- !query +SELECT `(A|b)` FROM testData2 WHERE a = 2 +-- !query schema +struct +-- !query output +2 1 +2 2 + + +-- !query +SELECT `(a|B)?+.+` FROM testData2 WHERE a = 2 +-- !query schema +struct +-- !query output +2 3 +2 3 + + +-- !query +SELECT `(A|b)?+.+` FROM testData2 WHERE a = 2 +-- !query schema +struct +-- !query output +2 3 +2 3 + + +-- !query +SELECT `(e|f)` FROM testData2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t.`(e|f)` FROM testData2 t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT p.`(KEY)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 +-- !query schema +struct +-- !query output +1 11 1 1 1 2 +1 11 2 1 1 2 +2 22 1 2 2 3 +2 22 2 2 2 3 + + +-- !query +SELECT p.`(key)?+.+`, b, testdata2.`(b)?+.+` FROM testData p join testData2 ON p.key = testData2.a WHERE key < 3 +-- !query schema +struct +-- !query output +1 11 1 1 1 2 +1 11 2 1 1 2 +2 22 1 2 2 3 +2 22 2 2 2 3 + + +-- !query +set spark.sql.caseSensitive=true +-- !query schema +struct +-- !query output +spark.sql.caseSensitive true + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testdata3 AS SELECT * FROM VALUES +(0, 1), (1, 2), (2, 3), (3, 4) +AS testdata3(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT `(A)?+.+` FROM testdata3 +-- !query schema +struct +-- !query output +0 1 +1 2 +2 3 +3 4 + + +-- !query +SELECT `(a)?+.+` FROM testdata3 +-- !query schema +struct +-- !query output +1 +2 +3 +4 + + +-- !query +SELECT `(A)?+.+` FROM testdata3 WHERE a > 1 +-- !query schema +struct +-- !query output +2 3 +3 4 + + +-- !query +SELECT `(a)?+.+` FROM testdata3 where `a` > 1 +-- !query schema +struct +-- !query output +3 +4 + + +-- !query +SELECT SUM(`a`) FROM testdata3 +-- !query schema +struct +-- !query output +6 + + +-- !query +SELECT SUM(`(a)`) FROM testdata3 +-- !query schema +struct +-- !query output +6 + + +-- !query +SELECT SUM(`(a)?+.+`) FROM testdata3 +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT SUM(a) FROM testdata3 GROUP BY `a` +-- !query schema +struct +-- !query output +0 +1 +2 +3 + + +-- !query +SELECT SUM(a) FROM testdata3 GROUP BY `(a)` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`(a)`", + "proposal" : "`a`, `b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 43, + "fragment" : "`(a)`" + } ] +} + + +-- !query +SELECT SUM(a) FROM testdata3 GROUP BY `(a)?+.+` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`(a)?+.+`", + "proposal" : "`a`, `b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 47, + "fragment" : "`(a)?+.+`" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/random.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/random.sql.out new file mode 100644 index 000000000000..049d134003cd --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/random.sql.out @@ -0,0 +1,921 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT rand(0) +-- !query schema +struct +-- !query output +0.7604953758285915 + + +-- !query +SELECT rand(cast(3 / 7 AS int)) +-- !query schema +struct +-- !query output +0.7604953758285915 + + +-- !query +SELECT rand(NULL) +-- !query schema +struct +-- !query output +0.7604953758285915 + + +-- !query +SELECT rand(cast(NULL AS int)) +-- !query schema +struct +-- !query output +0.7604953758285915 + + +-- !query +SELECT rand(1.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.0\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "first", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"rand(1.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "rand(1.0)" + } ] +} + + +-- !query +SELECT randn(0L) +-- !query schema +struct +-- !query output +1.6034991609278433 + + +-- !query +SELECT randn(cast(3 / 7 AS long)) +-- !query schema +struct +-- !query output +1.6034991609278433 + + +-- !query +SELECT randn(NULL) +-- !query schema +struct +-- !query output +1.6034991609278433 + + +-- !query +SELECT randn(cast(NULL AS long)) +-- !query schema +struct +-- !query output +1.6034991609278433 + + +-- !query +SELECT rand('1') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"STRING\"", + "paramIndex" : "first", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"rand(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "rand('1')" + } ] +} + + +-- !query +SELECT uniform(0, 1, 0) AS result +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT uniform(0, 10, 0) AS result +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT uniform(0L, 10L, 0) AS result +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT uniform(0, 10L, 0) AS result +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT uniform(0, cast(10 as tinyint), 0) AS result +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT uniform(0, cast(10 as smallint), 0) AS result +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT uniform(0, 10S, 0) AS result +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT uniform(10, 20, 0) AS result +-- !query schema +struct +-- !query output +17 + + +-- !query +SELECT uniform(10.0F, 20.0F, 0) AS result +-- !query schema +struct +-- !query output +17.604954 + + +-- !query +SELECT uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(10, 3)), 0) AS result +-- !query schema +struct +-- !query output +17.605 + + +-- !query +SELECT uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(11, 4)), 0) AS result +-- !query schema +struct +-- !query output +17.6050 + + +-- !query +SELECT uniform(10, cast(20 as decimal(10, 3)), 0) AS result +-- !query schema +struct +-- !query output +17.605 + + +-- !query +SELECT uniform(cast(10 as decimal(10, 3)), 20, 0) AS result +-- !query schema +struct +-- !query output +17.605 + + +-- !query +SELECT uniform(10.0D, 20.0D, CAST(3 / 7 AS LONG)) AS result +-- !query schema +struct +-- !query output +17.604953758285916 + + +-- !query +SELECT uniform(10, 20.0F, 0) AS result +-- !query schema +struct +-- !query output +17.604954 + + +-- !query +SELECT uniform(10, 20, 0) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query schema +struct +-- !query output +15 +16 +17 + + +-- !query +SELECT uniform(10, 20.0F) IS NOT NULL AS result +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT uniform(-10L, 10L, 0) AS result +-- !query schema +struct +-- !query output +5 + + +-- !query +SELECT uniform(-20L, -10L, 0) AS result +-- !query schema +struct +-- !query output +-12 + + +-- !query +SELECT uniform(-20L, -10L, -10) AS result +-- !query schema +struct +-- !query output +-17 + + +-- !query +SELECT uniform(NULL, 1, 0) AS result +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT uniform(cast(NULL AS int), 1, 0) AS result +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT uniform(cast(NULL AS float), 1, 0) AS result +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT uniform(0, NULL, 0) AS result +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT uniform(0, cast(NULL AS int), 0) AS result +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT uniform(0, cast(NULL AS float), 0) AS result +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT uniform(0, 1, NULL) AS result +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT uniform(NULL, NULL, 0) AS result +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT uniform(NULL, NULL, NULL) AS result +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT uniform(0, 1, cast(NULL as int)) AS result +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT uniform(0, 1, cast(NULL as float)) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(NULL AS FLOAT)\"", + "inputType" : "\"FLOAT\"", + "paramIndex" : "third", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"uniform(0, 1, CAST(NULL AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "uniform(0, 1, cast(NULL as float))" + } ] +} + + +-- !query +SELECT uniform(10, 20, col) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"col\"", + "inputName" : "`seed`", + "inputType" : "integer or floating-point", + "sqlExpr" : "\"uniform(10, 20, col)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "uniform(10, 20, col)" + } ] +} + + +-- !query +SELECT uniform(col, 10, 0) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"col\"", + "inputName" : "`min`", + "inputType" : "integer or floating-point", + "sqlExpr" : "\"uniform(col, 10, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "uniform(col, 10, 0)" + } ] +} + + +-- !query +SELECT uniform(10) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3]", + "functionName" : "`uniform`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 18, + "fragment" : "uniform(10)" + } ] +} + + +-- !query +SELECT uniform(10, 20, 30, 40) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "4", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3]", + "functionName" : "`uniform`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "uniform(10, 20, 30, 40)" + } ] +} + + +-- !query +SELECT uniform(10.0F, 20.0F, 0.0F) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"0.0\"", + "inputType" : "\"FLOAT\"", + "paramIndex" : "third", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"uniform(10.0, 20.0, 0.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "uniform(10.0F, 20.0F, 0.0F)" + } ] +} + + +-- !query +SELECT uniform(10.0F, 20.0F, 0.0D) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"0.0\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "third", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"uniform(10.0, 20.0, 0.0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "uniform(10.0F, 20.0F, 0.0D)" + } ] +} + + +-- !query +SELECT uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(10, 3)), cast(0 as decimal(10, 3))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(0 AS DECIMAL(10,3))\"", + "inputType" : "\"DECIMAL(10,3)\"", + "paramIndex" : "third", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"uniform(CAST(10 AS DECIMAL(10,3)), CAST(20 AS DECIMAL(10,3)), CAST(0 AS DECIMAL(10,3)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "uniform(cast(10 as decimal(10, 3)), cast(20 as decimal(10, 3)), cast(0 as decimal(10, 3)))" + } ] +} + + +-- !query +SELECT uniform('abc', 10, 0) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"abc\"", + "inputType" : "\"STRING\"", + "paramIndex" : "first", + "requiredType" : "\"NUMERIC\"", + "sqlExpr" : "\"uniform(abc, 10, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "uniform('abc', 10, 0)" + } ] +} + + +-- !query +SELECT uniform(0, 'def', 0) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"def\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"NUMERIC\"", + "sqlExpr" : "\"uniform(0, def, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "uniform(0, 'def', 0)" + } ] +} + + +-- !query +SELECT uniform(0, 10, 'ghi') AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"ghi\"", + "inputType" : "\"STRING\"", + "paramIndex" : "third", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"uniform(0, 10, ghi)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "uniform(0, 10, 'ghi')" + } ] +} + + +-- !query +SELECT randstr(1, 0) AS result +-- !query schema +struct +-- !query output +c + + +-- !query +SELECT randstr(5, 0) AS result +-- !query schema +struct +-- !query output +ceV0P + + +-- !query +SELECT randstr(10, 0) AS result +-- !query schema +struct +-- !query output +ceV0PXaR2I + + +-- !query +SELECT randstr(10S, 0) AS result +-- !query schema +struct +-- !query output +ceV0PXaR2I + + +-- !query +SELECT randstr(CAST(10 AS TINYINT), 0) AS result +-- !query schema +struct +-- !query output +ceV0PXaR2I + + +-- !query +SELECT randstr(CAST(10 AS BIGINT), 0) AS result +-- !query schema +struct +-- !query output +ceV0PXaR2I + + +-- !query +SELECT randstr(1.0F, 0) AS result +-- !query schema +struct +-- !query output +c + + +-- !query +SELECT randstr(1.0D, 0) AS result +-- !query schema +struct +-- !query output +c + + +-- !query +SELECT randstr(cast(1 AS DECIMAL(10, 2)), 0) AS result +-- !query schema +struct +-- !query output +c + + +-- !query +SELECT randstr(10, 0) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query schema +struct +-- !query output +ceV0PXaR2I +fYxVfArnv7 +iSIv0VT2XL + + +-- !query +SELECT randstr(10) IS NOT NULL AS result +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT randstr(1, -1) AS result +-- !query schema +struct +-- !query output +S + + +-- !query +SELECT randstr(10L, 0) AS result +-- !query schema +struct +-- !query output +ceV0PXaR2I + + +-- !query +SELECT randstr(10.0F, 0) AS result +-- !query schema +struct +-- !query output +ceV0PXaR2I + + +-- !query +SELECT randstr(10.0D, 0) AS result +-- !query schema +struct +-- !query output +ceV0PXaR2I + + +-- !query +SELECT randstr(NULL, 0) AS result +-- !query schema +struct +-- !query output + + + +-- !query +SELECT randstr(0, NULL) AS result +-- !query schema +struct +-- !query output + + + +-- !query +SELECT randstr(col, 0) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"col\"", + "inputName" : "`length`", + "inputType" : "integer", + "sqlExpr" : "\"randstr(col, 0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "randstr(col, 0)" + } ] +} + + +-- !query +SELECT randstr(10, col) AS result FROM VALUES (0), (1), (2) tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"col\"", + "inputName" : "`seed`", + "inputType" : "integer", + "sqlExpr" : "\"randstr(10, col)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "randstr(10, col)" + } ] +} + + +-- !query +SELECT randstr(10, 0, 1) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "3", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[1, 2]", + "functionName" : "`randstr`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "randstr(10, 0, 1)" + } ] +} + + +-- !query +SELECT randstr(-1, 0) AS result +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.LENGTH", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`randstr`", + "length" : "-1", + "parameter" : "`length`" + } +} + + +-- !query +SELECT randstr(10, "a") AS result FROM VALUES (0) tab(a) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"a\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"randstr(10, a)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "randstr(10, \"a\")" + } ] +} + + +-- !query +SELECT randstr(10, 1.5) AS result FROM VALUES (0) tab(a) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1.5\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"BIGINT\")", + "sqlExpr" : "\"randstr(10, 1.5)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "randstr(10, 1.5)" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/regexp-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/regexp-functions.sql.out new file mode 100644 index 000000000000..62a0e5ff917c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/regexp-functions.sql.out @@ -0,0 +1,732 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.REGEX_GROUP_INDEX", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`regexp_extract`", + "groupCount" : "0", + "groupIndex" : "1", + "parameter" : "`idx`" + } +} + + +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+', 0) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+', 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.REGEX_GROUP_INDEX", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`regexp_extract`", + "groupCount" : "0", + "groupIndex" : "1", + "parameter" : "`idx`" + } +} + + +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+', 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.REGEX_GROUP_INDEX", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`regexp_extract`", + "groupCount" : "0", + "groupIndex" : "2", + "parameter" : "`idx`" + } +} + + +-- !query +SELECT regexp_extract('1a 2b 14m', '\\d+', -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.REGEX_GROUP_INDEX", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`regexp_extract`", + "groupCount" : "0", + "groupIndex" : "-1", + "parameter" : "`idx`" + } +} + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)?', 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('a b m', '(\\d+)?', 1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)') +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 0) +-- !query schema +struct +-- !query output +1a + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 2) +-- !query schema +struct +-- !query output +a + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', 3) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.REGEX_GROUP_INDEX", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`regexp_extract`", + "groupCount" : "2", + "groupIndex" : "3", + "parameter" : "`idx`" + } +} + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)([a-z]+)', -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.REGEX_GROUP_INDEX", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`regexp_extract`", + "groupCount" : "2", + "groupIndex" : "-1", + "parameter" : "`idx`" + } +} + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_extract('a b m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT regexp_extract('1a 2b 14m', '(?l)') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.PATTERN", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`regexp_extract`", + "parameter" : "`regexp`", + "value" : "'(?l)'" + } +} + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.REGEX_GROUP_INDEX", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`regexp_extract_all`", + "groupCount" : "0", + "groupIndex" : "1", + "parameter" : "`idx`" + } +} + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 0) +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.REGEX_GROUP_INDEX", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`regexp_extract_all`", + "groupCount" : "0", + "groupIndex" : "1", + "parameter" : "`idx`" + } +} + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.REGEX_GROUP_INDEX", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`regexp_extract_all`", + "groupCount" : "0", + "groupIndex" : "2", + "parameter" : "`idx`" + } +} + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '\\d+', -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.REGEX_GROUP_INDEX", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`regexp_extract_all`", + "groupCount" : "0", + "groupIndex" : "-1", + "parameter" : "`idx`" + } +} + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?', 1) +-- !query schema +struct> +-- !query output +["1","","","2","","","14","",""] + + +-- !query +SELECT regexp_extract_all('a 2b 14m', '(\\d+)?', 1) +-- !query schema +struct> +-- !query output +["","","2","","","14","",""] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)') +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 0) +-- !query schema +struct> +-- !query output +["1a","2b","14m"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 1) +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 2) +-- !query schema +struct> +-- !query output +["a","b","m"] + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', 3) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.REGEX_GROUP_INDEX", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`regexp_extract_all`", + "groupCount" : "2", + "groupIndex" : "3", + "parameter" : "`idx`" + } +} + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)([a-z]+)', -1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.REGEX_GROUP_INDEX", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`regexp_extract_all`", + "groupCount" : "2", + "groupIndex" : "-1", + "parameter" : "`idx`" + } +} + + +-- !query +SELECT regexp_extract_all('1a 2b 14m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct> +-- !query output +["1","2","14"] + + +-- !query +SELECT regexp_extract_all('a 2b 14m', '(\\d+)?([a-z]+)', 1) +-- !query schema +struct> +-- !query output +["","2","14"] + + +-- !query +SELECT regexp_extract_all('abc', col0, 1) FROM VALUES('], [') AS t(col0) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.PATTERN", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`regexp_extract_all`", + "parameter" : "`regexp`", + "value" : "'], ['" + } +} + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something') +-- !query schema +struct +-- !query output +something, something, and wise + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', -2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", + "sqlState" : "42K09", + "messageParameters" : { + "currentValue" : "-2", + "exprName" : "position", + "sqlExpr" : "\"regexp_replace(healthy, wealthy, and wise, \\w+thy, something, -2)\"", + "valueRange" : "(0, 2147483647]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "regexp_replace('healthy, wealthy, and wise', '\\\\w+thy', 'something', -2)" + } ] +} + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE", + "sqlState" : "42K09", + "messageParameters" : { + "currentValue" : "0", + "exprName" : "position", + "sqlExpr" : "\"regexp_replace(healthy, wealthy, and wise, \\w+thy, something, 0)\"", + "valueRange" : "(0, 2147483647]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "regexp_replace('healthy, wealthy, and wise', '\\\\w+thy', 'something', 0)" + } ] +} + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 1) +-- !query schema +struct +-- !query output +something, something, and wise + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 2) +-- !query schema +struct +-- !query output +hsomething, something, and wise + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w+thy', 'something', 8) +-- !query schema +struct +-- !query output +healthy, something, and wise + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', 26) +-- !query schema +struct +-- !query output +healthy, wealthy, and wissomething + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', 27) +-- !query schema +struct +-- !query output +healthy, wealthy, and wise + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', 30) +-- !query schema +struct +-- !query output +healthy, wealthy, and wise + + +-- !query +SELECT regexp_replace('healthy, wealthy, and wise', '\\w', 'something', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT regexp_like('1a 2b 14m', '\\d+b') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT regexp_like('1a 2b 14m', '[a-z]+b') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT regexp('1a 2b 14m', '\\d+b') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT regexp('1a 2b 14m', '[a-z]+b') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT rlike('1a 2b 14m', '\\d+b') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT rlike('1a 2b 14m', '[a-z]+b') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT regexp_count('1a 2b 14m', '\\d+') +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT regexp_count('1a 2b 14m', 'mmm') +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT regexp_count('the fox', 'FOX') +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT regexp_count('the fox', '(?i)FOX') +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_count('passwd7 plain A1234 a1234', '(?=[^ ]*[a-z])(?=[^ ]*[0-9])[^ ]+') +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT regexp_count(null, 'abc') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT regexp_count('abc', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT regexp_substr('1a 2b 14m', '\\d+') +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT regexp_substr('1a 2b 14m', '\\d+ ') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT regexp_substr('1a 2b 14m', '\\d+(a|b|m)') +-- !query schema +struct +-- !query output +1a + + +-- !query +SELECT regexp_substr('1a 2b 14m', '\\d{2}(a|b|m)') +-- !query schema +struct +-- !query output +14m + + +-- !query +SELECT regexp_substr('1a 2b 14m', '') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT regexp_substr('Spark', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT regexp_substr(null, '.*') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT regexp_instr('abc', 'b') +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT regexp_instr('abc', 'x') +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT regexp_instr('ABC', '(?-i)b') +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT regexp_instr('1a 2b 14m', '\\d{2}(a|b|m)') +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT regexp_instr('abc', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT regexp_instr(null, 'b') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT regexp_instr('abc', col0, 1) FROM VALUES(') ?') AS t(col0) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.PATTERN", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`regexp_instr`", + "parameter" : "`regexp`", + "value" : "') ?'" + } +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/selectExcept.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/selectExcept.sql.out new file mode 100644 index 000000000000..b01d1a04e59f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/selectExcept.sql.out @@ -0,0 +1,496 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES + (10, "name1", named_struct("f1", 1, "s2", named_struct("f2", 101, "f3", "a"))), + (20, "name2", named_struct("f1", 2, "s2", named_struct("f2", 202, "f3", "b"))), + (30, "name3", named_struct("f1", 3, "s2", named_struct("f2", 303, "f3", "c"))), + (40, "name4", named_struct("f1", 4, "s2", named_struct("f2", 404, "f3", "d"))), + (50, "name5", named_struct("f1", 5, "s2", named_struct("f2", 505, "f3", "e"))), + (60, "name6", named_struct("f1", 6, "s2", named_struct("f2", 606, "f3", "f"))), + (70, "name7", named_struct("f1", 7, "s2", named_struct("f2", 707, "f3", "g"))) +AS tbl_view(id, name, data) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE ids (id INT) USING CSV +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM tbl_view +-- !query schema +struct>> +-- !query output +10 name1 {"f1":1,"s2":{"f2":101,"f3":"a"}} +20 name2 {"f1":2,"s2":{"f2":202,"f3":"b"}} +30 name3 {"f1":3,"s2":{"f2":303,"f3":"c"}} +40 name4 {"f1":4,"s2":{"f2":404,"f3":"d"}} +50 name5 {"f1":5,"s2":{"f2":505,"f3":"e"}} +60 name6 {"f1":6,"s2":{"f2":606,"f3":"f"}} +70 name7 {"f1":7,"s2":{"f2":707,"f3":"g"}} + + +-- !query +SELECT * EXCEPT (id) FROM tbl_view +-- !query schema +struct>> +-- !query output +name1 {"f1":1,"s2":{"f2":101,"f3":"a"}} +name2 {"f1":2,"s2":{"f2":202,"f3":"b"}} +name3 {"f1":3,"s2":{"f2":303,"f3":"c"}} +name4 {"f1":4,"s2":{"f2":404,"f3":"d"}} +name5 {"f1":5,"s2":{"f2":505,"f3":"e"}} +name6 {"f1":6,"s2":{"f2":606,"f3":"f"}} +name7 {"f1":7,"s2":{"f2":707,"f3":"g"}} + + +-- !query +SELECT * EXCEPT (name) FROM tbl_view +-- !query schema +struct>> +-- !query output +10 {"f1":1,"s2":{"f2":101,"f3":"a"}} +20 {"f1":2,"s2":{"f2":202,"f3":"b"}} +30 {"f1":3,"s2":{"f2":303,"f3":"c"}} +40 {"f1":4,"s2":{"f2":404,"f3":"d"}} +50 {"f1":5,"s2":{"f2":505,"f3":"e"}} +60 {"f1":6,"s2":{"f2":606,"f3":"f"}} +70 {"f1":7,"s2":{"f2":707,"f3":"g"}} + + +-- !query +SELECT * EXCEPT (data) FROM tbl_view +-- !query schema +struct +-- !query output +10 name1 +20 name2 +30 name3 +40 name4 +50 name5 +60 name6 +70 name7 + + +-- !query +SELECT * EXCEPT (data.f1) FROM tbl_view +-- !query schema +struct>> +-- !query output +10 name1 {"s2":{"f2":101,"f3":"a"}} +20 name2 {"s2":{"f2":202,"f3":"b"}} +30 name3 {"s2":{"f2":303,"f3":"c"}} +40 name4 {"s2":{"f2":404,"f3":"d"}} +50 name5 {"s2":{"f2":505,"f3":"e"}} +60 name6 {"s2":{"f2":606,"f3":"f"}} +70 name7 {"s2":{"f2":707,"f3":"g"}} + + +-- !query +SELECT * EXCEPT (data.s2) FROM tbl_view +-- !query schema +struct> +-- !query output +10 name1 {"f1":1} +20 name2 {"f1":2} +30 name3 {"f1":3} +40 name4 {"f1":4} +50 name5 {"f1":5} +60 name6 {"f1":6} +70 name7 {"f1":7} + + +-- !query +SELECT * EXCEPT (data.s2.f2) FROM tbl_view +-- !query schema +struct>> +-- !query output +10 name1 {"f1":1,"s2":{"f3":"a"}} +20 name2 {"f1":2,"s2":{"f3":"b"}} +30 name3 {"f1":3,"s2":{"f3":"c"}} +40 name4 {"f1":4,"s2":{"f3":"d"}} +50 name5 {"f1":5,"s2":{"f3":"e"}} +60 name6 {"f1":6,"s2":{"f3":"f"}} +70 name7 {"f1":7,"s2":{"f3":"g"}} + + +-- !query +SELECT * EXCEPT (data.f1, data.s2) FROM tbl_view +-- !query schema +struct> +-- !query output +10 name1 {} +20 name2 {} +30 name3 {} +40 name4 {} +50 name5 {} +60 name6 {} +70 name7 {} + + +-- !query +SELECT * EXCEPT (id, name, data) FROM tbl_view +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * EXCEPT (`a-b-c`) FROM (SELECT 1 a_b_c, 2 `a-b-c`) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT tbl_view.* EXCEPT (name) FROM tbl_view +-- !query schema +struct>> +-- !query output +10 {"f1":1,"s2":{"f2":101,"f3":"a"}} +20 {"f1":2,"s2":{"f2":202,"f3":"b"}} +30 {"f1":3,"s2":{"f2":303,"f3":"c"}} +40 {"f1":4,"s2":{"f2":404,"f3":"d"}} +50 {"f1":5,"s2":{"f2":505,"f3":"e"}} +60 {"f1":6,"s2":{"f2":606,"f3":"f"}} +70 {"f1":7,"s2":{"f2":707,"f3":"g"}} + + +-- !query +INSERT INTO ids +SELECT * EXCEPT (name, data) FROM tbl_view +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM ids +-- !query schema +struct +-- !query output +10 +20 +30 +40 +50 +60 +70 + + +-- !query +SELECT * EXCEPT (ids.id) FROM ids +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT data.* EXCEPT (s2) FROM tbl_view +-- !query schema +struct +-- !query output +1 +2 +3 +4 +5 +6 +7 + + +-- !query +SELECT data.* EXCEPT (s2.f2) FROM tbl_view +-- !query schema +struct> +-- !query output +1 {"f3":"a"} +2 {"f3":"b"} +3 {"f3":"c"} +4 {"f3":"d"} +5 {"f3":"e"} +6 {"f3":"f"} +7 {"f3":"g"} + + +-- !query +SELECT data.s2.* EXCEPT (f2) FROM tbl_view +-- !query schema +struct +-- !query output +a +b +c +d +e +f +g + + +-- !query +SELECT * EXCEPT name FROM tbl_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'name'", + "hint" : "" + } +} + + +-- !query +SELECT * EXCEPT() name FROM tbl_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "')'", + "hint" : "" + } +} + + +-- !query +SELECT * EXCEPT(invalid_column) FROM tbl_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`invalid_column`", + "proposal" : "`id`, `name`, `data`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "* EXCEPT(invalid_column)" + } ] +} + + +-- !query +SELECT * EXCEPT(id, invalid_column) FROM tbl_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`invalid_column`", + "proposal" : "`id`, `name`, `data`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "* EXCEPT(id, invalid_column)" + } ] +} + + +-- !query +SELECT * EXCEPT(id, id) FROM tbl_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "EXCEPT_OVERLAPPING_COLUMNS", + "sqlState" : "42702", + "messageParameters" : { + "columns" : "id, id" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "* EXCEPT(id, id)" + } ] +} + + +-- !query +SELECT * EXCEPT(data.s2, data.s2.f2) FROM tbl_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "EXCEPT_OVERLAPPING_COLUMNS", + "sqlState" : "42702", + "messageParameters" : { + "columns" : "data.s2, data.s2.f2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "* EXCEPT(data.s2, data.s2.f2)" + } ] +} + + +-- !query +DROP VIEW tbl_view +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v1 AS VALUES (1, 2, NULL, 4, 5) AS T(c1, c2, c3, c4, c5) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT coalesce(*) FROM v1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT coalesce(* EXCEPT(c1, c2)) FROM v1 +-- !query schema +struct +-- !query output +4 + + +-- !query +SELECT array(*) FROM v1 +-- !query schema +struct> +-- !query output +[1,2,null,4,5] + + +-- !query +SELECT array(v1.*) FROM v1 +-- !query schema +struct> +-- !query output +[1,2,null,4,5] + + +-- !query +SELECT concat_ws(',', *) FROM v1 +-- !query schema +struct +-- !query output +1,2,4,5 + + +-- !query +SELECT (*) FROM v1 +-- !query schema +struct +-- !query output +1 2 NULL 4 5 + + +-- !query +SELECT struct(*) FROM v1 +-- !query schema +struct> +-- !query output +{"c1":1,"c2":2,"c3":null,"c4":4,"c5":5} + + +-- !query +SELECT greatest(*) FROM v1 +-- !query schema +struct +-- !query output +5 + + +-- !query +SELECT 5 IN (*) FROM v1 +-- !query schema +struct<(5 IN (c1, c2, c3, c4, c5)):boolean> +-- !query output +true + + +-- !query +SELECT c1.* FROM VALUES(named_struct('a', 1, 'b', 2), 10, 20) as t(c1, c2, c3) +-- !query schema +struct +-- !query output +1 2 + + +-- !query +SELECT 1 FROM v1 WHERE coalesce(*) = 1 +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT 1 FROM v1 WHERE array(*) = array(1, 2, NULL, 4, 5) +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT 1 FROM v1 WHERE 4 IN (*) +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT T.* FROM v1, LATERAL (SELECT v1.*) AS T(c1, c2, c3, c4, c5) +-- !query schema +struct +-- !query output +1 2 NULL 4 5 + + +-- !query +SELECT T.* FROM v1, LATERAL (SELECT COALESCE(v1.*)) AS T(x) +-- !query schema +struct +-- !query output +1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/show-create-table.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/show-create-table.sql.out new file mode 100644 index 000000000000..c5e05411b6db --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/show-create-table.sql.out @@ -0,0 +1,451 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW CREATE TABLE tbl +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.default.tbl ( + a INT, + b STRING, + c INT) +USING parquet + + +-- !query +DROP TABLE tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +OPTIONS ('a' 1, 'password' = 'password') +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW CREATE TABLE tbl +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.default.tbl ( + a INT, + b STRING, + c INT) +USING parquet +OPTIONS ( + 'a' = '1', + 'password' = '*********(redacted)') + + +-- !query +DROP TABLE tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +OPTIONS ('path' '/path/to/table') +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW CREATE TABLE tbl +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.default.tbl ( + a INT, + b STRING, + c INT) +USING parquet +LOCATION 'file:/path/to/table' + + +-- !query +DROP TABLE tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +LOCATION '/path/to/table' +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW CREATE TABLE tbl +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.default.tbl ( + a INT, + b STRING, + c INT) +USING parquet +LOCATION 'file:/path/to/table' + + +-- !query +DROP TABLE tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +PARTITIONED BY (a) +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW CREATE TABLE tbl +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.default.tbl ( + b STRING, + c INT, + a INT) +USING parquet +PARTITIONED BY (a) + + +-- !query +DROP TABLE tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +CLUSTERED BY (a) SORTED BY (b ASC) INTO 2 BUCKETS +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW CREATE TABLE tbl +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.default.tbl ( + a INT, + b STRING, + c INT) +USING parquet +CLUSTERED BY (a) +SORTED BY (b) +INTO 2 BUCKETS + + +-- !query +DROP TABLE tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tbl (a INT DEFAULT 42, b STRING DEFAULT 'abc, def', c INT DEFAULT 42) USING parquet +COMMENT 'This is a comment' +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW CREATE TABLE tbl +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.default.tbl ( + a INT DEFAULT 42, + b STRING DEFAULT 'abc, def', + c INT DEFAULT 42) +USING parquet +COMMENT 'This is a comment' + + +-- !query +DROP TABLE tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +COMMENT 'This is a comment' +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW CREATE TABLE tbl +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.default.tbl ( + a INT, + b STRING, + c INT) +USING parquet +COMMENT 'This is a comment' + + +-- !query +DROP TABLE tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +TBLPROPERTIES ('a' = '1', 'password' = 'password') +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW CREATE TABLE tbl +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.default.tbl ( + a INT, + b STRING, + c INT) +USING parquet +TBLPROPERTIES ( + 'a' = '1', + 'password' = '*********(redacted)') + + +-- !query +DROP TABLE tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tbl (a REAL, b NUMERIC, c NUMERIC(10), d NUMERIC(10,1)) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW CREATE TABLE tbl +-- !query schema +struct +-- !query output +CREATE TABLE spark_catalog.default.tbl ( + a FLOAT, + b DECIMAL(10,0), + c DECIMAL(10,0), + d DECIMAL(10,1)) +USING parquet + + +-- !query +DROP TABLE tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW view_SPARK_30302 (aaa, bbb) +AS SELECT a, b FROM tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW CREATE TABLE view_SPARK_30302 AS SERDE +-- !query schema +struct +-- !query output +CREATE VIEW default.view_spark_30302 ( + aaa, + bbb) +WITH SCHEMA COMPENSATION +AS SELECT a, b FROM tbl + + +-- !query +SHOW CREATE TABLE view_SPARK_30302 +-- !query schema +struct +-- !query output +CREATE VIEW default.view_spark_30302 ( + aaa, + bbb) +WITH SCHEMA COMPENSATION +AS SELECT a, b FROM tbl + + +-- !query +DROP VIEW view_SPARK_30302 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW view_SPARK_30302 (aaa COMMENT 'comment with \'quoted text\' for aaa', bbb) +COMMENT 'This is a comment with \'quoted text\' for view' +AS SELECT a, b FROM tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW CREATE TABLE view_SPARK_30302 AS SERDE +-- !query schema +struct +-- !query output +CREATE VIEW default.view_spark_30302 ( + aaa COMMENT 'comment with \'quoted text\' for aaa', + bbb) +COMMENT 'This is a comment with \'quoted text\' for view' +WITH SCHEMA COMPENSATION +AS SELECT a, b FROM tbl + + +-- !query +SHOW CREATE TABLE view_SPARK_30302 +-- !query schema +struct +-- !query output +CREATE VIEW default.view_spark_30302 ( + aaa COMMENT 'comment with \'quoted text\' for aaa', + bbb) +COMMENT 'This is a comment with \'quoted text\' for view' +WITH SCHEMA COMPENSATION +AS SELECT a, b FROM tbl + + +-- !query +DROP VIEW view_SPARK_30302 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW view_SPARK_30302 (aaa, bbb) +TBLPROPERTIES ('a' = '1', 'b' = '2') +AS SELECT a, b FROM tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW CREATE TABLE view_SPARK_30302 AS SERDE +-- !query schema +struct +-- !query output +CREATE VIEW default.view_spark_30302 ( + aaa, + bbb) +TBLPROPERTIES ( + 'a' = '1', + 'b' = '2') +WITH SCHEMA COMPENSATION +AS SELECT a, b FROM tbl + + +-- !query +SHOW CREATE TABLE view_SPARK_30302 +-- !query schema +struct +-- !query output +CREATE VIEW default.view_spark_30302 ( + aaa, + bbb) +TBLPROPERTIES ( + 'a' = '1', + 'b' = '2') +WITH SCHEMA COMPENSATION +AS SELECT a, b FROM tbl + + +-- !query +DROP VIEW view_SPARK_30302 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE tbl +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/show-tables.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/show-tables.sql.out new file mode 100644 index 000000000000..93c144477759 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/show-tables.sql.out @@ -0,0 +1,322 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE DATABASE showdb +-- !query schema +struct<> +-- !query output + + + +-- !query +USE showdb +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE show_t1(a String, b Int, c String, d String) USING parquet PARTITIONED BY (c, d) +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER TABLE show_t1 ADD PARTITION (c='Us', d=1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE show_t2(b String, d Int) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW show_t3(e int) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE GLOBAL TEMP VIEW show_t4 AS SELECT 1 as col1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW TABLES +-- !query schema +struct +-- !query output +show_t1 +show_t2 +show_t3 + + +-- !query +SHOW TABLES IN showdb +-- !query schema +struct +-- !query output +show_t1 +show_t2 +show_t3 + + +-- !query +SHOW TABLES 'show_t*' +-- !query schema +struct +-- !query output +show_t1 +show_t2 +show_t3 + + +-- !query +SHOW TABLES LIKE 'show_t1*|show_t2*' +-- !query schema +struct +-- !query output +show_t1 +show_t2 + + +-- !query +SHOW TABLES IN showdb 'show_t*' +-- !query schema +struct +-- !query output +show_t1 +show_t2 +show_t3 + + +-- !query +SHOW TABLES IN showdb LIKE 'show_t*' +-- !query schema +struct +-- !query output +show_t1 +show_t2 +show_t3 + + +-- !query +SHOW TABLE EXTENDED LIKE 'show_t*' +-- !query schema +struct +-- !query output + show_t3 true Table: show_t3 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type: VIEW +View Schema Mode: BINDING +Schema: root + |-- e: integer (nullable = true) + + +showdb show_t1 false Catalog: spark_catalog +Database: showdb +Table: show_t1 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type: MANAGED +Provider: parquet +Location [not included in comparison]/{warehouse_dir}/showdb.db/show_t1 +Partition Provider: Catalog +Partition Columns: [`c`, `d`] +Schema: root + |-- a: string (nullable = true) + |-- b: integer (nullable = true) + |-- c: string (nullable = true) + |-- d: string (nullable = true) + + +showdb show_t2 false Catalog: spark_catalog +Database: showdb +Table: show_t2 +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type: MANAGED +Provider: parquet +Location [not included in comparison]/{warehouse_dir}/showdb.db/show_t2 +Schema: root + |-- b: string (nullable = true) + |-- d: integer (nullable = true) + + +-- !query +SHOW TABLE EXTENDED +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "end of input", + "hint" : "" + } +} + + +-- !query +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us', d=1) +-- !query schema +struct +-- !query output +showdb show_t1 false Partition Values: [c=Us, d=1] +Location [not included in comparison]/{warehouse_dir}/showdb.db/show_t1/c=Us/d=1 +Created Time [not included in comparison] +Last Access [not included in comparison] + + +-- !query +SHOW TABLE EXTENDED PARTITION(c='Us', d=1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'PARTITION'", + "hint" : "" + } +} + + +-- !query +SHOW TABLE EXTENDED LIKE 'show_t*' PARTITION(c='Us', d=1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`show_t*`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 34, + "fragment" : "'show_t*'" + } ] +} + + +-- !query +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1232", + "messageParameters" : { + "partitionColumnNames" : "c, d", + "specKeys" : "c", + "tableName" : "`spark_catalog`.`showdb`.`show_t1`" + } +} + + +-- !query +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(a='Us', d=1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PARTITIONS_NOT_FOUND", + "sqlState" : "428FT", + "messageParameters" : { + "partitionList" : "`a`", + "tableName" : "`spark_catalog`.`showdb`.`show_t1`" + } +} + + +-- !query +SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Ch', d=1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException +{ + "errorClass" : "PARTITIONS_NOT_FOUND", + "sqlState" : "428FT", + "messageParameters" : { + "partitionList" : "PARTITION (`c` = Ch, `d` = 1)", + "tableName" : "`showdb`.`show_t1`" + } +} + + +-- !query +DROP TABLE show_t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE show_t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW show_t3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW global_temp.show_t4 +-- !query schema +struct<> +-- !query output + + + +-- !query +USE default +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP DATABASE showdb +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/show-tblproperties.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/show-tblproperties.sql.out new file mode 100644 index 000000000000..d058d48fd215 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/show-tblproperties.sql.out @@ -0,0 +1,107 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +TBLPROPERTIES('p1'='v1', 'p2'='v2', password = 'password') +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW TBLPROPERTIES tbl +-- !query schema +struct +-- !query output +p1 v1 +p2 v2 +password *********(redacted) + + +-- !query +SHOW TBLPROPERTIES tbl("p1") +-- !query schema +struct +-- !query output +p1 v1 + + +-- !query +SHOW TBLPROPERTIES tbl("p3") +-- !query schema +struct +-- !query output +p3 Table spark_catalog.default.tbl does not have property: p3 + + +-- !query +DROP TABLE tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW view TBLPROPERTIES('p1'='v1', 'p2'='v2') AS SELECT 1 AS c1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW TBLPROPERTIES view +-- !query schema +struct +-- !query output +p1 v1 +p2 v2 + + +-- !query +SHOW TBLPROPERTIES view("p1") +-- !query schema +struct +-- !query output +p1 v1 + + +-- !query +SHOW TBLPROPERTIES view("p3") +-- !query schema +struct +-- !query output +p3 Table spark_catalog.default.view does not have property: p3 + + +-- !query +DROP VIEW view +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW tv AS SELECT 1 AS c1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW TBLPROPERTIES tv +-- !query schema +struct +-- !query output + + + +-- !query +DROP VIEW tv +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/show-views.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/show-views.sql.out new file mode 100644 index 000000000000..249f5a1d9552 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/show-views.sql.out @@ -0,0 +1,180 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE DATABASE showdb +-- !query schema +struct<> +-- !query output + + + +-- !query +USE showdb +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tbl(a STRING, b INT, c STRING, d STRING) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW view_1 AS SELECT * FROM tbl +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW view_2 AS SELECT * FROM tbl WHERE c='a' +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE GLOBAL TEMP VIEW view_3 AS SELECT 1 as col1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW view_4(e INT) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW VIEWS +-- !query schema +struct +-- !query output +view_1 +view_2 +view_4 + + +-- !query +SHOW VIEWS FROM showdb +-- !query schema +struct +-- !query output +view_1 +view_2 +view_4 + + +-- !query +SHOW VIEWS IN showdb +-- !query schema +struct +-- !query output +view_1 +view_2 +view_4 + + +-- !query +SHOW VIEWS IN global_temp +-- !query schema +struct +-- !query output +view_3 +view_4 + + +-- !query +SHOW VIEWS 'view_*' +-- !query schema +struct +-- !query output +view_1 +view_2 +view_4 + + +-- !query +SHOW VIEWS LIKE 'view_1*|view_2*' +-- !query schema +struct +-- !query output +view_1 +view_2 + + +-- !query +SHOW VIEWS IN showdb 'view_*' +-- !query schema +struct +-- !query output +view_1 +view_2 +view_4 + + +-- !query +SHOW VIEWS IN showdb LIKE 'view_*' +-- !query schema +struct +-- !query output +view_1 +view_2 +view_4 + + +-- !query +SHOW VIEWS IN wrongdb LIKE 'view_*' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException +{ + "errorClass" : "SCHEMA_NOT_FOUND", + "sqlState" : "42704", + "messageParameters" : { + "schemaName" : "`spark_catalog`.`wrongdb`" + } +} + + +-- !query +DROP VIEW global_temp.view_3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW view_4 +-- !query schema +struct<> +-- !query output + + + +-- !query +USE default +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP DATABASE showdb CASCADE +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/show_columns.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/show_columns.sql.out new file mode 100644 index 000000000000..bb4e7e08c6f5 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/show_columns.sql.out @@ -0,0 +1,273 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE DATABASE showdb +-- !query schema +struct<> +-- !query output + + + +-- !query +USE showdb +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE showcolumn1 (col1 int, `col 2` int) USING json +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE showcolumn2 (price int, qty int, year int, month int) USING parquet partitioned by (year, month) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW showColumn3 (col3 int, `col 4` int) USING json +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE GLOBAL TEMP VIEW showColumn4 AS SELECT 1 as col1, 'abc' as `col 5` +-- !query schema +struct<> +-- !query output + + + +-- !query +SHOW COLUMNS IN showcolumn1 +-- !query schema +struct +-- !query output +col 2 +col1 + + +-- !query +SHOW COLUMNS IN showdb.showcolumn1 +-- !query schema +struct +-- !query output +col 2 +col1 + + +-- !query +SHOW COLUMNS IN showcolumn1 FROM showdb +-- !query schema +struct +-- !query output +col 2 +col1 + + +-- !query +SHOW COLUMNS IN showcolumn2 IN showdb +-- !query schema +struct +-- !query output +month +price +qty +year + + +-- !query +SHOW COLUMNS IN badtable FROM showdb +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`showdb`.`badtable`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 24, + "fragment" : "badtable" + } ] +} + + +-- !query +SHOW COLUMNS IN showdb.showcolumn1 from SHOWDB +-- !query schema +struct +-- !query output +col 2 +col1 + + +-- !query +SHOW COLUMNS IN showdb.showcolumn1 FROM baddb +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "SHOW_COLUMNS_WITH_CONFLICT_NAMESPACE", + "sqlState" : "42K05", + "messageParameters" : { + "namespaceA" : "`baddb`", + "namespaceB" : "`showdb`" + } +} + + +-- !query +SHOW COLUMNS IN showcolumn3 +-- !query schema +struct +-- !query output +col 4 +col3 + + +-- !query +SHOW COLUMNS IN showdb.showcolumn3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`showdb`.`showcolumn3`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 34, + "fragment" : "showdb.showcolumn3" + } ] +} + + +-- !query +SHOW COLUMNS IN showcolumn3 FROM showdb +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`showdb`.`showcolumn3`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 27, + "fragment" : "showcolumn3" + } ] +} + + +-- !query +SHOW COLUMNS IN showcolumn4 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "TABLE_OR_VIEW_NOT_FOUND", + "sqlState" : "42P01", + "messageParameters" : { + "relationName" : "`showcolumn4`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 27, + "fragment" : "showcolumn4" + } ] +} + + +-- !query +SHOW COLUMNS IN global_temp.showcolumn4 +-- !query schema +struct +-- !query output +col 5 +col1 + + +-- !query +SHOW COLUMNS IN showcolumn4 FROM global_temp +-- !query schema +struct +-- !query output +col 5 +col1 + + +-- !query +DROP TABLE showcolumn1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE showColumn2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW showcolumn3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW global_temp.showcolumn4 +-- !query schema +struct<> +-- !query output + + + +-- !query +use default +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP DATABASE showdb +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/sql-compatibility-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/sql-compatibility-functions.sql.out new file mode 100644 index 000000000000..0dd8c738d212 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/sql-compatibility-functions.sql.out @@ -0,0 +1,136 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT ifnull(null, 'x'), ifnull('y', 'x'), ifnull(null, null) +-- !query schema +struct +-- !query output +x y NULL + + +-- !query +SELECT nullif('x', 'x'), nullif('x', 'y') +-- !query schema +struct +-- !query output +NULL x + + +-- !query +SELECT nvl(null, 'x'), nvl('y', 'x'), nvl(null, null) +-- !query schema +struct +-- !query output +x y NULL + + +-- !query +SELECT nvl2(null, 'x', 'y'), nvl2('n', 'x', 'y'), nvl2(null, null, null) +-- !query schema +struct +-- !query output +y x NULL + + +-- !query +SELECT ifnull(1, 2.1d), ifnull(null, 2.1d) +-- !query schema +struct +-- !query output +1.0 2.1 + + +-- !query +SELECT nullif(1, 2.1d), nullif(1, 1.0d) +-- !query schema +struct +-- !query output +1 NULL + + +-- !query +SELECT nvl(1, 2.1d), nvl(null, 2.1d) +-- !query schema +struct +-- !query output +1.0 2.1 + + +-- !query +SELECT nvl2(null, 1, 2.1d), nvl2('n', 1, 2.1d) +-- !query schema +struct +-- !query output +2.1 1.0 + + +-- !query +SELECT boolean(1), tinyint(1), smallint(1), int(1), bigint(1) +-- !query schema +struct<1:boolean,1:tinyint,1:smallint,1:int,1:bigint> +-- !query output +true 1 1 1 1 + + +-- !query +SELECT float(1), double(1), decimal(1) +-- !query schema +struct<1:float,1:double,1:decimal(10,0)> +-- !query output +1.0 1.0 1 + + +-- !query +SELECT date("2014-04-04"), timestamp(date("2014-04-04")) +-- !query schema +struct<2014-04-04:date,2014-04-04:timestamp> +-- !query output +2014-04-04 2014-04-04 00:00:00 + + +-- !query +SELECT string(1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`string`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "string(1, 2)" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW tempView1 AS VALUES (1, NAMED_STRUCT('col1', 'gamma', 'col2', 'delta')) AS T(id, st) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT nvl(st.col1, "value"), count(*) FROM from tempView1 GROUP BY nvl(st.col1, "value") +-- !query schema +struct +-- !query output +gamma 1 + + +-- !query +SELECT nullif(SUM(id), 0) from range(5) +-- !query schema +struct +-- !query output +10 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/sql-on-files.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/sql-on-files.sql.out new file mode 100644 index 000000000000..71d4216ea207 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/sql-on-files.sql.out @@ -0,0 +1,283 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE DATABASE IF NOT EXISTS sql_on_files +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE sql_on_files.test_parquet USING PARQUET AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM parquet.`` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EMPTY_LOCATION", + "sqlState" : "42K05", + "messageParameters" : { + "location" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 24, + "fragment" : "parquet.``" + } ] +} + + +-- !query +SELECT * FROM parquet.`/file/not/found` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PATH_NOT_FOUND", + "sqlState" : "42K03", + "messageParameters" : { + "path" : "file:/file/not/found" + } +} + + +-- !query +SELECT * FROM parquet.`${spark.sql.warehouse.dir}/sql_on_files.db/test_parquet` +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +DROP TABLE sql_on_files.test_parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE sql_on_files.test_orc USING ORC AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM orc.`` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EMPTY_LOCATION", + "sqlState" : "42K05", + "messageParameters" : { + "location" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 20, + "fragment" : "orc.``" + } ] +} + + +-- !query +SELECT * FROM orc.`/file/not/found` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PATH_NOT_FOUND", + "sqlState" : "42K03", + "messageParameters" : { + "path" : "file:/file/not/found" + } +} + + +-- !query +SELECT * FROM orc.`${spark.sql.warehouse.dir}/sql_on_files.db/test_orc` +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +DROP TABLE sql_on_files.test_orc +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE sql_on_files.test_csv USING CSV AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM csv.`` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EMPTY_LOCATION", + "sqlState" : "42K05", + "messageParameters" : { + "location" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 20, + "fragment" : "csv.``" + } ] +} + + +-- !query +SELECT * FROM csv.`/file/not/found` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PATH_NOT_FOUND", + "sqlState" : "42K03", + "messageParameters" : { + "path" : "file:/file/not/found" + } +} + + +-- !query +SELECT * FROM csv.`${spark.sql.warehouse.dir}/sql_on_files.db/test_csv` +-- !query schema +struct<_c0:string> +-- !query output +1 + + +-- !query +DROP TABLE sql_on_files.test_csv +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE sql_on_files.test_json USING JSON AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM json.`` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EMPTY_LOCATION", + "sqlState" : "42K05", + "messageParameters" : { + "location" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 21, + "fragment" : "json.``" + } ] +} + + +-- !query +SELECT * FROM json.`/file/not/found` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PATH_NOT_FOUND", + "sqlState" : "42K03", + "messageParameters" : { + "path" : "file:/file/not/found" + } +} + + +-- !query +SELECT * FROM json.`${spark.sql.warehouse.dir}/sql_on_files.db/test_json` +-- !query schema +struct<1:bigint> +-- !query output +1 + + +-- !query +DROP TABLE sql_on_files.test_json +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP DATABASE sql_on_files +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM json.`https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/src/main/resources/employees.json` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "FAILED_READ_FILE.UNSUPPORTED_FILE_SYSTEM", + "sqlState" : "KD001", + "messageParameters" : { + "fileSystemClass" : "org.apache.hadoop.fs.http.HttpsFileSystem", + "method" : "listStatus", + "path" : "https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/src/main/resources/employees.json" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 128, + "fragment" : "json.`https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/src/main/resources/employees.json`" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/sql-session-variables.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/sql-session-variables.sql.out new file mode 100644 index 000000000000..73d3ec737085 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/sql-session-variables.sql.out @@ -0,0 +1,2337 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SET spark.sql.ansi.enabled = true +-- !query schema +struct +-- !query output +spark.sql.ansi.enabled true + + +-- !query +DECLARE title STRING +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE title = '-- Basic sanity --' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE var1 INT = 5 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 +-- !query schema +struct +-- !query output +5 + + +-- !query +SET VARIABLE var1 = 6 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 +-- !query schema +struct +-- !query output +6 + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE title = 'Create Variable - Success Cases' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE VARIABLE var1 INT +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 'Expect: INT, NULL', typeof(var1), var1 +-- !query schema +struct +-- !query output +Expect: INT, NULL int NULL + + +-- !query +DECLARE OR REPLACE VARIABLE var1 DOUBLE +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 'Expect: DOUBLE, NULL', typeof(var1), var1 +-- !query schema +struct +-- !query output +Expect: DOUBLE, NULL double NULL + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var1 TIMESTAMP +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 'Expect: TIMESTAMP, NULL', typeof(var1), var1 +-- !query schema +struct +-- !query output +Expect: TIMESTAMP, NULL timestamp NULL + + +-- !query +SET VARIABLE title = 'Create Variable - Failure Cases' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE VARIABLE IF NOT EXISTS var1 INT +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'EXISTS'", + "hint" : "" + } +} + + +-- !query +DROP TEMPORARY VARIABLE IF EXISTS var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE title = 'Drop Variable' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE VAR var1 INT +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 +-- !query schema +struct +-- !query output +NULL + + +-- !query +DROP TEMPORARY VAR var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`var1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 11, + "fragment" : "var1" + } ] +} + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "VARIABLE_NOT_FOUND", + "sqlState" : "42883", + "messageParameters" : { + "variableName" : "`system`.`session`.`var1`" + } +} + + +-- !query +DROP TEMPORARY VARIABLE IF EXISTS var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE VARIABLE var1 INT +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VARIABLE var1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'VARIABLE'", + "hint" : "" + } +} + + +-- !query +DROP VARIABLE system.session.var1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'VARIABLE'", + "hint" : "" + } +} + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE title = 'Test qualifiers - success' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE VARIABLE var1 INT DEFAULT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 1 as Expected, var1 as Unqualified, session.var1 AS SchemaQualified, system.session.var1 AS fullyQualified +-- !query schema +struct +-- !query output +1 1 1 1 + + +-- !query +SET VARIABLE var1 = 2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 2 as Expected, var1 as Unqualified, session.var1 AS SchemaQualified, system.session.var1 AS fullyQualified +-- !query schema +struct +-- !query output +2 2 2 2 + + +-- !query +DECLARE OR REPLACE VARIABLE session.var1 INT DEFAULT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 1 as Expected, var1 as Unqualified, session.var1 AS SchemaQualified, system.session.var1 AS fullyQualified +-- !query schema +struct +-- !query output +1 1 1 1 + + +-- !query +SET VARIABLE session.var1 = 2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 2 as Expected, var1 as Unqualified, session.var1 AS SchemaQualified, system.session.var1 AS fullyQualified +-- !query schema +struct +-- !query output +2 2 2 2 + + +-- !query +DECLARE OR REPLACE VARIABLE system.session.var1 INT DEFAULT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 1 as Expected, var1 as Unqualified, session.var1 AS SchemaQualified, system.session.var1 AS fullyQualified +-- !query schema +struct +-- !query output +1 1 1 1 + + +-- !query +SET VARIABLE system.session.var1 = 2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 2 as Expected, var1 as Unqualified, session.var1 AS SchemaQualified, system.session.var1 AS fullyQualified +-- !query schema +struct +-- !query output +2 2 2 2 + + +-- !query +DECLARE OR REPLACE VARIABLE sySteM.sEssIon.vAr1 INT DEFAULT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 1 as Expected, var1 as Unqualified, sessIon.Var1 AS SchemaQualified, System.sessiOn.var1 AS fullyQualified +-- !query schema +struct +-- !query output +1 1 1 1 + + +-- !query +SET VARIABLE sYstem.sesSiOn.vaR1 = 2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 2 as Expected, VAR1 as Unqualified, SESSION.VAR1 AS SchemaQualified, SYSTEM.SESSION.VAR1 AS fullyQualified +-- !query schema +struct +-- !query output +2 2 2 2 + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "VARIABLE_NOT_FOUND", + "sqlState" : "42883", + "messageParameters" : { + "variableName" : "`system`.`session`.`var1`" + } +} + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY VARIABLE session.var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "VARIABLE_NOT_FOUND", + "sqlState" : "42883", + "messageParameters" : { + "variableName" : "`system`.`session`.`var1`" + } +} + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY VARIABLE system.session.var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "VARIABLE_NOT_FOUND", + "sqlState" : "42883", + "messageParameters" : { + "variableName" : "`system`.`session`.`var1`" + } +} + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY VARIABLE sysTem.sesSion.vAr1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "VARIABLE_NOT_FOUND", + "sqlState" : "42883", + "messageParameters" : { + "variableName" : "`system`.`session`.`var1`" + } +} + + +-- !query +SET VARIABLE title = 'Test variable in aggregate' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT (SELECT MAX(id) FROM RANGE(10) WHERE id < title) FROM VALUES 1, 2 AS t(title) +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +SET VARIABLE title = 'Test qualifiers - fail' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE builtin.var1 INT +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_VARIABLE", + "sqlState" : "42883", + "messageParameters" : { + "searchPath" : "`system`.`session`", + "variableName" : "`builtin`.`var1`" + } +} + + +-- !query +DECLARE OR REPLACE VARIABLE system.sesion.var1 INT +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_VARIABLE", + "sqlState" : "42883", + "messageParameters" : { + "searchPath" : "`system`.`session`", + "variableName" : "`system`.`sesion`.`var1`" + } +} + + +-- !query +DECLARE OR REPLACE VARIABLE sys.session.var1 INT +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_VARIABLE", + "sqlState" : "42883", + "messageParameters" : { + "searchPath" : "`system`.`session`", + "variableName" : "`sys`.`session`.`var1`" + } +} + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`var`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 10, + "fragment" : "var" + } ] +} + + +-- !query +SELECT ses.var1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`ses`.`var1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "ses.var1" + } ] +} + + +-- !query +SELECT b.sesson.var1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`b`.`sesson`.`var1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "b.sesson.var1" + } ] +} + + +-- !query +SELECT builtn.session.var1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`builtn`.`session`.`var1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "builtn.session.var1" + } ] +} + + +-- !query +SET VARIABLE ses.var1 = 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_VARIABLE", + "sqlState" : "42883", + "messageParameters" : { + "searchPath" : "`SYSTEM`.`SESSION`", + "variableName" : "`ses`.`var1`" + } +} + + +-- !query +SET VARIABLE builtn.session.var1 = 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_VARIABLE", + "sqlState" : "42883", + "messageParameters" : { + "searchPath" : "`SYSTEM`.`SESSION`", + "variableName" : "`builtn`.`session`.`var1`" + } +} + + +-- !query +SET VARIABLE title = 'Test DEFAULT on create - success' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 1 AS Expected, var1 AS result +-- !query schema +struct +-- !query output +1 1 + + +-- !query +DECLARE OR REPLACE VARIABLE var1 DOUBLE DEFAULT 1 + RAND(5) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT true AS Expected, var1 >= 1 AS result +-- !query schema +struct +-- !query output +true true + + +-- !query +DECLARE OR REPLACE VARIABLE var1 = 'Hello' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 'STRING, Hello' AS Expected, typeof(var1) AS type, var1 AS result +-- !query schema +struct +-- !query output +STRING, Hello string Hello + + +-- !query +DECLARE OR REPLACE VARIABLE var1 DEFAULT NULL +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 'VOID, NULL' AS Expected, typeof(var1) AS type, var1 AS result +-- !query schema +struct +-- !query output +VOID, NULL void NULL + + +-- !query +DECLARE OR REPLACE VARIABLE INT DEFAULT 5.0 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 'INT, 5' AS Expected, typeof(var1) AS type, var1 AS result +-- !query schema +struct +-- !query output +INT, 5 void NULL + + +-- !query +DECLARE OR REPLACE VARIABLE var1 MAP DEFAULT MAP('Hello', 5.1, 'World', -7.1E10) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 'MAP, [Hello -> 5.1, World -> -7E10]' AS Expected, typeof(var1) AS type, var1 AS result +-- !query schema +struct> +-- !query output +MAP, [Hello -> 5.1, World -> -7E10] map {"Hello":5.1,"World":-7.1E10} + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT NULL +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 'NULL' AS Expected, var1 AS result +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +DECLARE OR REPLACE VARIABLE var1 STRING DEFAULT CURRENT_DATABASE() +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 'true' AS Expected, length(var1) > 0 AS result +-- !query schema +struct +-- !query output +true true + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE var1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.VARIABLE_TYPE_OR_DEFAULT_REQUIRED", + "sqlState" : "42000", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 12, + "fragment" : "DECLARE var1" + } ] +} + + +-- !query +SET VARIABLE title = 'Test DEFAULT on create - failures' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT (SELECT c1 FROM VALUES(1) AS T(c1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_DEFAULT_VALUE.SUBQUERY_EXPRESSION", + "sqlState" : "42623", + "messageParameters" : { + "colName" : "`system`.`session`.`var1`", + "defaultValue" : "(SELECT c1 FROM VALUES(1) AS T(c1))", + "statement" : "DECLARE VARIABLE" + } +} + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 'hello' +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'hello'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 52, + "fragment" : "DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 'hello'" + } ] +} + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 1 / 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 46, + "stopIndex" : 50, + "fragment" : "1 / 0" + } ] +} + + +-- !query +DECLARE OR REPLACE VARIABLE var1 SMALLINT DEFAULT 100000 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"INT\"", + "targetType" : "\"SMALLINT\"", + "value" : "100000" + } +} + + +-- !query +SET VARIABLE title = 'SET VARIABLE - single target' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 5 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE var1 = 7 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 +-- !query schema +struct +-- !query output +7 + + +-- !query +SET VAR var1 = 8 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 +-- !query schema +struct +-- !query output +8 + + +-- !query +SET VARIABLE var1 = (SELECT c1 FROM VALUES(1) AS T(c1)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SET VARIABLE var1 = (SELECT c1 FROM VALUES(1) AS T(c1) WHERE 1=0) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 AS `null` +-- !query schema +struct +-- !query output +NULL + + +-- !query +SET VARIABLE var1 = (SELECT c1 FROM VALUES(1.0) AS T(c1)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SET VARIABLE var1 = (SELECT c1 FROM VALUES(1.0E10) AS T(c1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"DOUBLE\"", + "targetType" : "\"INT\"", + "value" : "1.0E10D" + } +} + + +-- !query +SELECT var1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SET VARIABLE var1 = (SELECT c1 FROM VALUES(1), (2) AS T(c1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +{ + "errorClass" : "SCALAR_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 21, + "stopIndex" : 60, + "fragment" : "(SELECT c1 FROM VALUES(1), (2) AS T(c1))" + } ] +} + + +-- !query +SET VARIABLE var1 = (SELECT c1, c1 FROM VALUES(1), (2) AS T(c1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_SUBQUERY_EXPRESSION.SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", + "sqlState" : "42823", + "messageParameters" : { + "number" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 21, + "stopIndex" : 64, + "fragment" : "(SELECT c1, c1 FROM VALUES(1), (2) AS T(c1))" + } ] +} + + +-- !query +SET VARIABLE var1 = (SELECT c1 FROM VALUES('hello') AS T(c1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'hello'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 61, + "fragment" : "SET VARIABLE var1 = (SELECT c1 FROM VALUES('hello') AS T(c1))" + } ] +} + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 5 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE var1 = var1 + 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 +-- !query schema +struct +-- !query output +6 + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE title = 'SET VARIABLE - comma separated target' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 5 +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var2 STRING DEFAULT 'hello' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var3 DOUBLE DEFAULT 2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE var1 = 6, var2 = 'world', var3 = pi() +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 AS `6`, var2 AS `world` , var3 as `3.14...` +-- !query schema +struct<6:int,world:string,3.14...:double> +-- !query output +6 world 3.141592653589793 + + +-- !query +SET VAR var1 = 7, var2 = 'universe', var3 = -1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 AS `7`, var2 AS `universe` , var3 as `-1` +-- !query schema +struct<7:int,universe:string,-1:double> +-- !query output +7 universe -1.0 + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 5 +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var2 STRING DEFAULT 'hello' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var3 DOUBLE DEFAULT 2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE var1 = var3, var2 = ascii(var1), var3 = var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 AS `2`, var2 AS `104`, var3 AS `5` +-- !query schema +struct<2:int,104:string,5:double> +-- !query output +2 53 5.0 + + +-- !query +SET VARIABLE var1 = var3, var2 = INTERVAL'5' HOUR, var3 = var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE var1 = 1, var2 = 0, vAr1 = 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DUPLICATE_ASSIGNMENTS", + "sqlState" : "42701", + "messageParameters" : { + "nameList" : "`var1`" + } +} + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY VARIABLE var2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY VARIABLE var3 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE title = 'SET VARIABLE - row assignment' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 5 +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var2 STRING DEFAULT 'hello' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var3 DOUBLE DEFAULT 2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE (var1) = (SELECT c1 FROM VALUES(1) AS T(c1)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SET VAR (var1) = (SELECT c1 FROM VALUES(2) AS T(c1)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 +-- !query schema +struct +-- !query output +2 + + +-- !query +SET VARIABLE (var1, var2) = (SELECT c1, c2 FROM VALUES(10, 11) AS T(c1, c2)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 AS `10`, var2 AS `11` +-- !query schema +struct<10:int,11:string> +-- !query output +10 11 + + +-- !query +SET VARIABLE (var1, var2, var3) = (SELECT c1, c2, c3 FROM VALUES(100, 110, 120) AS T(c1, c2, c3)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 AS `100`, var2 AS `110`, var3 AS `120` +-- !query schema +struct<100:int,110:string,120:double> +-- !query output +100 110 120.0 + + +-- !query +SET VARIABLE (var1, var2, var3) = (SELECT c1, c2, c3 FROM VALUES(100, 110, 120) AS T(c1, c2, c3) WHERE 1 = 0) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 AS `NULL`, var2 AS `NULL`, var3 AS `NULL` +-- !query schema +struct +-- !query output +NULL NULL NULL + + +-- !query +SET VARIABLE () = (SELECT 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SET_SYNTAX", + "sqlState" : "42000", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 28, + "fragment" : "SET VARIABLE () = (SELECT 1)" + } ] +} + + +-- !query +SET VARIABLE (var1, var2, var3) = (SELECT c1, c2, c3 FROM VALUES(100, 110, 120), (-100, -110, -120) AS T(c1, c2, c3)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +{ + "errorClass" : "ROW_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000" +} + + +-- !query +SET VARIABLE (var1, var2, var3) = (SELECT c1, c2 FROM VALUES(100, 110, 120) AS T(c1, c2, c3)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "ASSIGNMENT_ARITY_MISMATCH", + "sqlState" : "42802", + "messageParameters" : { + "numExpr" : "2", + "numTarget" : "3" + } +} + + +-- !query +SET VARIABLE (var1, var2, var3) = (SELECT c1, c2, c3, c1 FROM VALUES(100, 110, 120) AS T(c1, c2, c3)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "ASSIGNMENT_ARITY_MISMATCH", + "sqlState" : "42802", + "messageParameters" : { + "numExpr" : "4", + "numTarget" : "3" + } +} + + +-- !query +SET VARIABLE (var1, var2, var1) = (SELECT c1, c2, c3, c1 FROM VALUES(100, 110, 120) AS T(c1, c2, c3)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DUPLICATE_ASSIGNMENTS", + "sqlState" : "42701", + "messageParameters" : { + "nameList" : "`var1`" + } +} + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY VARIABLE var2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY VARIABLE var3 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE title = 'DEFAULT expression usage' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var1 STRING DEFAULT 'default1' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var2 STRING DEFAULT 'default2' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var3 STRING DEFAULT 'default3' +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE var1 = 'hello' +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE var1 = DEFAULT +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 AS `default` +-- !query schema +struct +-- !query output +default1 + + +-- !query +SET VARIABLE var1 = 'hello1' +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE var1 = 'hello2' +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE var1 = 'hello3' +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE var1 = DEFAULT, var2 = DEFAULT, var3 = DEFAULT +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 AS `default1`, var2 AS `default2`, var3 AS `default3` +-- !query schema +struct +-- !query output +default1 default2 default3 + + +-- !query +SET VARIABLE var1 = 'hello' +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE (var1) = (SELECT DEFAULT FROM VALUES(1) AS T(c1)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 AS `default` +-- !query schema +struct +-- !query output +default1 + + +-- !query +SET VARIABLE var1 = 'hello' +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE (var1) = (SELECT DEFAULT FROM VALUES('world') AS T(default)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 AS `world` +-- !query schema +struct +-- !query output +world + + +-- !query +SET VARIABLE var1 = 'hello' +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE (var1) = (SELECT DEFAULT FROM VALUES(1) AS T(c1) LIMIT 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 AS `default` +-- !query schema +struct +-- !query output +default1 + + +-- !query +SET VARIABLE var1 = 'hello' +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE (var1) = (SELECT DEFAULT FROM VALUES(1),(2),(3) AS T(c1) LIMIT 1 OFFSET 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 AS `default` +-- !query schema +struct +-- !query output +default1 + + +-- !query +SET VARIABLE var1 = 'hello' +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE (var1) = (SELECT DEFAULT FROM VALUES(1),(2),(3) AS T(c1) OFFSET 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +{ + "errorClass" : "ROW_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000" +} + + +-- !query +SELECT var1 AS `default` +-- !query schema +struct +-- !query output +hello + + +-- !query +SET VARIABLE var1 = 'hello' +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE (var1) = (WITH v1(c1) AS (VALUES(1) AS T(c1)) SELECT DEFAULT FROM VALUES(1),(2),(3) AS T(c1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +{ + "errorClass" : "ROW_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000" +} + + +-- !query +SELECT var1 AS `default` +-- !query schema +struct +-- !query output +hello + + +-- !query +SET VARIABLE var1 = 'Hello' || DEFAULT +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DEFAULT_PLACEMENT_INVALID", + "sqlState" : "42608" +} + + +-- !query +SET VARIABLE (var1) = (VALUES(DEFAULT)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE (var1) = (WITH v1(c1) AS (VALUES(1) AS T(c1)) SELECT DEFAULT + 1 FROM VALUES(1),(2),(3) AS T(c1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DEFAULT_PLACEMENT_INVALID", + "sqlState" : "42608" +} + + +-- !query +SET VARIABLE var1 = session.default +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`session`.`default`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 21, + "stopIndex" : 35, + "fragment" : "session.default" + } ] +} + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY VARIABLE var2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY VARIABLE var3 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE title = 'SET command' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET x.var1 = 5 +-- !query schema +struct +-- !query output +x.var1 5 + + +-- !query +SET x = 5 +-- !query schema +struct +-- !query output +x 5 + + +-- !query +SET system.x.var = 5 +-- !query schema +struct +-- !query output +system.x.var 5 + + +-- !query +SET x.session.var1 = 5 +-- !query schema +struct +-- !query output +x.session.var1 5 + + +-- !query +SET var1 = 5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.SET_VARIABLE_USING_SET", + "sqlState" : "0A000", + "messageParameters" : { + "variableName" : "`var1`" + } +} + + +-- !query +SET session.var1 = 5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.SET_VARIABLE_USING_SET", + "sqlState" : "0A000", + "messageParameters" : { + "variableName" : "`session`.`var1`" + } +} + + +-- !query +SET system.session.var1 = 5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.SET_VARIABLE_USING_SET", + "sqlState" : "0A000", + "messageParameters" : { + "variableName" : "`system`.`session`.`var1`" + } +} + + +-- !query +SET vAr1 = 5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.SET_VARIABLE_USING_SET", + "sqlState" : "0A000", + "messageParameters" : { + "variableName" : "`vAr1`" + } +} + + +-- !query +SET seSSion.var1 = 5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.SET_VARIABLE_USING_SET", + "sqlState" : "0A000", + "messageParameters" : { + "variableName" : "`seSSion`.`var1`" + } +} + + +-- !query +SET sYStem.session.var1 = 5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_FEATURE.SET_VARIABLE_USING_SET", + "sqlState" : "0A000", + "messageParameters" : { + "variableName" : "`sYStem`.`session`.`var1`" + } +} + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT var1 AS `2` FROM VALUES(2) AS T(var1) +-- !query schema +struct<2:int> +-- !query output +2 + + +-- !query +SELECT c1 AS `2` FROM VALUES(2) AS T(var1), LATERAL(SELECT var1) AS TT(c1) +-- !query schema +struct<2:int> +-- !query output +2 + + +-- !query +SELECT session.var1 AS `1` FROM VALUES(2) AS T(var1) +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT c1 AS `1` FROM VALUES(2) AS T(var1), LATERAL(SELECT session.var1) AS TT(c1) +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE title = 'variable references -- visibility' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +VALUES (var1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT var1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT sum(var1) FROM VALUES(1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT var1 + SUM(0) FROM VALUES(1) +-- !query schema +struct<(variablereference(system.session.var1=1) + sum(0)):bigint> +-- !query output +1 + + +-- !query +SELECT substr('12345', var1, 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT 1 FROM VALUES(1, 2) AS T(c1, c2) GROUP BY c1 + var1 +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT c1, sum(c2) FROM VALUES(1, 2) AS T(c1, c2) GROUP BY c1 HAVING sum(c1) != var1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 1 FROM VALUES(1) AS T(c1) WHERE c1 IN (var1) +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT sum(c1) FILTER (c1 != var1) FROM VALUES(1, 2), (2, 3) AS T(c1, c2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + } +} + + +-- !query +SELECT array(1, 2, 4)[var1] +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT 1 FROM VALUES(1) AS T(c1) WHERE c1 = var1 +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +WITH v1 AS (SELECT var1 AS c1) SELECT c1 AS `1` FROM v1 +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v AS SELECT var1 AS c1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 + + +-- !query +DROP VIEW v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE title = 'variable references -- prohibited' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var1 INT DEFAULT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v AS SELECT var1 AS c1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "sqlState" : "42K0F", + "messageParameters" : { + "obj" : "VIEW", + "objName" : "`spark_catalog`.`default`.`v`", + "tempObj" : "VARIABLE", + "tempObjName" : "`var1`" + } +} + + +-- !query +DROP VIEW IF EXISTS V +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VARIABLE title = 'variable references -- test constant folding' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE OR REPLACE VARIABLE var1 STRING DEFAULT 'a INT' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT from_json('{"a": 1}', var1) +-- !query schema +struct> +-- !query output +{"a":1} + + +-- !query +DROP TEMPORARY VARIABLE var1 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/sql-udf.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/sql-udf.sql.out new file mode 100644 index 000000000000..fc76ef810b5a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/sql-udf.sql.out @@ -0,0 +1,5435 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE FUNCTION foo1a0() RETURNS INT RETURN 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1a0() +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT foo1a0(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "0", + "functionName" : "`spark_catalog`.`default`.`foo1a0`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "foo1a0(1)" + } ] +} + + +-- !query +CREATE FUNCTION foo1a1(a INT) RETURNS INT RETURN 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1a1(1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT foo1a1(1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`spark_catalog`.`default`.`foo1a1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "foo1a1(1, 2)" + } ] +} + + +-- !query +CREATE FUNCTION foo1a2(a INT, b INT, c INT, d INT) RETURNS INT RETURN 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1a2(1, 2, 3, 4) +-- !query schema +struct +-- !query output +1 + + +-- !query +CREATE FUNCTION foo1b0() RETURNS TABLE (c1 INT) RETURN SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM foo1b0() +-- !query schema +struct +-- !query output +1 + + +-- !query +CREATE FUNCTION foo1b1(a INT) RETURNS TABLE (c1 INT) RETURN SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM foo1b1(1) +-- !query schema +struct +-- !query output +1 + + +-- !query +CREATE FUNCTION foo1b2(a INT, b INT, c INT, d INT) RETURNS TABLE(c1 INT) RETURN SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM foo1b2(1, 2, 3, 4) +-- !query schema +struct +-- !query output +1 + + +-- !query +CREATE FUNCTION foo1c1(duplicate INT, DUPLICATE INT) RETURNS INT RETURN 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DUPLICATE_ROUTINE_PARAMETER_NAMES", + "sqlState" : "42734", + "messageParameters" : { + "names" : "`duplicate`", + "routineName" : "foo1c1" + } +} + + +-- !query +CREATE FUNCTION foo1c2(a INT, b INT, thisisaduplicate INT, c INT, d INT, e INT, f INT, thisIsaDuplicate INT, g INT) + RETURNS TABLE (a INT) RETURN SELECT 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DUPLICATE_ROUTINE_PARAMETER_NAMES", + "sqlState" : "42734", + "messageParameters" : { + "names" : "`thisisaduplicate`", + "routineName" : "foo1c2" + } +} + + +-- !query +CREATE OR REPLACE FUNCTION foo1d1(a INT DEFAULT NULL) RETURNS INT RETURN a +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1d1(5), foo1d1() +-- !query schema +struct +-- !query output +5 NULL + + +-- !query +CREATE OR REPLACE FUNCTION foo1d1(a INT DEFAULT 10) RETURNS INT RETURN a +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1d1(5), foo1d1() +-- !query schema +struct +-- !query output +5 10 + + +-- !query +CREATE OR REPLACE FUNCTION foo1d1(a INT DEFAULT length(substr(current_database(), 1, 1))) RETURNS INT RETURN a +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1d1(5), foo1d1() +-- !query schema +struct +-- !query output +5 1 + + +-- !query +CREATE OR REPLACE FUNCTION foo1d1(a INT DEFAULT '5' || length(substr(current_database(), 1, 1))) + RETURNS INT RETURN a +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1d1(5), foo1d1() +-- !query schema +struct +-- !query output +5 51 + + +-- !query +CREATE OR REPLACE FUNCTION foo1d1(a INT DEFAULT RAND()::INT) RETURNS INT RETURN a +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1d1(5), foo1d1() +-- !query schema +struct +-- !query output +5 0 + + +-- !query +CREATE OR REPLACE FUNCTION foo1d1(a INT DEFAULT array(55, 17)) + RETURNS INT RETURN a +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"array(55, 17)\"", + "srcType" : "\"ARRAY\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 85, + "fragment" : "CREATE OR REPLACE FUNCTION foo1d1(a INT DEFAULT array(55, 17))\n RETURNS INT RETURN a" + } ] +} + + +-- !query +CREATE OR REPLACE FUNCTION foo1d1(a INT DEFAULT (SELECT max(c1) FROM VALUES (1) AS T(c1))) + RETURNS INT RETURN a +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "USER_DEFINED_FUNCTIONS.NOT_A_VALID_DEFAULT_EXPRESSION", + "sqlState" : "42601", + "messageParameters" : { + "functionName" : "foo1d1", + "parameterName" : "a" + } +} + + +-- !query +CREATE OR REPLACE FUNCTION foo1d2(a INT, b INT DEFAULT 7, c INT DEFAULT 8, d INT DEFAULT 9 COMMENT 'test') + RETURNS STRING RETURN a || ' ' || b || ' ' || c || ' ' || d +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1d2(1, 2, 3, 4), foo1d2(1, 2, 3), foo1d2(1, 2), foo1d2(1) +-- !query schema +struct +-- !query output +1 2 3 4 1 2 3 9 1 2 8 9 1 7 8 9 + + +-- !query +SELECT foo1d2() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "4", + "functionName" : "`spark_catalog`.`default`.`foo1d2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "foo1d2()" + } ] +} + + +-- !query +SELECT foo1d2(1, 2, 3, 4, 5) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "5", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "4", + "functionName" : "`spark_catalog`.`default`.`foo1d2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "foo1d2(1, 2, 3, 4, 5)" + } ] +} + + +-- !query +CREATE OR REPLACE FUNCTION foo1d2(a INT DEFAULT 5, b INT , c INT DEFAULT 8, d INT DEFAULT 9 COMMENT 'test') + RETURNS STRING RETURN a || ' ' || b || ' ' || c || ' ' || d +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "USER_DEFINED_FUNCTIONS.NOT_A_VALID_DEFAULT_PARAMETER_POSITION", + "sqlState" : "42601", + "messageParameters" : { + "functionName" : "foo1d2", + "nextParameterName" : "b", + "parameterName" : "a" + } +} + + +-- !query +CREATE OR REPLACE FUNCTION foo1d2(a INT, b INT DEFAULT 7, c INT DEFAULT 8, d INT COMMENT 'test') + RETURNS STRING RETURN a || ' ' || b || ' ' || c || ' ' || d +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "USER_DEFINED_FUNCTIONS.NOT_A_VALID_DEFAULT_PARAMETER_POSITION", + "sqlState" : "42601", + "messageParameters" : { + "functionName" : "foo1d2", + "nextParameterName" : "d", + "parameterName" : "c" + } +} + + +-- !query +CREATE OR REPLACE TEMPORARY FUNCTION foo1d3(a INT DEFAULT 7 COMMENT 'hello') RETURNS INT RETURN a +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1d3(5), foo1d3() +-- !query schema +struct +-- !query output +5 7 + + +-- !query +CREATE OR REPLACE FUNCTION foo1d4(a INT, b INT DEFAULT a) RETURNS INT RETURN a + b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 1, + "fragment" : "a" + } ] +} + + +-- !query +CREATE OR REPLACE FUNCTION foo1d4(a INT, b INT DEFAULT 3) RETURNS INT RETURN a + b +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE FUNCTION foo1d5(a INT, b INT DEFAULT foo1d4(6)) RETURNS INT RETURN a + b +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1d5(10), foo1d5(10, 2) +-- !query schema +struct +-- !query output +19 12 + + +-- !query +CREATE OR REPLACE FUNCTION foo1d5(a INT, b INT) RETURNS INT RETURN a + foo1d4(b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1d5(10, 2) +-- !query schema +struct +-- !query output +15 + + +-- !query +CREATE OR REPLACE FUNCTION foo1d6(a INT, b INT DEFAULT 7) RETURNS TABLE(a INT, b INT) RETURN SELECT a, b +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM foo1d6(5) +-- !query schema +struct +-- !query output +5 7 + + +-- !query +SELECT * FROM foo1d6(5, 2) +-- !query schema +struct +-- !query output +5 2 + + +-- !query +CREATE FUNCTION foo1e1(x INT NOT NULL, y INT) RETURNS INT RETURN 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "USER_DEFINED_FUNCTIONS.NOT_NULL_ON_FUNCTION_PARAMETERS", + "sqlState" : "42601", + "messageParameters" : { + "input" : "x INT NOT NULL, y INT" + } +} + + +-- !query +CREATE FUNCTION foo1e2(x INT, y INT NOT NULL) RETURNS TABLE (x INT) RETURN SELECT 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "USER_DEFINED_FUNCTIONS.NOT_NULL_ON_FUNCTION_PARAMETERS", + "sqlState" : "42601", + "messageParameters" : { + "input" : "x INT, y INT NOT NULL" + } +} + + +-- !query +CREATE FUNCTION foo1e3(x INT, y INT) RETURNS TABLE (x INT NOT NULL) RETURN SELECT 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "USER_DEFINED_FUNCTIONS.NOT_NULL_ON_FUNCTION_PARAMETERS", + "sqlState" : "42601", + "messageParameters" : { + "input" : "x INT NOT NULL" + } +} + + +-- !query +CREATE FUNCTION foo1f1(x INT, y INT GENERATED ALWAYS AS (x + 10)) RETURNS INT RETURN y + 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.CREATE_FUNC_WITH_GENERATED_COLUMNS_AS_PARAMETERS", + "sqlState" : "42000", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 90, + "fragment" : "CREATE FUNCTION foo1f1(x INT, y INT GENERATED ALWAYS AS (x + 10)) RETURNS INT RETURN y + 1" + } ] +} + + +-- !query +CREATE FUNCTION foo1f2(id BIGINT GENERATED ALWAYS AS IDENTITY) RETURNS BIGINT RETURN id + 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.CREATE_FUNC_WITH_GENERATED_COLUMNS_AS_PARAMETERS", + "sqlState" : "42000", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 91, + "fragment" : "CREATE FUNCTION foo1f2(id BIGINT GENERATED ALWAYS AS IDENTITY) RETURNS BIGINT RETURN id + 1" + } ] +} + + +-- !query +CREATE FUNCTION foo2a0() RETURNS TABLE() RETURN SELECT 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "')'", + "hint" : "" + } +} + + +-- !query +CREATE FUNCTION foo2a2() RETURNS TABLE(c1 INT, c2 INT) RETURN SELECT 1, 2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM foo2a2() +-- !query schema +struct +-- !query output +1 2 + + +-- !query +CREATE FUNCTION foo2a4() RETURNS TABLE(c1 INT, c2 INT, c3 INT, c4 INT) RETURN SELECT 1, 2, 3, 4 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM foo2a2() +-- !query schema +struct +-- !query output +1 2 + + +-- !query +CREATE FUNCTION foo2b1() RETURNS TABLE(DuPLiCatE INT, duplicate INT) RETURN SELECT 1, 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DUPLICATE_ROUTINE_RETURNS_COLUMNS", + "sqlState" : "42711", + "messageParameters" : { + "columns" : "`duplicate`", + "routineName" : "foo2b1" + } +} + + +-- !query +CREATE FUNCTION foo2b2() RETURNS TABLE(a INT, b INT, duplicate INT, c INT, d INT, e INT, DUPLICATE INT) +RETURN SELECT 1, 2, 3, 4, 5, 6, 7 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DUPLICATE_ROUTINE_RETURNS_COLUMNS", + "sqlState" : "42711", + "messageParameters" : { + "columns" : "`duplicate`", + "routineName" : "foo2b2" + } +} + + +-- !query +CREATE FUNCTION foo2c1() RETURNS TABLE(c1 INT DEFAULT 5) RETURN SELECT 1, 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'DEFAULT'", + "hint" : "" + } +} + + +-- !query +CREATE FUNCTION foo31() RETURNS INT RETURN (SELECT 1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_SUBQUERY_EXPRESSION.SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", + "sqlState" : "42823", + "messageParameters" : { + "number" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 56, + "fragment" : "CREATE FUNCTION foo31() RETURNS INT RETURN (SELECT 1, 2)" + } ] +} + + +-- !query +CREATE FUNCTION foo32() RETURNS TABLE(a INT) RETURN SELECT 1, 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "USER_DEFINED_FUNCTIONS.RETURN_COLUMN_COUNT_MISMATCH", + "sqlState" : "42601", + "messageParameters" : { + "name" : "spark_catalog.default.foo32", + "outputSize" : "2", + "returnParamSize" : "1" + } +} + + +-- !query +CREATE FUNCTION foo33() RETURNS TABLE(a INT, b INT) RETURN SELECT 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "USER_DEFINED_FUNCTIONS.RETURN_COLUMN_COUNT_MISMATCH", + "sqlState" : "42601", + "messageParameters" : { + "name" : "spark_catalog.default.foo33", + "outputSize" : "1", + "returnParamSize" : "2" + } +} + + +-- !query +CREATE FUNCTION foo41() RETURNS INT RETURN SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo42() RETURNS TABLE(a INT) RETURN 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "USER_DEFINED_FUNCTIONS.SQL_TABLE_UDF_BODY_MUST_BE_A_QUERY", + "sqlState" : "42601", + "messageParameters" : { + "name" : "foo42" + } +} + + +-- !query +CREATE FUNCTION foo51() RETURNS INT RETURN (SELECT a FROM VALUES(1), (2) AS T(a)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo51() +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +{ + "errorClass" : "SCALAR_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "fragment" : "" + } ] +} + + +-- !query +CREATE FUNCTION foo52() RETURNS INT RETURN (SELECT 1 FROM VALUES(1) WHERE 1 = 0) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo52() +-- !query schema +struct +-- !query output +NULL + + +-- !query +CREATE FUNCTION foo6c(` a` INT, a INT, `a b` INT) RETURNS INT RETURN 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo6c(1, 2, 3) +-- !query schema +struct +-- !query output +1 + + +-- !query +CREATE FUNCTION foo6d() RETURNS TABLE(` a` INT, a INT, `a b` INT) RETURN SELECT 1, 2, 3 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM foo6d() +-- !query schema +struct< a:int,a:int,a b:int> +-- !query output +1 2 3 + + +-- !query +CREATE FUNCTION foo7a(a STRING, b STRING, c STRING) RETURNS STRING RETURN +SELECT 'Foo.a: ' || a || ' Foo.a: ' || foo7a.a + || ' T.b: ' || b || ' Foo.b: ' || foo7a.b + || ' T.c: ' || c || ' T.c: ' || t.c FROM VALUES('t.b', 't.c') AS T(b, c) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo7a('Foo.a', 'Foo.b', 'Foo.c') +-- !query schema +struct +-- !query output +Foo.a: Foo.a Foo.a: Foo.a T.b: t.b Foo.b: Foo.b T.c: t.c T.c: t.c + + +-- !query +CREATE FUNCTION foo7at(a STRING, b STRING, c STRING) RETURNS TABLE (a STRING, b STRING, c STRING, d STRING, e STRING) RETURN +SELECT CONCAT('Foo.a: ', a), CONCAT('Foo.b: ', foo7at.b), CONCAT('T.b: ', b), + CONCAT('Foo.c: ', foo7at.c), CONCAT('T.c: ', c) +FROM VALUES ('t.b', 't.c') AS T(b, c) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM foo7at('Foo.a', 'Foo.b', 'Foo.c') +-- !query schema +struct +-- !query output +Foo.a: Foo.a Foo.b: Foo.b T.b: t.b Foo.c: Foo.c T.c: t.c + + +-- !query +CREATE FUNCTION foo9a(a BOOLEAN) RETURNS BOOLEAN RETURN NOT a +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo9a(true) +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT foo9a(5) +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT foo9a('Nonsense') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'Nonsense'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "fragment" : "" + } ] +} + + +-- !query +CREATE FUNCTION foo9b(a BYTE) RETURNS BYTE RETURN CAST(a AS SHORT) + 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo9b(126) +-- !query schema +struct +-- !query output +127 + + +-- !query +SELECT foo9b(127) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"INT\"", + "targetType" : "\"TINYINT\"", + "value" : "128" + } +} + + +-- !query +SELECT foo9b(128) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"INT\"", + "targetType" : "\"TINYINT\"", + "value" : "128" + } +} + + +-- !query +CREATE FUNCTION foo9c(a SHORT) RETURNS SHORT RETURN CAST(a AS INTEGER) + 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo9c(32766) +-- !query schema +struct +-- !query output +32767 + + +-- !query +SELECT foo9c(32767) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"INT\"", + "targetType" : "\"SMALLINT\"", + "value" : "32768" + } +} + + +-- !query +SELECT foo9c(32768) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"INT\"", + "targetType" : "\"SMALLINT\"", + "value" : "32768" + } +} + + +-- !query +CREATE FUNCTION foo9d(a INTEGER) RETURNS INTEGER RETURN CAST(a AS BIGINT) + 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo9d(2147483647 - 1) +-- !query schema +struct +-- !query output +2147483647 + + +-- !query +SELECT foo9d(2147483647) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"BIGINT\"", + "targetType" : "\"INT\"", + "value" : "2147483648L" + } +} + + +-- !query +SELECT foo9d(2147483647 + 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 14, + "stopIndex" : 27, + "fragment" : "2147483647 + 1" + } ] +} + + +-- !query +CREATE FUNCTION foo9e(a BIGINT) RETURNS BIGINT RETURN CAST(a AS DECIMAL(20, 0)) + 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo9e(9223372036854775807 - 1) +-- !query schema +struct +-- !query output +9223372036854775807 + + +-- !query +SELECT foo9e(9223372036854775807) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"DECIMAL(21,0)\"", + "targetType" : "\"BIGINT\"", + "value" : "9223372036854775808BD" + } +} + + +-- !query +SELECT foo9e(9223372036854775807.0 + 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "CAST_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "sourceType" : "\"DECIMAL(21,1)\"", + "targetType" : "\"BIGINT\"", + "value" : "9223372036854775808.0BD" + } +} + + +-- !query +CREATE FUNCTION foo9f(a DECIMAL( 5, 2 )) RETURNS DECIMAL (5, 2) RETURN CAST(a AS DECIMAL(6, 2)) + 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo9f(999 - 1) +-- !query schema +struct +-- !query output +999.00 + + +-- !query +SELECT foo9f(999) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "5", + "scale" : "2", + "value" : "1000.00" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "fragment" : "" + } ] +} + + +-- !query +SELECT foo9f(999 + 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", + "sqlState" : "22003", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "precision" : "5", + "scale" : "2", + "value" : "1000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "fragment" : "" + } ] +} + + +-- !query +CREATE FUNCTION foo9g(a FLOAT, b String) RETURNS FLOAT RETURN b || CAST(a AS String) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo9g(123.23, '7') +-- !query schema +struct +-- !query output +7123.23 + + +-- !query +SELECT foo9g('hello', '7') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'hello'", + "sourceType" : "\"STRING\"", + "targetType" : "\"FLOAT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "fragment" : "" + } ] +} + + +-- !query +SELECT foo9g(123.23, 'q') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'q123.23'", + "sourceType" : "\"STRING\"", + "targetType" : "\"FLOAT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "fragment" : "" + } ] +} + + +-- !query +CREATE FUNCTION foo9h(a DOUBLE, b String) RETURNS DOUBLE RETURN b || CAST(a AS String) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo9h(123.23, '7') +-- !query schema +struct +-- !query output +7123.23 + + +-- !query +SELECT foo9h('hello', '7') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'hello'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "fragment" : "" + } ] +} + + +-- !query +SELECT foo9h(123.23, 'q') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'q123.23'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "fragment" : "" + } ] +} + + +-- !query +CREATE FUNCTION foo9i(a VARCHAR(10), b VARCHAR(10)) RETURNS VARCHAR(12) RETURN a || b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_CHAR_OR_VARCHAR_AS_STRING", + "sqlState" : "0A000" +} + + +-- !query +CREATE FUNCTION foo9j(a STRING, b STRING) RETURNS STRING RETURN a || b +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo9j('1234567890', '12') +-- !query schema +struct +-- !query output +123456789012 + + +-- !query +SELECT foo9j(12345678901, '12') +-- !query schema +struct +-- !query output +1234567890112 + + +-- !query +CREATE FUNCTION foo9l(a DATE, b INTERVAL) RETURNS DATE RETURN a + b +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo9l(DATE '2020-02-02', INTERVAL '1' YEAR) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"INTERVAL '1' YEAR\"", + "srcType" : "\"INTERVAL YEAR\"", + "targetType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "foo9l(DATE '2020-02-02', INTERVAL '1' YEAR)" + } ] +} + + +-- !query +SELECT foo9l('2020-02-02', INTERVAL '1' YEAR) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"INTERVAL '1' YEAR\"", + "srcType" : "\"INTERVAL YEAR\"", + "targetType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "foo9l('2020-02-02', INTERVAL '1' YEAR)" + } ] +} + + +-- !query +SELECT foo9l(DATE '-7', INTERVAL '1' YEAR) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'-7'", + "valueType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 14, + "stopIndex" : 22, + "fragment" : "DATE '-7'" + } ] +} + + +-- !query +SELECT foo9l(DATE '2020-02-02', INTERVAL '9999999' YEAR) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"INTERVAL '9999999' YEAR\"", + "srcType" : "\"INTERVAL YEAR\"", + "targetType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "foo9l(DATE '2020-02-02', INTERVAL '9999999' YEAR)" + } ] +} + + +-- !query +CREATE FUNCTION foo9m(a TIMESTAMP, b INTERVAL) RETURNS TIMESTAMP RETURN a + b +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo9m(TIMESTAMP'2020-02-02 12:15:16.123', INTERVAL '1' YEAR) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"INTERVAL '1' YEAR\"", + "srcType" : "\"INTERVAL YEAR\"", + "targetType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 67, + "fragment" : "foo9m(TIMESTAMP'2020-02-02 12:15:16.123', INTERVAL '1' YEAR)" + } ] +} + + +-- !query +SELECT foo9m('2020-02-02 12:15:16.123', INTERVAL '1' YEAR) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"INTERVAL '1' YEAR\"", + "srcType" : "\"INTERVAL YEAR\"", + "targetType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "foo9m('2020-02-02 12:15:16.123', INTERVAL '1' YEAR)" + } ] +} + + +-- !query +SELECT foo9m(TIMESTAMP'2020-02-02 12:15:16.123', INTERVAL '999999' YEAR) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"INTERVAL '999999' YEAR\"", + "srcType" : "\"INTERVAL YEAR\"", + "targetType" : "\"INTERVAL\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "foo9m(TIMESTAMP'2020-02-02 12:15:16.123', INTERVAL '999999' YEAR)" + } ] +} + + +-- !query +CREATE FUNCTION foo9n(a ARRAY) RETURNS ARRAY RETURN a +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo9n(ARRAY(1, 2, 3)) +-- !query schema +struct> +-- !query output +[1,2,3] + + +-- !query +SELECT foo9n(from_json('[1, 2, 3]', 'array')) +-- !query schema +struct> +-- !query output +[1,2,3] + + +-- !query +CREATE FUNCTION foo9o(a MAP) RETURNS MAP RETURN a +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo9o(MAP('hello', 1, 'world', 2)) +-- !query schema +struct> +-- !query output +{"hello":1,"world":2} + + +-- !query +SELECT foo9o(from_json('{"hello":1, "world":2}', 'map')) +-- !query schema +struct> +-- !query output +{"hello":1,"world":2} + + +-- !query +CREATE FUNCTION foo9p(a STRUCT) RETURNS STRUCT RETURN a +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo9p(STRUCT(1, 'hello')) +-- !query schema +struct> +-- !query output +{"a1":1,"a2":"hello"} + + +-- !query +SELECT foo9p(from_json('{1:"hello"}', 'struct')) +-- !query schema +struct> +-- !query output +{"a1":null,"a2":null} + + +-- !query +CREATE FUNCTION foo9q(a ARRAY>) RETURNS ARRAY> RETURN a +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo9q(ARRAY(STRUCT(1, 'hello'), STRUCT(2, 'world'))) +-- !query schema +struct>> +-- !query output +[{"a1":1,"a2":"hello"},{"a1":2,"a2":"world"}] + + +-- !query +SELECT foo9q(ARRAY(NAMED_STRUCT('x', 1, 'y', 'hello'), NAMED_STRUCT('x', 2, 'y', 'world'))) +-- !query schema +struct>> +-- !query output +[{"a1":1,"a2":"hello"},{"a1":2,"a2":"world"}] + + +-- !query +SELECT foo9q(from_json('[{1:"hello"}, {2:"world"}]', 'array>')) +-- !query schema +struct>> +-- !query output +NULL + + +-- !query +CREATE FUNCTION foo9r(a ARRAY>) RETURNS ARRAY> RETURN a +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo9r(ARRAY(MAP('hello', 1), MAP('world', 2))) +-- !query schema +struct>> +-- !query output +[{"hello":1},{"world":2}] + + +-- !query +SELECT foo9r(from_json('[{"hello":1}, {"world":2}]', 'array>')) +-- !query schema +struct>> +-- !query output +[{"hello":1},{"world":2}] + + +-- !query +CREATE OR REPLACE FUNCTION foo1_10(a INT) RETURNS INT RETURN a + 2 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE FUNCTION bar1_10(b INT) RETURNS STRING RETURN foo1_10(TRY_CAST(b AS STRING)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT bar1_10(3) +-- !query schema +struct +-- !query output +5 + + +-- !query +CREATE OR REPLACE FUNCTION foo1_11a() RETURN 42 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1_11a() +-- !query schema +struct +-- !query output +42 + + +-- !query +CREATE OR REPLACE FUNCTION foo1_11b() RETURN 'hello world' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1_11b() +-- !query schema +struct +-- !query output +hello world + + +-- !query +CREATE OR REPLACE FUNCTION foo1_11c(a INT, b INT) RETURN a + b +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1_11c(3, 5) +-- !query schema +struct +-- !query output +8 + + +-- !query +CREATE OR REPLACE FUNCTION foo1_11d(a DOUBLE, b INT) RETURN a * b + 1.5 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1_11d(3.0, 5) +-- !query schema +struct +-- !query output +16.5 + + +-- !query +CREATE OR REPLACE FUNCTION foo1_11e(a INT) RETURN a > 10 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1_11e(15), foo1_11e(5) +-- !query schema +struct +-- !query output +true false + + +-- !query +CREATE OR REPLACE FUNCTION foo1_11f(d DATE) RETURN d + INTERVAL '1' DAY +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1_11f(DATE '2024-01-01') +-- !query schema +struct +-- !query output +2024-01-02 + + +-- !query +CREATE OR REPLACE FUNCTION foo1_11g(n INT) RETURN ARRAY(1, 2, n) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1_11g(5) +-- !query schema +struct> +-- !query output +[1,2,5] + + +-- !query +CREATE OR REPLACE FUNCTION foo1_11h(a INT, b STRING) RETURN STRUCT(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1_11h(1, 'test') +-- !query schema +struct> +-- !query output +{"a":1,"b":"test"} + + +-- !query +CREATE OR REPLACE FUNCTION foo1_11i(x INT) RETURN (SELECT x * 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1_11i(5) +-- !query schema +struct +-- !query output +10 + + +-- !query +CREATE OR REPLACE FUNCTION foo1_11j(s STRING) RETURN UPPER(s) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1_11j('hello') +-- !query schema +struct +-- !query output +HELLO + + +-- !query +CREATE OR REPLACE FUNCTION foo1_11k(a INT, b STRING) RETURN CONCAT(CAST(a AS STRING), '_', b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1_11k(123, 'test') +-- !query schema +struct +-- !query output +123_test + + +-- !query +CREATE OR REPLACE FUNCTION foo1_11l() RETURNS TABLE RETURN SELECT 1 as id, 'hello' as name +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM foo1_11l() +-- !query schema +struct +-- !query output +1 hello + + +-- !query +CREATE OR REPLACE FUNCTION foo1_11m(a INT, b STRING) RETURNS TABLE RETURN SELECT a * 2 as doubled, UPPER(b) as upper_name +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM foo1_11m(5, 'world') +-- !query schema +struct +-- !query output +10 WORLD + + +-- !query +CREATE OR REPLACE FUNCTION foo1_11n(arr ARRAY) RETURNS TABLE RETURN SELECT size(arr) as array_size, arr[0] as first_element +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM foo1_11n(ARRAY(1, 2, 3)) +-- !query schema +struct +-- !query output +3 1 + + +-- !query +CREATE OR REPLACE FUNCTION foo1_11o(id INT, name STRING) RETURNS TABLE RETURN SELECT STRUCT(id, name) as person_info, id + 100 as modified_id +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM foo1_11o(1, 'Alice') +-- !query schema +struct,modified_id:int> +-- !query output +{"id":1,"name":"Alice"} 101 + + +-- !query +CREATE FUNCTION foo2_1a(a INT) RETURNS INT RETURN a +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_1a(5) +-- !query schema +struct +-- !query output +5 + + +-- !query +CREATE FUNCTION foo2_1b(a INT, b INT) RETURNS INT RETURN a + b +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_1b(5, 6) +-- !query schema +struct +-- !query output +11 + + +-- !query +CREATE FUNCTION foo2_1c(a INT, b INT) RETURNS INT RETURN 10 * (a + b) + 100 * (a -b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_1c(5, 6) +-- !query schema +struct +-- !query output +10 + + +-- !query +CREATE FUNCTION foo2_1d(a INT, b INT) RETURNS INT RETURN ABS(a) - LENGTH(CAST(b AS VARCHAR(10))) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_1d(-5, 6) +-- !query schema +struct +-- !query output +4 + + +-- !query +CREATE FUNCTION foo2_2a(a INT) RETURNS INT RETURN SELECT a +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_2a(5) +-- !query schema +struct +-- !query output +5 + + +-- !query +CREATE FUNCTION foo2_2b(a INT) RETURNS INT RETURN 1 + (SELECT a) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_2b(5) +-- !query schema +struct +-- !query output +6 + + +-- !query +CREATE FUNCTION foo2_2c(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT a)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 21, + "stopIndex" : 21, + "fragment" : "a" + } ] +} + + +-- !query +CREATE FUNCTION foo2_2d(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT (SELECT (SELECT a)))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 37, + "stopIndex" : 37, + "fragment" : "a" + } ] +} + + +-- !query +CREATE FUNCTION foo2_2e(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) WHERE c1 = 2 +UNION ALL +SELECT a + 1 FROM (VALUES 1) AS V(c1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo2_2f(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) +EXCEPT +SELECT a + 1 FROM (VALUES 1) AS V(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo2_2g(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) +INTERSECT +SELECT a FROM (VALUES 1) AS V(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS ts +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS tm +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS ta +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS V1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS V2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS ts +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS tm +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS ta +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS V1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS V2 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo2_3(a INT, b INT) RETURNS INT RETURN a + b +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW V1(c1, c2) AS VALUES (1, 2), (3, 4), (5, 6) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW V2(c1, c2) AS VALUES (-1, -2), (-3, -4), (-5, -6) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_3(c1, c2), foo2_3(c2, 1), foo2_3(c1, c2) - foo2_3(c2, c1 - 1) FROM V1 ORDER BY 1, 2, 3 +-- !query schema +struct +-- !query output +3 3 1 +7 5 1 +11 7 1 + + +-- !query +SELECT * FROM V1 WHERE foo2_3(c1, 0) = c1 AND foo2_3(c1, c2) < 8 +-- !query schema +struct +-- !query output +1 2 +3 4 + + +-- !query +SELECT foo2_3(SUM(c1), SUM(c2)), SUM(c1) + SUM(c2), SUM(foo2_3(c1, c2) + foo2_3(c2, c1) - foo2_3(c2, c1)) +FROM V1 +-- !query schema +struct +-- !query output +21 21 21 + + +-- !query +CREATE FUNCTION foo2_4a(a ARRAY) RETURNS STRING RETURN +SELECT array_sort(a, (i, j) -> rank[i] - rank[j])[0] FROM (SELECT MAP('a', 1, 'b', 2) rank) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_4a(ARRAY('a', 'b')) +-- !query schema +struct +-- !query output +a + + +-- !query +CREATE FUNCTION foo2_4b(m MAP, k STRING) RETURNS STRING RETURN +SELECT v || ' ' || v FROM (SELECT upper(m[k]) AS v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_4b(map('a', 'hello', 'b', 'world'), 'a') +-- !query schema +struct +-- !query output +HELLO HELLO + + +-- !query +DROP VIEW V2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW V1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (0, 2), (1, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW t2(c1, c2) AS VALUES (0, 2), (0, 3) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW ts(x) AS VALUES NAMED_STRUCT('a', 1, 'b', 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW tm(x) AS VALUES MAP('a', 1, 'b', 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW ta(x) AS VALUES ARRAY(1, 2, 3) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_1a(a DOUBLE, b DOUBLE) RETURNS DOUBLE RETURN a * b +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_1b(x INT) RETURNS INT RETURN x +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_1c(x INT) RETURNS INT RETURN SELECT x +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_1d(x INT) RETURNS INT RETURN (SELECT SUM(c2) FROM t2 WHERE c1 = x) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_1e() RETURNS INT RETURN foo3_1d(0) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_1f() RETURNS INT RETURN SELECT SUM(c2) FROM t2 WHERE c1 = 0 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_1g(x INT) RETURNS INT RETURN SELECT (SELECT x) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT a, b, foo3_1a(a + 1, b + 1) FROM t1 AS t(a, b) +-- !query schema +struct +-- !query output +0 1 2.0 +0 2 3.0 +1 2 6.0 + + +-- !query +SELECT x, foo3_1c(x) FROM t1 AS t(x, y) +-- !query schema +struct +-- !query output +0 0 +0 0 +1 1 + + +-- !query +SELECT c1, foo3_1d(c1) FROM t1 +-- !query schema +struct +-- !query output +0 5 +0 5 +1 NULL + + +-- !query +SELECT c1, foo3_1a(foo3_1b(c1), foo3_1b(c1)) FROM t1 +-- !query schema +struct +-- !query output +0 0.0 +0 0.0 +1 1.0 + + +-- !query +SELECT c1, foo3_1d(foo3_1c(foo3_1b(c1))) FROM t1 +-- !query schema +struct +-- !query output +0 5 +0 5 +1 NULL + + +-- !query +SELECT c1, foo3_1a(foo3_1c(foo3_1b(c1)), foo3_1d(foo3_1b(c1))) FROM t1 +-- !query schema +struct +-- !query output +0 0.0 +0 0.0 +1 NULL + + +-- !query +SELECT foo3_1c(foo3_1e()) FROM t1 +-- !query schema +struct +-- !query output +5 +5 +5 + + +-- !query +SELECT foo3_1a(MAX(c1), MAX(c2)) FROM t1 +-- !query schema +struct +-- !query output +2.0 + + +-- !query +SELECT foo3_1a(MAX(c1), c2) FROM t1 GROUP BY c2 +-- !query schema +struct +-- !query output +0.0 +2.0 + + +-- !query +SELECT foo3_1a(c1, c2) FROM t1 GROUP BY c1, c2 +-- !query schema +struct +-- !query output +0.0 +0.0 +2.0 + + +-- !query +SELECT MAX(foo3_1a(c1, c2)) FROM t1 GROUP BY c1, c2 +-- !query schema +struct +-- !query output +0.0 +0.0 +2.0 + + +-- !query +SELECT MAX(c1) + foo3_1b(MAX(c1)) FROM t1 GROUP BY c2 +-- !query schema +struct<(max(c1) + spark_catalog.default.foo3_1b(max(c1))):int> +-- !query output +0 +2 + + +-- !query +SELECT c1, SUM(foo3_1c(c2)) FROM t1 GROUP BY c1 +-- !query schema +struct +-- !query output +0 3 +1 2 + + +-- !query +SELECT c1, SUM(foo3_1d(c2)) FROM t1 GROUP BY c1 +-- !query schema +struct +-- !query output +0 NULL +1 NULL + + +-- !query +SELECT foo3_1c(c1), foo3_1d(c1) FROM t1 GROUP BY c1 +-- !query schema +struct +-- !query output +0 5 +1 NULL + + +-- !query +SELECT foo3_1a(SUM(c1), rand(0) * 0) FROM t1 +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT foo3_1a(SUM(c1) + rand(0) * 0, SUM(c2)) FROM t1 +-- !query schema +struct +-- !query output +5.0 + + +-- !query +SELECT foo3_1b(SUM(c1) + rand(0) * 0) FROM t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT foo3_1b(SUM(1) + rand(0) * 0) FROM t1 GROUP BY c2 +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT foo3_1c(SUM(c2) + rand(0) * 0) FROM t1 GROUP by c1 +-- !query schema +struct +-- !query output +2 +3 + + +-- !query +SELECT foo3_1b(foo3_1b(MAX(c2))) FROM t1 +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT foo3_1b(MAX(foo3_1b(c2))) FROM t1 +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT foo3_1a(foo3_1b(c1), MAX(c2)) FROM t1 GROUP BY c1 +-- !query schema +struct +-- !query output +0.0 +2.0 + + +-- !query +SELECT c1, foo3_1b(c1) FROM t1 GROUP BY c1 +-- !query schema +struct +-- !query output +0 0 +1 1 + + +-- !query +SELECT c1, foo3_1b(c1 + 1) FROM t1 GROUP BY c1 +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +SELECT c1, foo3_1b(c1 + rand(0) * 0) FROM t1 GROUP BY c1 +-- !query schema +struct +-- !query output +0 0 +1 1 + + +-- !query +SELECT c1, foo3_1a(c1, MIN(c2)) FROM t1 GROUP BY c1 +-- !query schema +struct +-- !query output +0 0.0 +1 2.0 + + +-- !query +SELECT c1, foo3_1a(c1 + 1, MIN(c2 + 1)) FROM t1 GROUP BY c1 +-- !query schema +struct +-- !query output +0 2.0 +1 6.0 + + +-- !query +SELECT c1, c2, foo3_1a(c1, c2) FROM t1 GROUP BY c1, c2 +-- !query schema +struct +-- !query output +0 1 0.0 +0 2 0.0 +1 2 2.0 + + +-- !query +SELECT c1, c2, foo3_1a(1, 2) FROM t1 GROUP BY c1, c2 +-- !query schema +struct +-- !query output +0 1 2.0 +0 2 2.0 +1 2 2.0 + + +-- !query +SELECT c1 + c2, foo3_1b(c1 + c2 + 1) FROM t1 GROUP BY c1 + c2 +-- !query schema +struct<(c1 + c2):int,spark_catalog.default.foo3_1b(((c1 + c2) + 1)):int> +-- !query output +1 2 +2 3 +3 4 + + +-- !query +SELECT COUNT(*) + foo3_1b(c1) + foo3_1b(SUM(c2)) + SUM(foo3_1b(c2)) FROM t1 GROUP BY c1 +-- !query schema +struct<(((count(1) + spark_catalog.default.foo3_1b(c1)) + spark_catalog.default.foo3_1b(sum(c2))) + sum(spark_catalog.default.foo3_1b(c2))):bigint> +-- !query output +6 +8 + + +-- !query +SELECT c1, COUNT(*), foo3_1b(SUM(c2)) FROM t1 GROUP BY c1 HAVING COUNT(*) > 0 +-- !query schema +struct +-- !query output +0 2 3 +1 1 2 + + +-- !query +SELECT c1, COUNT(*), foo3_1b(SUM(c2)) FROM t1 GROUP BY c1 HAVING foo3_1b(SUM(c2)) > 0 +-- !query schema +struct +-- !query output +0 2 3 +1 1 2 + + +-- !query +SELECT c1, COUNT(*), foo3_1b(SUM(c2)) FROM t1 GROUP BY c1 HAVING SUM(foo3_1b(c2)) > 0 +-- !query schema +struct +-- !query output +0 2 3 +1 1 2 + + +-- !query +SELECT foo3_1b(c1), MIN(c2) FROM t1 GROUP BY 1 +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +SELECT foo3_1a(c1 + rand(0) * 0, c2) FROM t1 GROUP BY 1 +-- !query schema +struct +-- !query output +0.0 +2.0 + + +-- !query +SELECT c1, c2, foo3_1a(c1, c2) FROM t1 GROUP BY c1, c2, 3 +-- !query schema +struct +-- !query output +0 1 0.0 +0 2 0.0 +1 2 2.0 + + +-- !query +SELECT c1, (SELECT c1), (SELECT foo3_1b(c1)), SUM(c2) FROM t1 GROUP BY 1, 2, 3 +-- !query schema +struct +-- !query output +0 0 0 3 +1 1 1 2 + + +-- !query +SELECT c1, SUM(c2) + foo3_1a(MIN(c2), MAX(c2)) + (SELECT SUM(c2)) FROM t1 GROUP BY c1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"sum(c2) AS `sum(outer(spark_catalog.default.t1.c2))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 64, + "fragment" : "SELECT SUM(c2)" + } ] +} + + +-- !query +SELECT foo3_1b(SUM(c1)) + (SELECT foo3_1b(SUM(c1))) FROM t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"sum(c1) AS `sum(outer(spark_catalog.default.t1.c1))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 51, + "fragment" : "(SELECT foo3_1b(SUM(c1)))" + } ] +} + + +-- !query +SELECT SUM(foo3_1b(SUM(c1))) FROM t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NESTED_AGGREGATE_FUNCTION", + "sqlState" : "42607", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 27, + "fragment" : "foo3_1b(SUM(c1))" + } ] +} + + +-- !query +SELECT foo3_1b(SUM(c1)) + (SELECT SUM(SUM(c1))) FROM t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NESTED_AGGREGATE_FUNCTION", + "sqlState" : "42607", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 45, + "fragment" : "SUM(c1)" + } ] +} + + +-- !query +SELECT foo3_1b(SUM(c1) + SUM(SUM(c1))) FROM t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NESTED_AGGREGATE_FUNCTION", + "sqlState" : "42607", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 36, + "fragment" : "SUM(c1)" + } ] +} + + +-- !query +SELECT foo3_1b(SUM(c1 + rand(0) * 0)) FROM t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "AGGREGATE_FUNCTION_WITH_NONDETERMINISTIC_EXPRESSION", + "sqlState" : "42845", + "messageParameters" : { + "sqlExpr" : "\"sum((c1 + (rand(0) * 0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 20, + "stopIndex" : 35, + "fragment" : "c1 + rand(0) * 0" + } ] +} + + +-- !query +SELECT SUM(foo3_1b(c1) + rand(0) * 0) FROM t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "AGGREGATE_FUNCTION_WITH_NONDETERMINISTIC_EXPRESSION", + "sqlState" : "42845", + "messageParameters" : { + "sqlExpr" : "\"sum((spark_catalog.default.foo3_1b(foo3_1b.x) + (rand(0) * 0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 36, + "fragment" : "foo3_1b(c1) + rand(0) * 0" + } ] +} + + +-- !query +SELECT SUM(foo3_1b(c1 + rand(0) * 0)) FROM t1 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT foo3_1b(SUM(c1) + foo3_1b(SUM(c1))) FROM t1 +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT foo3_1b(SUM(c2) + foo3_1b(SUM(c1))) AS foo FROM t1 HAVING foo > 0 +-- !query schema +struct +-- !query output +6 + + +-- !query +SELECT c1, COUNT(*), foo3_1b(SUM(c2) + foo3_1b(SUM(c2))) FROM t1 GROUP BY c1 HAVING COUNT(*) > 0 +-- !query schema +struct +-- !query output +0 2 6 +1 1 4 + + +-- !query +SELECT foo3_1a(c1, MAX(c2)) FROM t1 GROUP BY c1, 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "spark_catalog.default.foo3_1a(spark_catalog.default.t1.c1, max(spark_catalog.default.t1.c2)) AS `spark_catalog.default.foo3_1a(c1, max(c2))`", + "index" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 50, + "stopIndex" : 50, + "fragment" : "1" + } ] +} + + +-- !query +WITH cte AS (SELECT foo3_1a(c1, c2) FROM t1) +SELECT * FROM cte +-- !query schema +struct +-- !query output +0.0 +0.0 +2.0 + + +-- !query +SELECT SUM(c2) FROM t1 GROUP BY foo3_1b(c1) +-- !query schema +struct +-- !query output +2 +3 + + +-- !query +SELECT foo3_1b(c1), SUM(c2) FROM t1 GROUP BY 1 +-- !query schema +struct +-- !query output +0 3 +1 2 + + +-- !query +SELECT foo3_1b(c1), c2, GROUPING(foo3_1b(c1)), SUM(c1) FROM t1 GROUP BY ROLLUP(foo3_1b(c1), c2) +-- !query schema +struct +-- !query output +0 1 0 0 +0 2 0 0 +0 NULL 0 0 +1 2 0 1 +1 NULL 0 1 +NULL NULL 1 1 + + +-- !query +SELECT c1, SUM(c2) FROM t1 GROUP BY c1 HAVING foo3_1b(SUM(c2)) > 1 +-- !query schema +struct +-- !query output +0 3 +1 2 + + +-- !query +SELECT c1, SUM(c2) FROM t1 GROUP BY CUBE(c1) HAVING foo3_1b(GROUPING(c1)) = 0 +-- !query schema +struct +-- !query output +0 3 +1 2 + + +-- !query +SELECT * FROM t1 JOIN t2 ON foo3_1a(t1.c1, t2.c2) >= 2 +-- !query schema +struct +-- !query output +1 2 0 2 +1 2 0 3 + + +-- !query +SELECT * FROM t1 JOIN t2 ON foo3_1b(t1.c2) = foo3_1b(t2.c2) +-- !query schema +struct +-- !query output +0 2 0 2 +1 2 0 2 + + +-- !query +SELECT * FROM t1 JOIN t2 ON foo3_1b(t1.c1 + t2.c1 + 2) > 2 +-- !query schema +struct +-- !query output +1 2 0 2 +1 2 0 3 + + +-- !query +SELECT * FROM t1 JOIN t2 ON foo3_1a(foo3_1b(t1.c1), t2.c2) >= 2 +-- !query schema +struct +-- !query output +1 2 0 2 +1 2 0 3 + + +-- !query +SELECT * FROM t1 JOIN t2 ON foo3_1f() > 0 +-- !query schema +struct +-- !query output +0 1 0 2 +0 1 0 3 +0 2 0 2 +0 2 0 3 +1 2 0 2 +1 2 0 3 + + +-- !query +SELECT * FROM t1 JOIN t2 ON foo3_1b(t1.c1 + rand(0) * 0) > 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_NON_DETERMINISTIC_EXPRESSIONS", + "sqlState" : "42K0E", + "messageParameters" : { + "sqlExprs" : "\"(spark_catalog.default.foo3_1b(foo3_1b.x) > 1)\"" + } +} + + +-- !query +SELECT * FROM t1 JOIN t2 ON foo3_1c(t1.c1) = 2 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM t1 JOIN t2 ON foo3_1g(t1.c1) = 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_SCALAR_SUBQUERY", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Join Inner, (spark_catalog.default.foo3_1g(x#x) = 2)\n: +- Project [outer(x#x)]\n: +- OneRowRelation\n:- SubqueryAlias spark_catalog.default.t1\n: +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])\n: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n: +- LocalRelation [col1#x, col2#x]\n+- SubqueryAlias spark_catalog.default.t2\n +- View (`spark_catalog`.`default`.`t2`, [c1#x, c2#x])\n +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n +- LocalRelation [col1#x, col2#x]\n" + } +} + + +-- !query +SELECT * FROM t1 ORDER BY foo3_1b(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SQL_UDF_USAGE", + "sqlState" : "0A000", + "messageParameters" : { + "functionName" : "`spark_catalog`.`default`.`foo3_1b`", + "nodeName" : "Sort" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 37, + "fragment" : "foo3_1b(c1)" + } ] +} + + +-- !query +SELECT * FROM t1 LIMIT foo3_1b(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SQL_UDF_USAGE", + "sqlState" : "0A000", + "messageParameters" : { + "functionName" : "`spark_catalog`.`default`.`foo3_1b`", + "nodeName" : "GlobalLimit" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 33, + "fragment" : "foo3_1b(1)" + } ] +} + + +-- !query +SELECT * FROM ta LATERAL VIEW EXPLODE(ARRAY(foo3_1b(x[0]), foo3_1b(x[1]))) AS t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SQL_UDF_USAGE", + "sqlState" : "0A000", + "messageParameters" : { + "functionName" : "`spark_catalog`.`default`.`foo3_1b`", + "nodeName" : "Generate" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 57, + "fragment" : "foo3_1b(x[0])" + } ] +} + + +-- !query +SELECT CASE WHEN foo3_1b(rand(0) * 0 < 1 THEN 1 ELSE -1 END +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'foo3_1b'", + "hint" : "" + } +} + + +-- !query +SELECT (SELECT SUM(c2) FROM t2 WHERE c1 = foo3_1b(t1.c1)) FROM t1 +-- !query schema +struct +-- !query output +5 +5 +NULL + + +-- !query +SELECT foo3_1b((SELECT SUM(c1) FROM t1)) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT foo3_1a(c1, (SELECT MIN(c1) FROM t1)) FROM t1 +-- !query schema +struct +-- !query output +0.0 +0.0 +0.0 + + +-- !query +SELECT foo3_1b((SELECT SUM(c1))) FROM t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"sum(c1) AS `sum(outer(spark_catalog.default.t1.c1))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 17, + "stopIndex" : 30, + "fragment" : "SELECT SUM(c1)" + } ] +} + + +-- !query +SELECT foo3_1b((SELECT SUM(c1) FROM t1 WHERE c2 = t2.c2)) FROM t2 +-- !query schema +struct +-- !query output +1 +NULL + + +-- !query +SELECT c2, AVG(foo3_1b((SELECT COUNT(*) FROM t1 WHERE c2 = t2.c2))) OVER (PARTITION BY c1) AS r FROM t2 +-- !query schema +struct +-- !query output +2 1.0 +3 1.0 + + +-- !query +CREATE FUNCTION foo3_1x(x STRUCT) RETURNS INT RETURN x.a + x.b +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_1y(x ARRAY) RETURNS INT RETURN aggregate(x, BIGINT(0), (x, y) -> x + y) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo3_1a(x.a, x.b) FROM ts +-- !query schema +struct +-- !query output +2.0 + + +-- !query +SELECT foo3_1x(x) FROM ts +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT foo3_1a(x['a'], x['b']) FROM tm +-- !query schema +struct +-- !query output +2.0 + + +-- !query +SELECT foo3_1a(x[0], x[1]) FROM ta +-- !query schema +struct +-- !query output +2.0 + + +-- !query +SELECT foo3_1y(x) FROM ta +-- !query schema +struct +-- !query output +6 + + +-- !query +CREATE FUNCTION foo3_2a() RETURNS INT RETURN FLOOR(RAND() * 6) + 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT CASE WHEN foo3_2a() > 6 THEN FALSE ELSE TRUE END +-- !query schema +struct 6) THEN false ELSE true END:boolean> +-- !query output +true + + +-- !query +SELECT * FROM t1 JOIN t2 ON foo3_2a() = 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_NON_DETERMINISTIC_EXPRESSIONS", + "sqlState" : "42K0E", + "messageParameters" : { + "sqlExprs" : "\"(spark_catalog.default.foo3_2a() = 1)\"" + } +} + + +-- !query +CREATE FUNCTION foo3_2b1(x INT) RETURNS BOOLEAN RETURN x IN (SELECT 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM t1 WHERE foo3_2b1(c1) +-- !query schema +struct +-- !query output +1 2 + + +-- !query +CREATE FUNCTION foo3_2b2(x INT) RETURNS INT RETURN IF(x IN (SELECT 1), 1, 0) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM t1 WHERE foo3_2b2(c1) = 0 +-- !query schema +struct +-- !query output +0 1 +0 2 + + +-- !query +SELECT foo3_2b2(c1) FROM t1 +-- !query schema +struct +-- !query output +0 +0 +1 + + +-- !query +CREATE FUNCTION foo3_2b3(x INT) RETURNS BOOLEAN RETURN x IN (SELECT c1 FROM t2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM t1 WHERE foo3_2b3(c1) +-- !query schema +struct +-- !query output +0 1 +0 2 + + +-- !query +CREATE FUNCTION foo3_2b4(x INT) RETURNS BOOLEAN RETURN x NOT IN (SELECT c2 FROM t2 WHERE x = c1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM t1 WHERE foo3_2b4(c1) +-- !query schema +struct +-- !query output +0 1 +0 2 +1 2 + + +-- !query +CREATE FUNCTION foo3_2b5(x INT) RETURNS BOOLEAN RETURN SUM(1) + IF(x IN (SELECT 1), 1, 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "USER_DEFINED_FUNCTIONS.CANNOT_CONTAIN_COMPLEX_FUNCTIONS", + "sqlState" : "42601", + "messageParameters" : { + "queryText" : "SUM(1) + IF(x IN (SELECT 1), 1, 0)" + } +} + + +-- !query +CREATE FUNCTION foo3_2b5(x INT) RETURNS BOOLEAN RETURN y IN (SELECT 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`y`", + "proposal" : "`x`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 1, + "fragment" : "y" + } ] +} + + +-- !query +CREATE FUNCTION foo3_2b5(x INT) RETURNS BOOLEAN RETURN x IN (SELECT x WHERE x = 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_2c1(x INT) RETURNS BOOLEAN RETURN EXISTS(SELECT 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM t1 WHERE foo3_2c1(c1) +-- !query schema +struct +-- !query output +0 1 +0 2 +1 2 + + +-- !query +CREATE FUNCTION foo3_2c2(x INT) RETURNS BOOLEAN RETURN NOT EXISTS(SELECT * FROM t2 WHERE c1 = x) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM t1 WHERE foo3_2c2(c1) +-- !query schema +struct +-- !query output +1 2 + + +-- !query +CREATE FUNCTION foo3_2d1(x INT) RETURNS INT RETURN SELECT (SELECT x) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_2d2(x INT) RETURNS INT RETURN SELECT (SELECT 1 WHERE EXISTS (SELECT * FROM t2 WHERE c1 = x)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`x`", + "proposal" : "`c1`, `c2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 60, + "stopIndex" : 60, + "fragment" : "x" + } ] +} + + +-- !query +CREATE FUNCTION foo3_2e1( + occurrences ARRAY>, + instance_start_time TIMESTAMP +) RETURNS STRING RETURN +WITH t AS ( + SELECT transform(occurrences, x -> named_struct( + 'diff', abs(unix_millis(x.start_time) - unix_millis(instance_start_time)), + 'id', x.occurrence_id + )) AS diffs +) +SELECT CASE WHEN occurrences IS NULL OR size(occurrences) = 0 + THEN NULL + ELSE sort_array(diffs)[0].id END AS id +FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo3_2e1( + ARRAY(STRUCT('2022-01-01 10:11:12', '1'), STRUCT('2022-01-01 10:11:15', '2')), + '2022-01-01') +-- !query schema +struct +-- !query output +1 + + +-- !query +SET spark.sql.ansi.enabled=true +-- !query schema +struct +-- !query output +spark.sql.ansi.enabled true + + +-- !query +CREATE FUNCTION foo3_3a(x INT) RETURNS DOUBLE RETURN 1 / x +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_3at(x INT) RETURNS TABLE (a DOUBLE) RETURN SELECT 1 / x +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY FUNCTION foo3_3b(x INT) RETURNS DOUBLE RETURN 1 / x +-- !query schema +struct<> +-- !query output + + + +-- !query +SET spark.sql.ansi.enabled=false +-- !query schema +struct +-- !query output +spark.sql.ansi.enabled false + + +-- !query +SELECT foo3_3a(0) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 5, + "fragment" : "1 / x" + } ] +} + + +-- !query +SELECT foo3_3b(0) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 5, + "fragment" : "1 / x" + } ] +} + + +-- !query +SELECT * FROM foo3_3at(0) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 12, + "fragment" : "1 / x" + } ] +} + + +-- !query +CREATE OR REPLACE FUNCTION foo3_3a(x INT) RETURNS DOUBLE RETURN 1 / x +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE FUNCTION foo3_3at(x INT) RETURNS TABLE (a DOUBLE) RETURN SELECT 1 / x +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY FUNCTION foo3_3b(x INT) RETURNS DOUBLE RETURN 1 / x +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo3_3a(0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT foo3_3b(0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT * FROM foo3_3at(0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +CREATE FUNCTION foo3_3c() RETURNS INT RETURN CAST('a' AS INT) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_3ct() RETURNS TABLE (a INT) RETURN SELECT CAST('a' AS INT) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_3d() RETURNS INT RETURN 'a' + 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_3dt() RETURNS TABLE (a INT) RETURN SELECT 'a' + 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo3_3c() +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT foo3_3d() +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT * FROM foo3_3ct() +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT * FROM foo3_3dt() +-- !query schema +struct +-- !query output +NULL + + +-- !query +SET spark.sql.ansi.enabled=true +-- !query schema +struct +-- !query output +spark.sql.ansi.enabled true + + +-- !query +SELECT foo3_3c() +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT foo3_3d() +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT * FROM foo3_3ct() +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT * FROM foo3_3dt() +-- !query schema +struct +-- !query output +NULL + + +-- !query +RESET spark.sql.ansi.enabled +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_4a(x INT) RETURNS INT RETURN 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_4b(x INT) RETURNS INT RETURN foo3_4a(x) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_4c(x INT) RETURNS INT RETURN foo3_4b(x) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_4d(x INT) RETURNS INT RETURN (SELECT foo3_4c(x)) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_4e(x INT) RETURNS TABLE(a INT) RETURN SELECT foo3_4a(x) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_4f(x INT) RETURNS TABLE(b INT) RETURN SELECT * FROM foo3_4e(x) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMP FUNCTION foo3_4g(x INT) RETURN x + 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMP FUNCTION foo3_4h(x INT) RETURN foo3_4g(x) + 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE FUNCTION foo3_4a(x INT) RETURNS INT RETURN foo3_4b(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CYCLIC_FUNCTION_REFERENCE", + "sqlState" : "42887", + "messageParameters" : { + "path" : "spark_catalog.default.foo3_4a -> spark_catalog.default.foo3_4b -> spark_catalog.default.foo3_4a" + } +} + + +-- !query +CREATE OR REPLACE FUNCTION foo3_4a(x INT) RETURNS INT RETURN foo3_4c(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CYCLIC_FUNCTION_REFERENCE", + "sqlState" : "42887", + "messageParameters" : { + "path" : "spark_catalog.default.foo3_4a -> spark_catalog.default.foo3_4c -> spark_catalog.default.foo3_4b -> spark_catalog.default.foo3_4a" + } +} + + +-- !query +CREATE OR REPLACE FUNCTION foo3_4a(x INT) RETURNS INT RETURN foo3_4d(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CYCLIC_FUNCTION_REFERENCE", + "sqlState" : "42887", + "messageParameters" : { + "path" : "spark_catalog.default.foo3_4a -> spark_catalog.default.foo3_4d -> spark_catalog.default.foo3_4c -> spark_catalog.default.foo3_4b -> spark_catalog.default.foo3_4a" + } +} + + +-- !query +CREATE OR REPLACE FUNCTION foo3_4a(x INT) RETURN FoO3_4b(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CYCLIC_FUNCTION_REFERENCE", + "sqlState" : "42887", + "messageParameters" : { + "path" : "spark_catalog.default.foo3_4a -> spark_catalog.default.foo3_4b -> spark_catalog.default.foo3_4a" + } +} + + +-- !query +CREATE OR REPLACE FUNCTION foo3_4a(x INT) RETURNS INT RETURN SELECT SUM(a) FROM foo3_4e(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CYCLIC_FUNCTION_REFERENCE", + "sqlState" : "42887", + "messageParameters" : { + "path" : "spark_catalog.default.foo3_4a -> spark_catalog.default.foo3_4e -> spark_catalog.default.foo3_4a" + } +} + + +-- !query +CREATE OR REPLACE FUNCTION foo3_4e(x INT) RETURNS TABLE (c INT) RETURN SELECT * FROM foo3_4f(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CYCLIC_FUNCTION_REFERENCE", + "sqlState" : "42887", + "messageParameters" : { + "path" : "spark_catalog.default.foo3_4e -> spark_catalog.default.foo3_4f -> spark_catalog.default.foo3_4e" + } +} + + +-- !query +CREATE OR REPLACE FUNCTION foo3_4e(x INT) RETURNS TABLE RETURN SELECT * FROM fOo3_4F(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CYCLIC_FUNCTION_REFERENCE", + "sqlState" : "42887", + "messageParameters" : { + "path" : "spark_catalog.default.foo3_4e -> spark_catalog.default.foo3_4f -> spark_catalog.default.foo3_4e" + } +} + + +-- !query +CREATE OR REPLACE TEMP FUNCTION foo3_4g(x INT) RETURN foo3_4h(x) + 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CYCLIC_FUNCTION_REFERENCE", + "sqlState" : "42887", + "messageParameters" : { + "path" : "foo3_4g -> foo3_4h -> foo3_4g" + } +} + + +-- !query +CREATE TEMPORARY VIEW t AS VALUES (0) t(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY FUNCTION foo3_5a(x INT) RETURNS INT RETURN x +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY FUNCTION foo3_5b(x INT) RETURNS INT RETURN (SELECT SUM(a) FROM t) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY FUNCTION foo3_5c(x INT) RETURNS TABLE (a INT) RETURN SELECT a FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_5d(x INT) RETURNS INT RETURN foo3_5a(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "sqlState" : "42K0F", + "messageParameters" : { + "obj" : "FUNCTION", + "objName" : "`default`.`foo3_5d`", + "tempObj" : "FUNCTION", + "tempObjName" : "`foo3_5a`" + } +} + + +-- !query +CREATE FUNCTION foo3_5d(x INT) RETURNS TABLE (a INT) RETURN SELECT foo3_5a(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "sqlState" : "42K0F", + "messageParameters" : { + "obj" : "FUNCTION", + "objName" : "`default`.`foo3_5d`", + "tempObj" : "FUNCTION", + "tempObjName" : "`foo3_5a`" + } +} + + +-- !query +CREATE FUNCTION foo3_5d(x INT) RETURNS INT RETURN (SELECT SUM(a) FROM t) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "sqlState" : "42K0F", + "messageParameters" : { + "obj" : "FUNCTION", + "objName" : "`default`.`foo3_5d`", + "tempObj" : "VIEW", + "tempObjName" : "`t`" + } +} + + +-- !query +CREATE FUNCTION foo3_5d(x INT) RETURNS TABLE (a INT) RETURN SELECT a FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TEMP_OBJ_REFERENCE", + "sqlState" : "42K0F", + "messageParameters" : { + "obj" : "FUNCTION", + "objName" : "`default`.`foo3_5d`", + "tempObj" : "VIEW", + "tempObjName" : "`t`" + } +} + + +-- !query +CREATE FUNCTION foo3_14a() RETURNS INT RETURN 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo3_14b() RETURNS TABLE (a INT) RETURN SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM foo3_14a() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_TABLE_FUNCTION", + "sqlState" : "42887", + "messageParameters" : { + "functionName" : "`spark_catalog`.`default`.`foo3_14a`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 24, + "fragment" : "foo3_14a()" + } ] +} + + +-- !query +SELECT foo3_14b() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NOT_A_SCALAR_FUNCTION", + "sqlState" : "42887", + "messageParameters" : { + "functionName" : "`spark_catalog`.`default`.`foo3_14b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "foo3_14b()" + } ] +} + + +-- !query +CREATE FUNCTION foo4_0() RETURNS TABLE (x INT) RETURN SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo4_1(x INT) RETURNS TABLE (a INT) RETURN SELECT x +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo4_2(x INT) RETURNS TABLE (a INT) RETURN SELECT c2 FROM t2 WHERE c1 = x +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo4_3(x INT) RETURNS TABLE (a INT, cnt INT) RETURN SELECT c1, COUNT(*) FROM t2 WHERE c1 = x GROUP BY c1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM foo4_0() +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM foo4_1(1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM foo4_2(2) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM foo4_3(0) +-- !query schema +struct +-- !query output +0 2 + + +-- !query +SELECT * FROM foo4_1(rand(0) * 0) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT * FROM foo4_1(x => 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM t1, LATERAL foo4_1(c1) +-- !query schema +struct +-- !query output +0 1 0 +0 2 0 +1 2 1 + + +-- !query +SELECT * FROM t1, LATERAL foo4_2(c1) +-- !query schema +struct +-- !query output +0 1 2 +0 1 3 +0 2 2 +0 2 3 + + +-- !query +SELECT * FROM t1 JOIN LATERAL foo4_2(c1) ON t1.c2 = foo4_2.a +-- !query schema +struct +-- !query output +0 2 2 + + +-- !query +SELECT * FROM t1, LATERAL foo4_3(c1) +-- !query schema +struct +-- !query output +0 1 0 2 +0 2 0 2 + + +-- !query +SELECT * FROM t1, LATERAL (SELECT cnt FROM foo4_3(c1)) +-- !query schema +struct +-- !query output +0 1 2 +0 2 2 + + +-- !query +SELECT * FROM t1, LATERAL foo4_1(c1 + rand(0) * 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.NON_DETERMINISTIC_LATERAL_SUBQUERIES", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "LateralJoin lateral-subquery#x [c1#x], Inner\n: +- SQLFunctionNode spark_catalog.default.foo4_1\n: +- SubqueryAlias foo4_1\n: +- Project [cast(x#x as int) AS a#x]\n: +- LateralJoin lateral-subquery#x [x#x], Inner\n: : +- Project [outer(x#x) AS x#x]\n: : +- OneRowRelation\n: +- Project [cast((cast(outer(c1#x) as double) + (rand(number) * cast(0 as double))) as int) AS x#x]\n: +- OneRowRelation\n+- SubqueryAlias spark_catalog.default.t1\n +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x])\n +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n +- LocalRelation [col1#x, col2#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 10, + "stopIndex" : 50, + "fragment" : "FROM t1, LATERAL foo4_1(c1 + rand(0) * 0)" + } ] +} + + +-- !query +SELECT * FROM t1 JOIN foo4_1(1) AS foo4_1(x) ON t1.c1 = foo4_1.x +-- !query schema +struct +-- !query output +1 2 1 + + +-- !query +SELECT * FROM t1, LATERAL foo4_1(c1), LATERAL foo4_2(foo4_1.a + c1) +-- !query schema +struct +-- !query output +0 1 0 2 +0 1 0 3 +0 2 0 2 +0 2 0 3 + + +-- !query +SELECT (SELECT MAX(a) FROM foo4_1(c1)) FROM t1 +-- !query schema +struct +-- !query output +0 +0 +1 + + +-- !query +SELECT (SELECT MAX(a) FROM foo4_1(c1) WHERE a = c2) FROM t1 +-- !query schema +struct +-- !query output +NULL +NULL +NULL + + +-- !query +SELECT (SELECT MAX(cnt) FROM foo4_3(c1)) FROM t1 +-- !query schema +struct +-- !query output +2 +2 +NULL + + +-- !query +DROP VIEW t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1a0 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1a1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1a2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1b0 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1b1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1b2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1c1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1c2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1d1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1d2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1d4 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1d5 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1d6 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1e1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1e2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1e3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1f1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1f2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1g1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1g2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2a0 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2a2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2a4 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2b1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2b2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2c1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo31 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo32 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo33 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo41 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo42 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo51 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo52 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo6c +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo6d +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo7a +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo7at +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo9a +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo9b +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo9c +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo9d +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo9e +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo9f +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo9g +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo9h +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo9i +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo9j +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo9l +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo9m +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo9n +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo9o +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo9p +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo9q +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo9r +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1_10 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS bar1_10 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1_11a +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1_11b +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1_11c +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1_11d +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1_11e +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1_11f +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1_11g +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1_11h +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1_11i +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1_11j +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1_11k +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1_11l +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1_11m +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1_11n +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo1_11o +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2_1a +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2_1b +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2_1c +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2_1d +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2_2a +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2_2b +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2_2c +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2_2d +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2_2e +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2_2f +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2_2g +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2_3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2_4a +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo2_4b +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_1a +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_1b +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_1c +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_1d +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_1e +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_1f +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_1g +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_1x +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_1y +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_2a +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_2b1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_2b2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_2b3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_2b4 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_2b5 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_2c1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_2c2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_2d1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_2d2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_2e1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_3a +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_3at +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_14a +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_14b +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_3c +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_3ct +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_3d +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_3dt +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_4a +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_4b +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_4c +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_4d +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_4e +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo3_4f +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo4_0 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo4_1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo4_2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION IF EXISTS foo4_3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY FUNCTION IF EXISTS foo1d3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY FUNCTION IF EXISTS foo3_3b +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY FUNCTION IF EXISTS foo3_5a +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY FUNCTION IF EXISTS foo3_5b +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TEMPORARY FUNCTION IF EXISTS foo3_5c +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/string-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/string-functions.sql.out new file mode 100644 index 000000000000..23c59f3111a1 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/string-functions.sql.out @@ -0,0 +1,2347 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select concat_ws() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 0", + "functionName" : "`concat_ws`" + } +} + + +-- !query +select format_string() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 0", + "functionName" : "`format_string`" + } +} + + +-- !query +select 'a' || 'b' || 'c' +-- !query schema +struct +-- !query output +abc + + +-- !query +select replace('abc', 'b', '123') +-- !query schema +struct +-- !query output +a123c + + +-- !query +select replace('abc', 'b') +-- !query schema +struct +-- !query output +ac + + +-- !query +select length(uuid()), (uuid() <> uuid()) +-- !query schema +struct +-- !query output +36 true + + +-- !query +select position('bar' in 'foobarbar'), position(null, 'foobarbar'), position('aaads', null) +-- !query schema +struct +-- !query output +4 NULL NULL + + +-- !query +select left("abcd", 2), left("abcd", 5), left("abcd", '2'), left("abcd", null) +-- !query schema +struct +-- !query output +ab abcd ab NULL + + +-- !query +select left(null, -2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select left("abcd", -2), left("abcd", 0), left("abcd", 'a') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 43, + "stopIndex" : 59, + "fragment" : "left(\"abcd\", 'a')" + } ] +} + + +-- !query +select right("abcd", 2), right("abcd", 5), right("abcd", '2'), right("abcd", null) +-- !query schema +struct +-- !query output +cd abcd cd NULL + + +-- !query +select right(null, -2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select right("abcd", -2), right("abcd", 0), right("abcd", 'a') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 62, + "fragment" : "right(\"abcd\", 'a')" + } ] +} + + +-- !query +SELECT split('aa1cc2ee3', '[1-9]+') +-- !query schema +struct> +-- !query output +["aa","cc","ee",""] + + +-- !query +SELECT split('aa1cc2ee3', '[1-9]+', 2) +-- !query schema +struct> +-- !query output +["aa","cc2ee3"] + + +-- !query +SELECT split('hello', '') +-- !query schema +struct> +-- !query output +["h","e","l","l","o"] + + +-- !query +SELECT split('', '') +-- !query schema +struct> +-- !query output +[""] + + +-- !query +SELECT split('abc', null) +-- !query schema +struct> +-- !query output +NULL + + +-- !query +SELECT split(null, 'b') +-- !query schema +struct> +-- !query output +NULL + + +-- !query +SELECT split_part('11.12.13', '.', 2) +-- !query schema +struct +-- !query output +12 + + +-- !query +SELECT split_part('11.12.13', '.', -1) +-- !query schema +struct +-- !query output +13 + + +-- !query +SELECT split_part('11.12.13', '.', -3) +-- !query schema +struct +-- !query output +11 + + +-- !query +SELECT split_part('11.12.13', '', 1) +-- !query schema +struct +-- !query output +11.12.13 + + +-- !query +SELECT split_part('11ab12ab13', 'ab', 1) +-- !query schema +struct +-- !query output +11 + + +-- !query +SELECT split_part('11.12.13', '.', 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_INDEX_OF_ZERO", + "sqlState" : "22003" +} + + +-- !query +SELECT split_part('11.12.13', '.', 4) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT split_part('11.12.13', '.', 5) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT split_part('11.12.13', '.', -5) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT split_part(null, '.', 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT split_part(str, delimiter, partNum) FROM VALUES ('11.12.13', '.', 3) AS v1(str, delimiter, partNum) +-- !query schema +struct +-- !query output +13 + + +-- !query +SELECT substr('Spark SQL', 5) +-- !query schema +struct +-- !query output +k SQL + + +-- !query +SELECT substr('Spark SQL', -3) +-- !query schema +struct +-- !query output +SQL + + +-- !query +SELECT substr('Spark SQL', 5, 1) +-- !query schema +struct +-- !query output +k + + +-- !query +SELECT substr('Spark SQL' from 5) +-- !query schema +struct +-- !query output +k SQL + + +-- !query +SELECT substr('Spark SQL' from -3) +-- !query schema +struct +-- !query output +SQL + + +-- !query +SELECT substr('Spark SQL' from 5 for 1) +-- !query schema +struct +-- !query output +k + + +-- !query +SELECT substring('Spark SQL', 5) +-- !query schema +struct +-- !query output +k SQL + + +-- !query +SELECT substring('Spark SQL', -3) +-- !query schema +struct +-- !query output +SQL + + +-- !query +SELECT substring('Spark SQL', 5, 1) +-- !query schema +struct +-- !query output +k + + +-- !query +SELECT substring('Spark SQL' from 5) +-- !query schema +struct +-- !query output +k SQL + + +-- !query +SELECT substring('Spark SQL' from -3) +-- !query schema +struct +-- !query output +SQL + + +-- !query +SELECT substring('Spark SQL' from 5 for 1) +-- !query schema +struct +-- !query output +k + + +-- !query +SELECT trim(" xyz "), ltrim(" xyz "), rtrim(" xyz ") +-- !query schema +struct +-- !query output +xyz xyz xyz + + +-- !query +SELECT trim(BOTH 'xyz' FROM 'yxTomxx'), trim('xyz' FROM 'yxTomxx') +-- !query schema +struct +-- !query output +Tom Tom + + +-- !query +SELECT trim(BOTH 'x' FROM 'xxxbarxxx'), trim('x' FROM 'xxxbarxxx') +-- !query schema +struct +-- !query output +bar bar + + +-- !query +SELECT trim(LEADING 'xyz' FROM 'zzzytest') +-- !query schema +struct +-- !query output +test + + +-- !query +SELECT trim(LEADING 'xyz' FROM 'zzzytestxyz') +-- !query schema +struct +-- !query output +testxyz + + +-- !query +SELECT trim(LEADING 'xy' FROM 'xyxXxyLAST WORD') +-- !query schema +struct +-- !query output +XxyLAST WORD + + +-- !query +SELECT trim(TRAILING 'xyz' FROM 'testxxzx') +-- !query schema +struct +-- !query output +test + + +-- !query +SELECT trim(TRAILING 'xyz' FROM 'xyztestxxzx') +-- !query schema +struct +-- !query output +xyztest + + +-- !query +SELECT trim(TRAILING 'xy' FROM 'TURNERyxXxy') +-- !query schema +struct +-- !query output +TURNERyxX + + +-- !query +SELECT btrim('xyxtrimyyx', 'xy') +-- !query schema +struct +-- !query output +trim + + +-- !query +SELECT btrim(encode(" xyz ", 'utf-8')) +-- !query schema +struct +-- !query output +xyz + + +-- !query +SELECT btrim(encode('yxTomxx', 'utf-8'), encode('xyz', 'utf-8')) +-- !query schema +struct +-- !query output +Tom + + +-- !query +SELECT btrim(encode('xxxbarxxx', 'utf-8'), encode('x', 'utf-8')) +-- !query schema +struct +-- !query output +bar + + +-- !query +SELECT lpad('hi', 'invalid_length') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'invalid_length'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "lpad('hi', 'invalid_length')" + } ] +} + + +-- !query +SELECT rpad('hi', 'invalid_length') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'invalid_length'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "rpad('hi', 'invalid_length')" + } ] +} + + +-- !query +SELECT hex(lpad(unhex(''), 5)) +-- !query schema +struct +-- !query output +0000000000 + + +-- !query +SELECT hex(lpad(unhex('aabb'), 5)) +-- !query schema +struct +-- !query output +000000AABB + + +-- !query +SELECT hex(lpad(unhex('aabbcc'), 2)) +-- !query schema +struct +-- !query output +AABB + + +-- !query +SELECT hex(lpad(unhex('123'), 2)) +-- !query schema +struct +-- !query output +0123 + + +-- !query +SELECT hex(lpad(unhex('12345'), 2)) +-- !query schema +struct +-- !query output +0123 + + +-- !query +SELECT hex(lpad(unhex(''), 5, unhex('1f'))) +-- !query schema +struct +-- !query output +1F1F1F1F1F + + +-- !query +SELECT hex(lpad(unhex('aa'), 5, unhex('1f'))) +-- !query schema +struct +-- !query output +1F1F1F1FAA + + +-- !query +SELECT hex(lpad(unhex('aa'), 6, unhex('1f'))) +-- !query schema +struct +-- !query output +1F1F1F1F1FAA + + +-- !query +SELECT hex(lpad(unhex(''), 5, unhex('1f2e'))) +-- !query schema +struct +-- !query output +1F2E1F2E1F + + +-- !query +SELECT hex(lpad(unhex('aa'), 5, unhex('1f2e'))) +-- !query schema +struct +-- !query output +1F2E1F2EAA + + +-- !query +SELECT hex(lpad(unhex('aa'), 6, unhex('1f2e'))) +-- !query schema +struct +-- !query output +1F2E1F2E1FAA + + +-- !query +SELECT hex(lpad(unhex(''), 6, unhex(''))) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT hex(lpad(unhex('aabbcc'), 6, unhex(''))) +-- !query schema +struct +-- !query output +AABBCC + + +-- !query +SELECT hex(lpad(unhex('aabbcc'), 2, unhex('ff'))) +-- !query schema +struct +-- !query output +AABB + + +-- !query +SELECT hex(rpad(unhex(''), 5)) +-- !query schema +struct +-- !query output +0000000000 + + +-- !query +SELECT hex(rpad(unhex('aabb'), 5)) +-- !query schema +struct +-- !query output +AABB000000 + + +-- !query +SELECT hex(rpad(unhex('aabbcc'), 2)) +-- !query schema +struct +-- !query output +AABB + + +-- !query +SELECT hex(rpad(unhex('123'), 2)) +-- !query schema +struct +-- !query output +0123 + + +-- !query +SELECT hex(rpad(unhex('12345'), 2)) +-- !query schema +struct +-- !query output +0123 + + +-- !query +SELECT hex(rpad(unhex(''), 5, unhex('1f'))) +-- !query schema +struct +-- !query output +1F1F1F1F1F + + +-- !query +SELECT hex(rpad(unhex('aa'), 5, unhex('1f'))) +-- !query schema +struct +-- !query output +AA1F1F1F1F + + +-- !query +SELECT hex(rpad(unhex('aa'), 6, unhex('1f'))) +-- !query schema +struct +-- !query output +AA1F1F1F1F1F + + +-- !query +SELECT hex(rpad(unhex(''), 5, unhex('1f2e'))) +-- !query schema +struct +-- !query output +1F2E1F2E1F + + +-- !query +SELECT hex(rpad(unhex('aa'), 5, unhex('1f2e'))) +-- !query schema +struct +-- !query output +AA1F2E1F2E + + +-- !query +SELECT hex(rpad(unhex('aa'), 6, unhex('1f2e'))) +-- !query schema +struct +-- !query output +AA1F2E1F2E1F + + +-- !query +SELECT hex(rpad(unhex(''), 6, unhex(''))) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT hex(rpad(unhex('aabbcc'), 6, unhex(''))) +-- !query schema +struct +-- !query output +AABBCC + + +-- !query +SELECT hex(rpad(unhex('aabbcc'), 2, unhex('ff'))) +-- !query schema +struct +-- !query output +AABB + + +-- !query +SELECT lpad('abc', 5, x'57') +-- !query schema +struct +-- !query output +WWabc + + +-- !query +SELECT lpad(x'57', 5, 'abc') +-- !query schema +struct +-- !query output +abcaW + + +-- !query +SELECT rpad('abc', 5, x'57') +-- !query schema +struct +-- !query output +abcWW + + +-- !query +SELECT rpad(x'57', 5, 'abc') +-- !query schema +struct +-- !query output +Wabca + + +-- !query +set spark.sql.legacy.javaCharsets=true +-- !query schema +struct +-- !query output +spark.sql.legacy.javaCharsets true + + +-- !query +select encode('hello', 'WINDOWS-1252') +-- !query schema +struct +-- !query output +hello + + +-- !query +select encode(scol, ecol) from values('hello', 'WINDOWS-1252') as t(scol, ecol) +-- !query schema +struct +-- !query output +hello + + +-- !query +set spark.sql.legacy.javaCharsets=false +-- !query schema +struct +-- !query output +spark.sql.legacy.javaCharsets false + + +-- !query +select encode('hello', 'WINDOWS-1252') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.CHARSET", + "sqlState" : "22023", + "messageParameters" : { + "charset" : "WINDOWS-1252", + "charsets" : "iso-8859-1, us-ascii, utf-16, utf-16be, utf-16le, utf-32, utf-8", + "functionName" : "`encode`", + "parameter" : "`charset`" + } +} + + +-- !query +select encode(scol, ecol) from values('hello', 'WINDOWS-1252') as t(scol, ecol) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.CHARSET", + "sqlState" : "22023", + "messageParameters" : { + "charset" : "WINDOWS-1252", + "charsets" : "iso-8859-1, us-ascii, utf-16, utf-16be, utf-16le, utf-32, utf-8", + "functionName" : "`encode`", + "parameter" : "`charset`" + } +} + + +-- !query +select encode('hello', 'Windows-xxx') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.CHARSET", + "sqlState" : "22023", + "messageParameters" : { + "charset" : "Windows-xxx", + "charsets" : "iso-8859-1, us-ascii, utf-16, utf-16be, utf-16le, utf-32, utf-8", + "functionName" : "`encode`", + "parameter" : "`charset`" + } +} + + +-- !query +select encode(scol, ecol) from values('hello', 'Windows-xxx') as t(scol, ecol) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.CHARSET", + "sqlState" : "22023", + "messageParameters" : { + "charset" : "Windows-xxx", + "charsets" : "iso-8859-1, us-ascii, utf-16, utf-16be, utf-16le, utf-32, utf-8", + "functionName" : "`encode`", + "parameter" : "`charset`" + } +} + + +-- !query +set spark.sql.legacy.codingErrorAction=true +-- !query schema +struct +-- !query output +spark.sql.legacy.codingErrorAction true + + +-- !query +select encode('渭城朝雨浥轻尘', 'US-ASCII') +-- !query schema +struct +-- !query output +??????? + + +-- !query +select encode(scol, ecol) from values('渭城朝雨浥轻尘', 'US-ASCII') as t(scol, ecol) +-- !query schema +struct +-- !query output +??????? + + +-- !query +set spark.sql.legacy.codingErrorAction=false +-- !query schema +struct +-- !query output +spark.sql.legacy.codingErrorAction false + + +-- !query +select encode('客舍青青柳色新', 'US-ASCII') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "MALFORMED_CHARACTER_CODING", + "sqlState" : "22000", + "messageParameters" : { + "charset" : "US-ASCII", + "function" : "`encode`" + } +} + + +-- !query +select encode(scol, ecol) from values('客舍青青柳色新', 'US-ASCII') as t(scol, ecol) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "MALFORMED_CHARACTER_CODING", + "sqlState" : "22000", + "messageParameters" : { + "charset" : "US-ASCII", + "function" : "`encode`" + } +} + + +-- !query +select encode(decode(encode('白日依山尽,黄河入海流。欲穷千里目,更上一层楼。', 'UTF-16'), 'UTF-16'), 'UTF-8') +-- !query schema +struct +-- !query output +白日依山尽,黄河入海流。欲穷千里目,更上一层楼。 + + +-- !query +select encode(decode(encode('南山經之首曰䧿山。其首曰招搖之山,臨於西海之上。', 'UTF-16'), 'UTF-16'), 'UTF-8') +-- !query schema +struct +-- !query output +南山經之首曰䧿山。其首曰招搖之山,臨於西海之上。 + + +-- !query +select encode(decode(encode('세계에서 가장 인기 있는 빅데이터 처리 프레임워크인 Spark', 'UTF-16'), 'UTF-16'), 'UTF-8') +-- !query schema +struct +-- !query output +세계에서 가장 인기 있는 빅데이터 처리 프레임워크인 Spark + + +-- !query +select encode(decode(encode('το Spark είναι το πιο δημοφιλές πλαίσιο επεξεργασίας μεγάλων δεδομένων παγκοσμίως', 'UTF-16'), 'UTF-16'), 'UTF-8') +-- !query schema +struct +-- !query output +το Spark είναι το πιο δημοφιλές πλαίσιο επεξεργασίας μεγάλων δεδομένων παγκοσμίως + + +-- !query +select encode(decode(encode('Sparkは世界で最も人気のあるビッグデータ処理フレームワークである。', 'UTF-16'), 'UTF-16'), 'UTF-8') +-- !query schema +struct +-- !query output +Sparkは世界で最も人気のあるビッグデータ処理フレームワークである。 + + +-- !query +select decode() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "2", + "functionName" : "`decode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "decode()" + } ] +} + + +-- !query +select decode(encode('abc', 'utf-8')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "2", + "functionName" : "`decode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "decode(encode('abc', 'utf-8'))" + } ] +} + + +-- !query +select decode(encode('abc', 'utf-8'), 'utf-8') +-- !query schema +struct +-- !query output +abc + + +-- !query +select decode(encode('大千世界', 'utf-32'), 'utf-32') +-- !query schema +struct +-- !query output +大千世界 + + +-- !query +select decode(1, 1, 'Southlake') +-- !query schema +struct +-- !query output +Southlake + + +-- !query +select decode(2, 1, 'Southlake') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select decode(2, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle', 'Non domestic') +-- !query schema +struct +-- !query output +San Francisco + + +-- !query +select decode(6, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle', 'Non domestic') +-- !query schema +struct +-- !query output +Non domestic + + +-- !query +select decode(6, 1, 'Southlake', 2, 'San Francisco', 3, 'New Jersey', 4, 'Seattle') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select decode(null, 6, 'Spark', NULL, 'SQL', 4, 'rocks') +-- !query schema +struct +-- !query output +SQL + + +-- !query +select decode(null, 6, 'Spark', NULL, 'SQL', 4, 'rocks', NULL, '.') +-- !query schema +struct +-- !query output +SQL + + +-- !query +select decode(X'68656c6c6f', 'Windows-xxx') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.CHARSET", + "sqlState" : "22023", + "messageParameters" : { + "charset" : "Windows-xxx", + "charsets" : "iso-8859-1, us-ascii, utf-16, utf-16be, utf-16le, utf-32, utf-8", + "functionName" : "`decode`", + "parameter" : "`charset`" + } +} + + +-- !query +select decode(scol, ecol) from values(X'68656c6c6f', 'Windows-xxx') as t(scol, ecol) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.CHARSET", + "sqlState" : "22023", + "messageParameters" : { + "charset" : "Windows-xxx", + "charsets" : "iso-8859-1, us-ascii, utf-16, utf-16be, utf-16le, utf-32, utf-8", + "functionName" : "`decode`", + "parameter" : "`charset`" + } +} + + +-- !query +set spark.sql.legacy.javaCharsets=true +-- !query schema +struct +-- !query output +spark.sql.legacy.javaCharsets true + + +-- !query +select decode(X'68656c6c6f', 'WINDOWS-1252') +-- !query schema +struct +-- !query output +hello + + +-- !query +select decode(scol, ecol) from values(X'68656c6c6f', 'WINDOWS-1252') as t(scol, ecol) +-- !query schema +struct +-- !query output +hello + + +-- !query +set spark.sql.legacy.javaCharsets=false +-- !query schema +struct +-- !query output +spark.sql.legacy.javaCharsets false + + +-- !query +select decode(X'68656c6c6f', 'WINDOWS-1252') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.CHARSET", + "sqlState" : "22023", + "messageParameters" : { + "charset" : "WINDOWS-1252", + "charsets" : "iso-8859-1, us-ascii, utf-16, utf-16be, utf-16le, utf-32, utf-8", + "functionName" : "`decode`", + "parameter" : "`charset`" + } +} + + +-- !query +select decode(scol, ecol) from values(X'68656c6c6f', 'WINDOWS-1252') as t(scol, ecol) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.CHARSET", + "sqlState" : "22023", + "messageParameters" : { + "charset" : "WINDOWS-1252", + "charsets" : "iso-8859-1, us-ascii, utf-16, utf-16be, utf-16le, utf-32, utf-8", + "functionName" : "`decode`", + "parameter" : "`charset`" + } +} + + +-- !query +set spark.sql.legacy.codingErrorAction=true +-- !query schema +struct +-- !query output +spark.sql.legacy.codingErrorAction true + + +-- !query +select decode(X'E58A9DE5909BE69BB4E5B0BDE4B880E69DAFE98592', 'US-ASCII') +-- !query schema +struct +-- !query output +��������������������� + + +-- !query +select decode(scol, ecol) from values(X'E58A9DE5909BE69BB4E5B0BDE4B880E69DAFE98592', 'US-ASCII') as t(scol, ecol) +-- !query schema +struct +-- !query output +��������������������� + + +-- !query +set spark.sql.legacy.codingErrorAction=false +-- !query schema +struct +-- !query output +spark.sql.legacy.codingErrorAction false + + +-- !query +select decode(X'E8A5BFE587BAE998B3E585B3E697A0E69585E4BABA', 'US-ASCII') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "MALFORMED_CHARACTER_CODING", + "sqlState" : "22000", + "messageParameters" : { + "charset" : "US-ASCII", + "function" : "`decode`" + } +} + + +-- !query +select decode(scol, ecol) from values(X'E8A5BFE587BAE998B3E585B3E697A0E69585E4BABA', 'US-ASCII') as t(scol, ecol) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "MALFORMED_CHARACTER_CODING", + "sqlState" : "22000", + "messageParameters" : { + "charset" : "US-ASCII", + "function" : "`decode`" + } +} + + +-- !query +SELECT CONTAINS(null, 'Spark') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CONTAINS('Spark SQL', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CONTAINS(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT CONTAINS('Spark SQL', 'Spark') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT CONTAINS('Spark SQL', 'SQL') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT CONTAINS('Spark SQL', 'SPARK') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT startswith('Spark SQL', 'ark') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT startswith('Spark SQL', 'Spa') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT startswith(null, 'Spark') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT startswith('Spark', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT startswith(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT endswith('Spark SQL', 'QL') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT endswith('Spark SQL', 'Spa') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT endswith(null, 'Spark') +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT endswith('Spark', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT endswith(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT contains(x'537061726b2053514c', x'537061726b') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT contains(x'', x'') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT contains(x'537061726b2053514c', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT contains(12, '1') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT contains(true, 'ru') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT contains(x'12', 12) +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT contains(true, false) +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT startswith(x'537061726b2053514c', x'537061726b') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT startswith(x'537061726b2053514c', x'') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT startswith(x'', x'') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT startswith(x'537061726b2053514c', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT endswith(x'537061726b2053514c', x'53516c') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT endsWith(x'537061726b2053514c', x'537061726b') +-- !query schema +struct +-- !query output +false + + +-- !query +SELECT endsWith(x'537061726b2053514c', x'') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT endsWith(x'', x'') +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT endsWith(x'537061726b2053514c', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_number('454', '000') +-- !query schema +struct +-- !query output +454 + + +-- !query +select to_number('454.2', '000.0') +-- !query schema +struct +-- !query output +454.2 + + +-- !query +select to_number('12,454', '00,000') +-- !query schema +struct +-- !query output +12454 + + +-- !query +select to_number('$78.12', '$00.00') +-- !query schema +struct +-- !query output +78.12 + + +-- !query +select to_number('+454', 'S000') +-- !query schema +struct +-- !query output +454 + + +-- !query +select to_number('-454', 'S000') +-- !query schema +struct +-- !query output +-454 + + +-- !query +select to_number('12,454.8-', '00,000.9MI') +-- !query schema +struct +-- !query output +-12454.8 + + +-- !query +select to_number('00,454.8-', '00,000.9MI') +-- !query schema +struct +-- !query output +-454.8 + + +-- !query +select to_number('<00,454.8>', '00,000.9PR') +-- !query schema +struct, 00,000.9PR):decimal(6,1)> +-- !query output +-454.8 + + +-- !query +select to_binary('', 'base64') +-- !query schema +struct +-- !query output + + + +-- !query +select to_binary(' ', 'base64') +-- !query schema +struct +-- !query output + + + +-- !query +select to_binary(' ab cd ', 'base64') +-- !query schema +struct +-- !query output +i� + + +-- !query +select to_binary(' ab c=', 'base64') +-- !query schema +struct +-- !query output +i� + + +-- !query +select to_binary(' ab cdef= = ', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "' ab cdef= = '", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary( + concat(' b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0', + 'd2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl'), 'base64') +-- !query schema +struct +-- !query output +one two three four five six seven eight nine ten eleven twelve thirteen fourteen fivteen sixteen seventeen eightee + + +-- !query +select to_binary('a', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "'a'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('a?', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "'a?'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('abcde', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "'abcde'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('abcd=', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "'abcd='", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('a===', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "'a==='", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('ab==f', 'base64') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "'ab==f'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary( + '∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β)', 'utf-8') +-- !query schema +struct +-- !query output +∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β) + + +-- !query +select to_binary('大千世界', 'utf8') +-- !query schema +struct +-- !query output +大千世界 + + +-- !query +select to_binary('', 'utf-8') +-- !query schema +struct +-- !query output + + + +-- !query +select to_binary(' ', 'utf8') +-- !query schema +struct +-- !query output + + + +-- !query +select to_binary('737472696E67') +-- !query schema +struct +-- !query output +string + + +-- !query +select to_binary('737472696E67', 'hex') +-- !query schema +struct +-- !query output +string + + +-- !query +select to_binary('') +-- !query schema +struct +-- !query output + + + +-- !query +select to_binary('1', 'hex') +-- !query schema +struct +-- !query output + + + +-- !query +select to_binary('FF') +-- !query schema +struct +-- !query output +� + + +-- !query +select to_binary('123', 'hex') +-- !query schema +struct +-- !query output +# + + +-- !query +select to_binary('12345', 'hex') +-- !query schema +struct +-- !query output +#E + + +-- !query +select to_binary('GG') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'HEX'", + "str" : "'GG'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('01 AF', 'hex') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'HEX'", + "str" : "'01 AF'", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('abc', concat('utf', '-8')) +-- !query schema +struct +-- !query output +abc + + +-- !query +select to_binary(' ab cdef= = ', substr('base64whynot', 0, 6)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'BASE64'", + "str" : "' ab cdef= = '", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary(' ab cdef= = ', replace('HEX0', '0')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CONVERSION_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "fmt" : "'HEX'", + "str" : "' ab cdef= = '", + "suggestion" : "`try_to_binary`", + "targetType" : "\"BINARY\"" + } +} + + +-- !query +select to_binary('abc', 'Hex') +-- !query schema +struct +-- !query output + +� + + +-- !query +select to_binary('abc', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_binary(null, 'utf-8') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_binary(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_binary(null, cast(null as string)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_binary('abc', 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE", + "sqlState" : "42K09", + "messageParameters" : { + "inputName" : "fmt", + "inputValue" : "'1'", + "requireType" : "case-insensitive \"STRING\"", + "sqlExpr" : "\"to_binary(abc, 1)\"", + "validValues" : "'hex', 'utf-8', 'utf8', or 'base64'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "to_binary('abc', 1)" + } ] +} + + +-- !query +select to_binary('abc', 'invalidFormat') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE", + "sqlState" : "42K09", + "messageParameters" : { + "inputName" : "fmt", + "inputValue" : "'invalidformat'", + "requireType" : "case-insensitive \"STRING\"", + "sqlExpr" : "\"to_binary(abc, invalidFormat)\"", + "validValues" : "'hex', 'utf-8', 'utf8', or 'base64'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "to_binary('abc', 'invalidFormat')" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW fmtTable(fmtField) AS SELECT * FROM VALUES ('invalidFormat') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT to_binary('abc', fmtField) FROM fmtTable +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"fmtField\"", + "inputName" : "`fmt`", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"to_binary(abc, fmtField)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "to_binary('abc', fmtField)" + } ] +} + + +-- !query +DROP VIEW IF EXISTS fmtTable +-- !query schema +struct<> +-- !query output + + + +-- !query +select luhn_check('4111111111111111') +-- !query schema +struct +-- !query output +true + + +-- !query +select luhn_check('5500000000000004') +-- !query schema +struct +-- !query output +true + + +-- !query +select luhn_check('340000000000009') +-- !query schema +struct +-- !query output +true + + +-- !query +select luhn_check('6011000000000004') +-- !query schema +struct +-- !query output +true + + +-- !query +select luhn_check('6011000000000005') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check('378282246310006') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check('0') +-- !query schema +struct +-- !query output +true + + +-- !query +select luhn_check('4111111111111111 ') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check('4111111 111111111') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check(' 4111111111111111') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check('') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check(' ') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check('510B105105105106') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check('ABCDED') +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select luhn_check(6011111111111117) +-- !query schema +struct +-- !query output +true + + +-- !query +select luhn_check(6011111111111118) +-- !query schema +struct +-- !query output +false + + +-- !query +select luhn_check(123.456) +-- !query schema +struct +-- !query output +false + + +-- !query +select is_valid_utf8('') +-- !query schema +struct +-- !query output +true + + +-- !query +select is_valid_utf8('abc') +-- !query schema +struct +-- !query output +true + + +-- !query +select is_valid_utf8(x'80') +-- !query schema +struct +-- !query output +false + + +-- !query +select make_valid_utf8('') +-- !query schema +struct +-- !query output + + + +-- !query +select make_valid_utf8('abc') +-- !query schema +struct +-- !query output +abc + + +-- !query +select make_valid_utf8(x'80') +-- !query schema +struct +-- !query output +� + + +-- !query +select validate_utf8('') +-- !query schema +struct +-- !query output + + + +-- !query +select validate_utf8('abc') +-- !query schema +struct +-- !query output +abc + + +-- !query +select validate_utf8(x'80') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_UTF8_STRING", + "sqlState" : "22029", + "messageParameters" : { + "str" : "\\x80" + } +} + + +-- !query +select try_validate_utf8('') +-- !query schema +struct +-- !query output + + + +-- !query +select try_validate_utf8('abc') +-- !query schema +struct +-- !query output +abc + + +-- !query +select try_validate_utf8(x'80') +-- !query schema +struct +-- !query output +NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/struct.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/struct.sql.out new file mode 100644 index 000000000000..d642321e218d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/struct.sql.out @@ -0,0 +1,87 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tbl_x AS VALUES + (1, NAMED_STRUCT('C', 'gamma', 'D', 'delta')), + (2, NAMED_STRUCT('C', 'epsilon', 'D', 'eta')), + (3, NAMED_STRUCT('C', 'theta', 'D', 'iota')) + AS T(ID, ST) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT STRUCT('alpha', 'beta') ST +-- !query schema +struct> +-- !query output +{"col1":"alpha","col2":"beta"} + + +-- !query +SELECT STRUCT('alpha' AS A, 'beta' AS B) ST +-- !query schema +struct> +-- !query output +{"A":"alpha","B":"beta"} + + +-- !query +SELECT ID, STRUCT(ST.*) NST FROM tbl_x +-- !query schema +struct> +-- !query output +1 {"C":"gamma","D":"delta"} +2 {"C":"epsilon","D":"eta"} +3 {"C":"theta","D":"iota"} + + +-- !query +SELECT ID, STRUCT(ST.*,CAST(ID AS STRING) AS E) NST FROM tbl_x +-- !query schema +struct> +-- !query output +1 {"C":"gamma","D":"delta","E":"1"} +2 {"C":"epsilon","D":"eta","E":"2"} +3 {"C":"theta","D":"iota","E":"3"} + + +-- !query +SELECT ID, STRUCT(CAST(ID AS STRING) AS AA, ST.*) NST FROM tbl_x +-- !query schema +struct> +-- !query output +1 {"AA":"1","C":"gamma","D":"delta"} +2 {"AA":"2","C":"epsilon","D":"eta"} +3 {"AA":"3","C":"theta","D":"iota"} + + +-- !query +SELECT ID, STRUCT(ST.*).C NST FROM tbl_x +-- !query schema +struct +-- !query output +1 gamma +2 epsilon +3 theta + + +-- !query +SELECT ID, STRUCT(ST.C, ST.D).D NST FROM tbl_x +-- !query schema +struct +-- !query output +1 delta +2 eta +3 iota + + +-- !query +SELECT ID, STRUCT(ST.C as STC, ST.D as STD).STD FROM tbl_x +-- !query schema +struct +-- !query output +1 delta +2 eta +3 iota diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subexp-elimination.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subexp-elimination.sql.out new file mode 100644 index 000000000000..e35505589516 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subexp-elimination.sql.out @@ -0,0 +1,113 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +('{"a":1, "b":"2"}', '[{"a": 1, "b":2}, {"a":2, "b":2}]'), ('{"a":1, "b":"2"}', null), ('{"a":2, "b":"3"}', '[{"a": 3, "b":4}, {"a":4, "b":5}]'), ('{"a":5, "b":"6"}', '[{"a": 6, "b":7}, {"a":8, "b":9}]'), (null, '[{"a": 1, "b":2}, {"a":2, "b":2}]') +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT from_json(a, 'struct').a, from_json(a, 'struct').b, from_json(b, 'array>')[0].a, from_json(b, 'array>')[0].b FROM testData +-- !query schema +struct +-- !query output +1 2 1 2 +1 2 NULL NULL +2 3 3 4 +5 6 6 7 +NULL NULL 1 2 + + +-- !query +SELECT if(from_json(a, 'struct').a > 1, from_json(b, 'array>')[0].a, from_json(b, 'array>')[0].a + 1) FROM testData +-- !query schema +struct<(IF((from_json(a).a > 1), from_json(b)[0].a, (from_json(b)[0].a + 1))):int> +-- !query output +2 +2 +3 +6 +NULL + + +-- !query +SELECT if(isnull(from_json(a, 'struct').a), from_json(b, 'array>')[0].b + 1, from_json(b, 'array>')[0].b) FROM testData +-- !query schema +struct<(IF((from_json(a).a IS NULL), (from_json(b)[0].b + 1), from_json(b)[0].b)):int> +-- !query output +2 +3 +4 +7 +NULL + + +-- !query +SELECT case when from_json(a, 'struct').a > 5 then from_json(a, 'struct').b when from_json(a, 'struct').a > 4 then from_json(a, 'struct').b + 1 else from_json(a, 'struct').b + 2 end FROM testData +-- !query schema +struct 5) THEN from_json(a).b WHEN (from_json(a).a > 4) THEN (from_json(a).b + 1) ELSE (from_json(a).b + 2) END:bigint> +-- !query output +4 +4 +5 +7 +NULL + + +-- !query +SELECT case when from_json(a, 'struct').a > 5 then from_json(b, 'array>')[0].b when from_json(a, 'struct').a > 4 then from_json(b, 'array>')[0].b + 1 else from_json(b, 'array>')[0].b + 2 end FROM testData +-- !query schema +struct 5) THEN from_json(b)[0].b WHEN (from_json(a).a > 4) THEN (from_json(b)[0].b + 1) ELSE (from_json(b)[0].b + 2) END:int> +-- !query output +4 +4 +6 +8 +NULL + + +-- !query +SELECT from_json(a, 'struct').a + random() > 2, from_json(a, 'struct').b, from_json(b, 'array>')[0].a, from_json(b, 'array>')[0].b + + random() > 2 FROM testData +-- !query schema +struct<((from_json(a).a + rand()) > 2):boolean,from_json(a).b:string,from_json(b)[0].a:int,((from_json(b)[0].b + (+ rand())) > 2):boolean> +-- !query output +NULL NULL 1 true +false 2 1 true +false 2 NULL NULL +true 3 3 true +true 6 6 true + + +-- !query +SELECT if(from_json(a, 'struct').a + random() > 5, from_json(b, 'array>')[0].a, from_json(b, 'array>')[0].a + 1) FROM testData +-- !query schema +struct<(IF(((from_json(a).a + rand()) > 5), from_json(b)[0].a, (from_json(b)[0].a + 1))):int> +-- !query output +2 +2 +4 +6 +NULL + + +-- !query +SELECT case when from_json(a, 'struct').a > 5 then from_json(a, 'struct').b + random() > 5 when from_json(a, 'struct').a > 4 then from_json(a, 'struct').b + 1 + random() > 2 else from_json(a, 'struct').b + 2 + random() > 5 end FROM testData +-- !query schema +struct 5) THEN ((from_json(a).b + rand()) > 5) WHEN (from_json(a).a > 4) THEN (((from_json(a).b + 1) + rand()) > 2) ELSE (((from_json(a).b + 2) + rand()) > 5) END:boolean> +-- !query output +NULL +false +false +true +true + + +-- !query +DROP VIEW IF EXISTS testData +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-aggregate.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-aggregate.sql.out new file mode 100644 index 000000000000..ddf90c1127cf --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-aggregate.sql.out @@ -0,0 +1,304 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT emp.dept_id, + avg(salary), + sum(salary) +FROM emp +WHERE EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id) +GROUP BY dept_id +-- !query schema +struct +-- !query output +10 133.33333333333334 400.0 +20 300.0 300.0 +30 400.0 400.0 +70 150.0 150.0 + + +-- !query +SELECT emp_name +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id) +-- !query schema +struct +-- !query output +emp 1 +emp 1 +emp 2 +emp 3 +emp 4 +emp 8 + + +-- !query +SELECT count(*) +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id) +-- !query schema +struct +-- !query output +6 + + +-- !query +SELECT * +FROM bonus +WHERE EXISTS (SELECT 1 + FROM emp + WHERE emp.emp_name = bonus.emp_name + AND EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE emp.dept_id = dept.dept_id + GROUP BY dept.dept_id)) +-- !query schema +struct +-- !query output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 +emp 4 100.0 + + +-- !query +SELECT emp.dept_id, + Avg(salary), + Sum(salary) +FROM emp +WHERE NOT EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id) +GROUP BY dept_id +-- !query schema +struct +-- !query output +100 400.0 800.0 +NULL 400.0 400.0 + + +-- !query +SELECT emp_name +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id) +-- !query schema +struct +-- !query output +emp 5 +emp 6 - no dept +emp 7 + + +-- !query +SELECT count(*) +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept.dept_id) +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT * +FROM bonus +WHERE NOT EXISTS (SELECT 1 + FROM emp + WHERE emp.emp_name = bonus.emp_name + AND EXISTS (SELECT Max(dept.dept_id) + FROM dept + WHERE emp.dept_id = dept.dept_id + GROUP BY dept.dept_id)) +-- !query schema +struct +-- !query output +emp 5 1000.0 +emp 6 - no dept 500.0 + + +-- !query +SELECT * +FROM BONUS +WHERE EXISTS(SELECT RANK() OVER (PARTITION BY hiredate ORDER BY salary) AS s + FROM EMP, DEPT where EMP.dept_id = DEPT.dept_id + AND DEPT.dept_name < BONUS.emp_name) +-- !query schema +struct +-- !query output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 +emp 4 100.0 +emp 5 1000.0 +emp 6 - no dept 500.0 + + +-- !query +SELECT tt1.emp_name +FROM EMP as tt1 +WHERE EXISTS ( + select max(tt2.id) + from EMP as tt2 + where tt1.emp_name is null +) +-- !query schema +struct +-- !query output +emp 1 +emp 1 +emp 2 +emp 3 +emp 4 +emp 5 +emp 6 - no dept +emp 7 +emp 8 + + +-- !query +SELECT + emp.dept_id, + EXISTS (SELECT dept.dept_id FROM dept) +FROM emp +GROUP BY emp.dept_id ORDER BY emp.dept_id +-- !query schema +struct +-- !query output +NULL true +10 true +20 true +30 true +70 true +100 true + + +-- !query +SELECT + emp.dept_id, + EXISTS (SELECT dept.dept_id FROM dept) +FROM emp +GROUP BY emp.dept_id ORDER BY emp.dept_id +-- !query schema +struct +-- !query output +NULL true +10 true +20 true +30 true +70 true +100 true + + +-- !query +SELECT + emp.dept_id, + NOT EXISTS (SELECT dept.dept_id FROM dept) +FROM emp +GROUP BY emp.dept_id ORDER BY emp.dept_id +-- !query schema +struct +-- !query output +NULL false +10 false +20 false +30 false +70 false +100 false + + +-- !query +SELECT + emp.dept_id, + SUM( + CASE WHEN EXISTS (SELECT dept.dept_id FROM dept WHERE dept.dept_id = emp.dept_id) THEN 1 + ELSE 0 END) +FROM emp +GROUP BY emp.dept_id ORDER BY emp.dept_id +-- !query schema +struct +-- !query output +NULL 0 +10 3 +20 1 +30 1 +70 1 +100 0 + + +-- !query +SELECT + cast(EXISTS (SELECT id FROM dept where dept.dept_id = emp.dept_id) AS int) +FROM emp +GROUP BY + cast(EXISTS (SELECT id FROM dept where dept.dept_id = emp.dept_id) AS int) +-- !query schema +struct +-- !query output +0 +1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-basic.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-basic.sql.out new file mode 100644 index 000000000000..cb806fdaf820 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-basic.sql.out @@ -0,0 +1,211 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id + OR emp.dept_id IS NULL) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id) + AND emp.id > 200 +-- !query schema +struct +-- !query output +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id) + AND emp.id > 200 +-- !query schema +struct +-- !query output +emp 3 +emp 4 +emp 8 + + +-- !query +SELECT * +FROM dept +WHERE NOT EXISTS (SELECT emp_name + FROM emp + WHERE emp.dept_id = dept.dept_id) +-- !query schema +struct +-- !query output +40 dept 4 - unassigned OR +50 dept 5 - unassigned NJ + + +-- !query +SELECT * +FROM dept +WHERE NOT EXISTS (SELECT emp_name + FROM emp + WHERE emp.dept_id = dept.dept_id + OR state = 'NJ') +-- !query schema +struct +-- !query output +40 dept 4 - unassigned OR + + +-- !query +SELECT * +FROM bonus +WHERE NOT EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary) +-- !query schema +struct +-- !query output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 4 100.0 + + +-- !query +SELECT emp.* +FROM emp +WHERE NOT EXISTS (SELECT NULL + FROM bonus + WHERE bonus.emp_name = emp.emp_name) +-- !query schema +struct +-- !query output +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM bonus +WHERE EXISTS (SELECT emp_name + FROM emp + WHERE bonus.emp_name = emp.emp_name + AND EXISTS (SELECT state + FROM dept + WHERE dept.dept_id = emp.dept_id)) +-- !query schema +struct +-- !query output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 +emp 4 100.0 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-count-bug.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-count-bug.sql.out new file mode 100644 index 000000000000..ee9deab84d2e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-count-bug.sql.out @@ -0,0 +1,159 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1(c1, c2) as values (0, 1), (1, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2(c1, c2) as values (0, 2), (0, 3) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t3(c1, c2) as values (0, 3), (1, 4), (2, 5) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from t1 where exists (select count(*) from t2 where t2.c1 = t1.c1) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +select * from t1 where not exists (select count(*) from t2 where t2.c1 = t1.c1) +-- !query schema +struct +-- !query output + + + +-- !query +select *, exists (select count(*) from t2 where t2.c1 = t1.c1) from t1 +-- !query schema +struct +-- !query output +0 1 true +1 2 true + + +-- !query +select *, not exists (select count(*) from t2 where t2.c1 = t1.c1) from t1 +-- !query schema +struct +-- !query output +0 1 false +1 2 false + + +-- !query +select * from t1 where + exists(select count(*) + 1 from t2 where t2.c1 = t1.c1) OR + not exists (select count(*) - 1 from t2 where t2.c1 = t1.c1) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +select * from t1 where + (exists(select count(*) + 1 from t2 where t2.c1 = t1.c1) OR + not exists(select count(*) - 1 from t2 where t2.c1 = t1.c1)) AND + exists(select count(*) from t2 where t2.c1 = t1.c2) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +select * from t1 where exists (select count(*) from t2 where t1.c1 = 100) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +set spark.sql.optimizer.decorrelateExistsSubqueryLegacyIncorrectCountHandling.enabled = true +-- !query schema +struct +-- !query output +spark.sql.optimizer.decorrelateExistsSubqueryLegacyIncorrectCountHandling.enabled true + + +-- !query +select * from t1 where exists (select count(*) from t2 where t2.c1 = t1.c1) +-- !query schema +struct +-- !query output +0 1 + + +-- !query +select * from t1 where not exists (select count(*) from t2 where t2.c1 = t1.c1) +-- !query schema +struct +-- !query output +1 2 + + +-- !query +select *, exists (select count(*) from t2 where t2.c1 = t1.c1) from t1 +-- !query schema +struct +-- !query output +0 1 true +1 2 false + + +-- !query +select *, not exists (select count(*) from t2 where t2.c1 = t1.c1) from t1 +-- !query schema +struct +-- !query output +0 1 false +1 2 true + + +-- !query +select * from t1 where + exists(select count(*) + 1 from t2 where t2.c1 = t1.c1) OR + not exists (select count(*) - 1 from t2 where t2.c1 = t1.c1) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +select * from t1 where exists (select count(*) from t2 where t1.c1 = 100) +-- !query schema +struct +-- !query output + + + +-- !query +set spark.sql.optimizer.decorrelateExistsSubqueryLegacyIncorrectCountHandling.enabled = false +-- !query schema +struct +-- !query output +spark.sql.optimizer.decorrelateExistsSubqueryLegacyIncorrectCountHandling.enabled false diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-cte.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-cte.sql.out new file mode 100644 index 000000000000..09d8cb3bccc8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-cte.sql.out @@ -0,0 +1,197 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH bonus_cte + AS (SELECT * + FROM bonus + WHERE EXISTS (SELECT dept.dept_id, + emp.emp_name, + Max(salary), + Count(*) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY dept.dept_id, + emp.emp_name + ORDER BY emp.emp_name)) +SELECT * +FROM bonus a +WHERE a.bonus_amt > 30 + AND EXISTS (SELECT 1 + FROM bonus_cte b + WHERE a.emp_name = b.emp_name) +-- !query schema +struct +-- !query output +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 +emp 4 100.0 + + +-- !query +WITH emp_cte + AS (SELECT * + FROM emp + WHERE id >= 100 + AND id <= 300), + dept_cte + AS (SELECT * + FROM dept + WHERE dept_id = 10) +SELECT * +FROM bonus +WHERE EXISTS (SELECT * + FROM emp_cte a + JOIN dept_cte b + ON a.dept_id = b.dept_id + WHERE bonus.emp_name = a.emp_name) +-- !query schema +struct +-- !query output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 + + +-- !query +WITH emp_cte + AS (SELECT * + FROM emp + WHERE id >= 100 + AND id <= 300), + dept_cte + AS (SELECT * + FROM dept + WHERE dept_id = 10) +SELECT DISTINCT b.emp_name, + b.bonus_amt +FROM bonus b, + emp_cte e, + dept d +WHERE e.dept_id = d.dept_id + AND e.emp_name = b.emp_name + AND EXISTS (SELECT * + FROM emp_cte a + LEFT JOIN dept_cte b + ON a.dept_id = b.dept_id + WHERE e.emp_name = a.emp_name) +-- !query schema +struct +-- !query output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 + + +-- !query +WITH empdept + AS (SELECT id, + salary, + emp_name, + dept.dept_id + FROM emp + LEFT JOIN dept + ON emp.dept_id = dept.dept_id + WHERE emp.id IN ( 100, 200 )) +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE EXISTS (SELECT dept_id, + max(salary) + FROM empdept + GROUP BY dept_id + HAVING count(*) > 1) +GROUP BY emp_name +-- !query schema +struct +-- !query output +emp 1 30.0 +emp 2 400.0 +emp 3 300.0 +emp 4 100.0 +emp 5 1000.0 +emp 6 - no dept 500.0 + + +-- !query +WITH empdept + AS (SELECT id, + salary, + emp_name, + dept.dept_id + FROM emp + LEFT JOIN dept + ON emp.dept_id = dept.dept_id + WHERE emp.id IN ( 100, 200 )) +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE NOT EXISTS (SELECT dept_id, + Max(salary) + FROM empdept + GROUP BY dept_id + HAVING count(*) < 1) +GROUP BY emp_name +-- !query schema +struct +-- !query output +emp 1 30.0 +emp 2 400.0 +emp 3 300.0 +emp 4 100.0 +emp 5 1000.0 +emp 6 - no dept 500.0 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-having.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-having.sql.out new file mode 100644 index 000000000000..2a84516e90ab --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-having.sql.out @@ -0,0 +1,147 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW EMP(id, emp_name, hiredate, salary, dept_id) AS VALUES + (100, 'emp 1', date '2005-01-01', double(100.00), 10), + (100, 'emp 1', date '2005-01-01', double(100.00), 10), + (200, 'emp 2', date '2003-01-01', double(200.00), 10), + (300, 'emp 3', date '2002-01-01', double(300.00), 20), + (400, 'emp 4', date '2005-01-01', double(400.00), 30), + (500, 'emp 5', date '2001-01-01', double(400.00), NULL), + (600, 'emp 6 - no dept', date '2001-01-01', double(400.00), 100), + (700, 'emp 7', date '2010-01-01', double(400.00), 100), + (800, 'emp 8', date '2016-01-01', double(150.00), 70) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW DEPT(dept_id, dept_name, state) AS VALUES + (10, 'dept 1', 'CA'), + (20, 'dept 2', 'NY'), + (30, 'dept 3', 'TX'), + (40, 'dept 4 - unassigned', 'OR'), + (50, 'dept 5 - unassigned', 'NJ'), + (70, 'dept 7', 'FL') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW BONUS(emp_name, bonus_amt) AS VALUES + ('emp 1', double(10.00)), + ('emp 1', double(20.00)), + ('emp 2', double(300.00)), + ('emp 2', double(100.00)), + ('emp 3', double(300.00)), + ('emp 4', double(100.00)), + ('emp 5', double(1000.00)), + ('emp 6 - no dept', double(500.00)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT dept_id, count(*) +FROM emp +GROUP BY dept_id +HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt < min(emp.salary)) +-- !query schema +struct +-- !query output +10 3 +100 2 +20 1 +30 1 +70 1 +NULL 1 + + +-- !query +SELECT * +FROM dept +WHERE EXISTS (SELECT dept_id, + Count(*) + FROM emp + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt < Min(emp.salary))) +-- !query schema +struct +-- !query output +10 dept 1 CA +20 dept 2 NY +30 dept 3 TX +40 dept 4 - unassigned OR +50 dept 5 - unassigned NJ +70 dept 7 FL + + +-- !query +SELECT dept_id, + Max(salary) +FROM emp gp +WHERE EXISTS (SELECT dept_id, + Count(*) + FROM emp p + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt < Min(p.salary))) +GROUP BY gp.dept_id +-- !query schema +struct +-- !query output +10 200.0 +100 400.0 +20 300.0 +30 400.0 +70 150.0 +NULL 400.0 + + +-- !query +SELECT * +FROM dept +WHERE EXISTS (SELECT dept_id, + Count(*) + FROM emp + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE bonus_amt > Min(emp.salary))) +-- !query schema +struct +-- !query output +10 dept 1 CA +20 dept 2 NY +30 dept 3 TX +40 dept 4 - unassigned OR +50 dept 5 - unassigned NJ +70 dept 7 FL + + +-- !query +SELECT * +FROM dept +WHERE EXISTS (SELECT dept_id, + count(emp.dept_id) + FROM emp + WHERE dept.dept_id = dept_id + GROUP BY dept_id + HAVING EXISTS (SELECT 1 + FROM bonus + WHERE ( bonus_amt > min(emp.salary) + AND count(emp.dept_id) > 1 ))) +-- !query schema +struct +-- !query output +10 dept 1 CA diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-in-join-condition.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-in-join-condition.sql.out new file mode 100644 index 000000000000..c9c68a5f0602 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-in-join-condition.sql.out @@ -0,0 +1,504 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMP VIEW x(x1, x2) AS VALUES + (2, 1), + (1, 1), + (3, 4) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMP VIEW y(y1, y2) AS VALUES + (0, 2), + (1, 4), + (4, 11) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMP VIEW z(z1, z2) AS VALUES + (4, 2), + (3, 3), + (8, 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from x inner join y on x1 = y1 and exists (select * from z where z2 = x2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 + + +-- !query +select * from x inner join y on x1 = y1 and not exists (select * from z where z2 = x2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output + + + +-- !query +select * from x inner join y on x1 = y1 and exists (select * from z where z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output + + + +-- !query +select * from x inner join y on x1 = y1 and not exists (select * from z where z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 + + +-- !query +select * from x left join y on x1 = y1 and exists (select * from z where z2 = x2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x left join y on x1 = y1 and not exists (select * from z where z2 = x2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 NULL NULL +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x left join y on x1 = y1 and exists (select * from z where z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 NULL NULL +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x left join y on x1 = y1 and not exists (select * from z where z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x right join y on x1 = y1 and exists (select * from z where z2 = x2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 4 11 +1 1 1 4 + + +-- !query +select * from x right join y on x1 = y1 and not exists (select * from z where z2 = x2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 1 4 +NULL NULL 4 11 + + +-- !query +select * from x right join y on x1 = y1 and exists (select * from z where z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 1 4 +NULL NULL 4 11 + + +-- !query +select * from x right join y on x1 = y1 and not exists (select * from z where z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 4 11 +1 1 1 4 + + +-- !query +select * from x right join y on x1 = y1 and exists (select * from z where z2 = x2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 4 11 +1 1 1 4 + + +-- !query +select * from x right join y on x1 = y1 and not exists (select * from z where z2 = x2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 1 4 +NULL NULL 4 11 + + +-- !query +select * from x right join y on x1 = y1 and exists (select * from z where z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 1 4 +NULL NULL 4 11 + + +-- !query +select * from x right join y on x1 = y1 and not exists (select * from z where z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 4 11 +1 1 1 4 + + +-- !query +select * from x left semi join y on x1 = y1 and exists (select * from z where z2 = x2) order by x1, x2 +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select * from x left semi join y on x1 = y1 and not exists (select * from z where z2 = x2) order by x1, x2 +-- !query schema +struct +-- !query output + + + +-- !query +select * from x left semi join y on x1 = y1 and exists (select * from z where z2 = y2) order by x1, x2 +-- !query schema +struct +-- !query output + + + +-- !query +select * from x left semi join y on x1 = y1 and not exists (select * from z where z2 = y2) order by x1, x2 +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select * from x left anti join y on x1 = y1 and exists (select * from z where z2 = x2) order by x1, x2 +-- !query schema +struct +-- !query output +2 1 +3 4 + + +-- !query +select * from x left anti join y on x1 = y1 and not exists (select * from z where z2 = x2) order by x1, x2 +-- !query schema +struct +-- !query output +1 1 +2 1 +3 4 + + +-- !query +select * from x left anti join y on x1 = y1 and exists (select * from z where z2 = y2) order by x1, x2 +-- !query schema +struct +-- !query output +1 1 +2 1 +3 4 + + +-- !query +select * from x left anti join y on x1 = y1 and not exists (select * from z where z2 = y2) order by x1, x2 +-- !query schema +struct +-- !query output +2 1 +3 4 + + +-- !query +select * from x full outer join y on x1 = y1 and exists (select * from z where z2 = x2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 4 11 +1 1 1 4 +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x full outer join y on x1 = y1 and not exists (select * from z where z2 = x2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 1 4 +NULL NULL 4 11 +1 1 NULL NULL +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x full outer join y on x1 = y1 and exists (select * from z where z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 1 4 +NULL NULL 4 11 +1 1 NULL NULL +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x full outer join y on x1 = y1 and not exists (select * from z where z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 4 11 +1 1 1 4 +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x inner join y on x1 = y1 or exists (select * from z where z1 = x1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 +3 4 0 2 +3 4 1 4 +3 4 4 11 + + +-- !query +select * from x inner join y on x1 = y1 or not exists (select * from z where z1 = x1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 0 2 +1 1 1 4 +1 1 4 11 +2 1 0 2 +2 1 1 4 +2 1 4 11 + + +-- !query +select * from x left join y on x1 = y1 or exists (select * from z where z1 = x1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 +2 1 NULL NULL +3 4 0 2 +3 4 1 4 +3 4 4 11 + + +-- !query +select * from x left join y on x1 = y1 or not exists (select * from z where z1 = x1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 0 2 +1 1 1 4 +1 1 4 11 +2 1 0 2 +2 1 1 4 +2 1 4 11 +3 4 NULL NULL + + +-- !query +select * from x inner join y on x1 = y1 or exists (select * from z where z1 = y1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 +1 1 4 11 +2 1 4 11 +3 4 4 11 + + +-- !query +select * from x inner join y on x1 = y1 or not exists (select * from z where z1 = y1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 0 2 +1 1 1 4 +2 1 0 2 +2 1 1 4 +3 4 0 2 +3 4 1 4 + + +-- !query +select * from x left join y on x1 = y1 or exists (select * from z where z1 = y1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 +1 1 4 11 +2 1 4 11 +3 4 4 11 + + +-- !query +select * from x left join y on x1 = y1 or not exists (select * from z where z1 = y1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 0 2 +1 1 1 4 +2 1 0 2 +2 1 1 4 +3 4 0 2 +3 4 1 4 + + +-- !query +select * from x inner join y on x1 = y1 and exists (select * from z where z1 = x1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output + + + +-- !query +select * from x inner join y on x1 = y1 and not exists (select * from z where z1 = x1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 + + +-- !query +select * from x left join y on x1 = y1 and exists (select * from z where z1 = x1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 NULL NULL +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x left join y on x1 = y1 and not exists (select * from z where z1 = x1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x inner join y on x1 = y1 and exists (select * from z where z1 = y1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output + + + +-- !query +select * from x inner join y on x1 = y1 and not exists (select * from z where z1 = y1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 + + +-- !query +select * from x left join y on x1 = y1 and exists (select * from z where z1 = y1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 NULL NULL +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x left join y on x1 = y1 and not exists (select * from z where z1 = y1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x join y on x1 = y1 and exists (select * from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_EXPRESSION_IN_JOIN_CONDITION", + "sqlState" : "0A000", + "messageParameters" : { + "subqueryExpression" : "exists(x.x2, y.y2, (z.z2 = x.x2), (z.z2 = y.y2))" + } +} + + +-- !query +select * from x join y on x1 = y1 and not exists (select * from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_EXPRESSION_IN_JOIN_CONDITION", + "sqlState" : "0A000", + "messageParameters" : { + "subqueryExpression" : "exists(x.x2, y.y2, (z.z2 = x.x2), (z.z2 = y.y2))" + } +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out new file mode 100644 index 000000000000..d030b92ed4a6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-joins-and-set-ops.sql.out @@ -0,0 +1,512 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * +FROM emp, + dept +WHERE emp.dept_id = dept.dept_id + AND EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 10 dept 1 CA +100 emp 1 2005-01-01 100.0 10 10 dept 1 CA +200 emp 2 2003-01-01 200.0 10 10 dept 1 CA +300 emp 3 2002-01-01 300.0 20 20 dept 2 NY +400 emp 4 2005-01-01 400.0 30 30 dept 3 TX + + +-- !query +SELECT * +FROM emp + JOIN dept + ON emp.dept_id = dept.dept_id +WHERE EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 10 dept 1 CA +100 emp 1 2005-01-01 100.0 10 10 dept 1 CA +200 emp 2 2003-01-01 200.0 10 10 dept 1 CA +300 emp 3 2002-01-01 300.0 20 20 dept 2 NY +400 emp 4 2005-01-01 400.0 30 30 dept 3 TX + + +-- !query +SELECT * +FROM emp + LEFT JOIN dept + ON emp.dept_id = dept.dept_id +WHERE EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 10 dept 1 CA +100 emp 1 2005-01-01 100.0 10 10 dept 1 CA +200 emp 2 2003-01-01 200.0 10 10 dept 1 CA +300 emp 3 2002-01-01 300.0 20 20 dept 2 NY +400 emp 4 2005-01-01 400.0 30 30 dept 3 TX +500 emp 5 2001-01-01 400.0 NULL NULL NULL NULL +600 emp 6 - no dept 2001-01-01 400.0 100 NULL NULL NULL + + +-- !query +SELECT * +FROM emp, + dept +WHERE emp.dept_id = dept.dept_id + AND NOT EXISTS (SELECT * + FROM bonus + WHERE bonus.emp_name = emp.emp_name) +-- !query schema +struct +-- !query output +800 emp 8 2016-01-01 150.0 70 70 dept 7 FL + + +-- !query +SELECT * +FROM bonus +WHERE EXISTS (SELECT * + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name) +-- !query schema +struct +-- !query output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 +emp 4 100.0 + + +-- !query +SELECT * +FROM bonus +WHERE EXISTS (SELECT * + FROM emp + RIGHT JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name) +-- !query schema +struct +-- !query output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 +emp 4 100.0 + + +-- !query +SELECT * +FROM bonus +WHERE EXISTS (SELECT dept.dept_id, + emp.emp_name, + Max(salary), + Count(*) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY dept.dept_id, + emp.emp_name + ORDER BY emp.emp_name) +-- !query schema +struct +-- !query output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 +emp 4 100.0 + + +-- !query +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE EXISTS (SELECT emp_name, + Max(salary) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY emp_name + HAVING Count(*) > 1 + ORDER BY emp_name) +GROUP BY emp_name +-- !query schema +struct +-- !query output +emp 1 30.0 + + +-- !query +SELECT emp_name, + Sum(bonus_amt) +FROM bonus +WHERE NOT EXISTS (SELECT emp_name, + Max(salary) + FROM emp + JOIN dept + ON dept.dept_id = emp.dept_id + WHERE bonus.emp_name = emp.emp_name + GROUP BY emp_name + HAVING Count(*) > 1 + ORDER BY emp_name) +GROUP BY emp_name +-- !query schema +struct +-- !query output +emp 2 400.0 +emp 3 300.0 +emp 4 100.0 +emp 5 1000.0 +emp 6 - no dept 500.0 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + UNION + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + EXCEPT + SELECT * + FROM dept + WHERE dept_id > 50) +UNION ALL +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + EXCEPT + SELECT * + FROM dept + WHERE dept_id > 50) +UNION +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id < 30 + INTERSECT + SELECT * + FROM dept + WHERE dept_id >= 30 + AND dept_id <= 50) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + UNION + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX") +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +400 emp 4 2005-01-01 400.0 30 + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + UNION + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX") +-- !query schema +struct +-- !query output +300 emp 3 2002-01-01 300.0 20 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + INTERSECT ALL + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX") +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + INTERSECT DISTINCT + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX") +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + EXCEPT ALL + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX") +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + EXCEPT DISTINCT + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX") +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + INTERSECT ALL + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX") +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "CA" + EXCEPT DISTINCT + SELECT * + FROM dept + WHERE dept_id = emp.dept_id and state = "TX") +-- !query schema +struct +-- !query output +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-orderby-limit.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-orderby-limit.sql.out new file mode 100644 index 000000000000..7d7537e38441 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-orderby-limit.sql.out @@ -0,0 +1,590 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state) +ORDER BY hiredate +-- !query schema +struct +-- !query output +300 emp 3 2002-01-01 300.0 20 +200 emp 2 2003-01-01 200.0 10 +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +400 emp 4 2005-01-01 400.0 30 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT id, + hiredate +FROM emp +WHERE EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state) +ORDER BY hiredate DESC +-- !query schema +struct +-- !query output +800 2016-01-01 +100 2005-01-01 +100 2005-01-01 +400 2005-01-01 +200 2003-01-01 +300 2002-01-01 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state + LIMIT 1) +ORDER BY hiredate +-- !query schema +struct +-- !query output +300 emp 3 2002-01-01 300.0 20 +200 emp 2 2003-01-01 200.0 10 +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +400 emp 4 2005-01-01 400.0 30 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state + LIMIT 0) +ORDER BY hiredate +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state) +ORDER BY hiredate +-- !query schema +struct +-- !query output +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 + + +-- !query +SELECT emp_name +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY state + ORDER BY state) +-- !query schema +struct +-- !query output +emp 5 +emp 6 - no dept +emp 7 + + +-- !query +SELECT count(*) +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY dept_id + ORDER BY dept_id) +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state + LIMIT 1) +ORDER BY hiredate +-- !query schema +struct +-- !query output +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_id + FROM dept + WHERE emp.dept_id = dept.dept_id + ORDER BY state + LIMIT 0) +ORDER BY hiredate +-- !query schema +struct +-- !query output +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +300 emp 3 2002-01-01 300.0 20 +200 emp 2 2003-01-01 200.0 10 +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +400 emp 4 2005-01-01 400.0 30 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 10 + LIMIT 1) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + GROUP BY state + LIMIT 1) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 100 + LIMIT 1) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE dept.dept_id > 100 + GROUP BY state + LIMIT 1) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT emp_name +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY state + ORDER BY state + LIMIT 2 + OFFSET 1) +-- !query schema +struct +-- !query output +emp 1 +emp 1 +emp 2 +emp 3 +emp 4 +emp 5 +emp 6 - no dept +emp 7 +emp 8 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 10 + LIMIT 1 + OFFSET 2) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > emp.dept_id + LIMIT 1) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + GROUP BY state + LIMIT 1 + OFFSET 2) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE dept.dept_id <> emp.dept_id + GROUP BY state + LIMIT 1) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE emp.salary > 200 + LIMIT 1) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT state, max(dept.dept_name) + FROM dept + WHERE emp.salary > 200 + GROUP BY state + LIMIT 1) +-- !query schema +struct +-- !query output +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 100 + LIMIT 1 + OFFSET 2) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE dept.dept_id > 100 + GROUP BY state + LIMIT 1 + OFFSET 2) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id <> emp.dept_id + LIMIT 1 + OFFSET 2) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 10 + OFFSET 2) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) + FROM dept + GROUP BY state + OFFSET 2) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT dept.dept_name + FROM dept + WHERE dept.dept_id > 100 + OFFSET 2) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT * +FROM emp +WHERE NOT EXISTS (SELECT max(dept.dept_id) + FROM dept + WHERE dept.dept_id > 100 + GROUP BY state + OFFSET 2) +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +600 emp 6 - no dept 2001-01-01 400.0 100 +700 emp 7 2010-01-01 400.0 100 +800 emp 8 2016-01-01 150.0 70 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-outside-filter.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-outside-filter.sql.out new file mode 100644 index 000000000000..8e92017f24db --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-outside-filter.sql.out @@ -0,0 +1,253 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + emp_name, + EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30) +FROM emp +-- !query schema +struct +-- !query output +emp 1 true +emp 1 true +emp 2 true +emp 3 true +emp 4 true +emp 5 true +emp 6 - no dept true +emp 7 true +emp 8 true + + +-- !query +SELECT + emp_name, + EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id) +FROM emp +-- !query schema +struct +-- !query output +emp 1 true +emp 1 true +emp 2 true +emp 3 true +emp 4 true +emp 5 false +emp 6 - no dept false +emp 7 false +emp 8 true + + +-- !query +SELECT + sum(salary), + sum(salary) FILTER (WHERE EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30)) +FROM emp +-- !query schema +struct +-- !query output +2450.0 2450.0 + + +-- !query +SELECT + sum(salary), + sum(salary) FILTER (WHERE EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id)) +FROM emp +-- !query schema +struct +-- !query output +2450.0 1250.0 + + +-- !query +SELECT + sum(salary), + sum(salary) FILTER (WHERE EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id) + OR EXISTS (SELECT 1 + FROM bonus + WHERE emp.emp_name = bonus.emp_name)) +FROM emp +-- !query schema +struct +-- !query output +2450.0 2050.0 + + +-- !query +SELECT + sum(DISTINCT salary), + count(DISTINCT hiredate) FILTER (WHERE EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id)) +FROM emp +-- !query schema +struct +-- !query output +1150.0 4 + + +-- !query +SELECT + count(hiredate), + sum(salary) +FROM emp +GROUP BY EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id) +-- !query schema +struct +-- !query output +3 1200.0 +6 1250.0 + + +-- !query +SELECT + count(DISTINCT hiredate), + sum(DISTINCT salary) +FROM emp +GROUP BY EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id) +-- !query schema +struct +-- !query output +2 400.0 +4 1150.0 + + +-- !query +SELECT + count(CASE WHEN EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30) THEN 1 END), + sum(CASE WHEN EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30) THEN salary END) +FROM emp +-- !query schema +struct +-- !query output +9 2450.0 + + +-- !query +SELECT + count(CASE WHEN EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id) THEN 1 END), + sum(CASE WHEN EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id) THEN salary END) +FROM emp +-- !query schema +struct +-- !query output +6 1250.0 + + +-- !query +SELECT + emp_name, + sum(salary) OVER (PARTITION BY EXISTS (SELECT 1 + FROM dept + WHERE dept.dept_id > 10 + AND dept.dept_id < 30)) +FROM emp +-- !query schema +struct +-- !query output +emp 1 2450.0 +emp 1 2450.0 +emp 2 2450.0 +emp 3 2450.0 +emp 4 2450.0 +emp 5 2450.0 +emp 6 - no dept 2450.0 +emp 7 2450.0 +emp 8 2450.0 + + +-- !query +SELECT + emp_name, + sum(salary) OVER (PARTITION BY EXISTS (SELECT 1 + FROM dept + WHERE emp.dept_id = dept.dept_id)) +FROM emp +-- !query schema +struct +-- !query output +emp 1 1250.0 +emp 1 1250.0 +emp 2 1250.0 +emp 3 1250.0 +emp 4 1250.0 +emp 5 1200.0 +emp 6 - no dept 1200.0 +emp 7 1200.0 +emp 8 1250.0 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-within-and-or.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-within-and-or.sql.out new file mode 100644 index 000000000000..b6f4c5c5c319 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/exists-subquery/exists-within-and-or.sql.out @@ -0,0 +1,153 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id) + OR emp.id > 200 +-- !query schema +struct +-- !query output +emp 1 +emp 1 +emp 2 +emp 3 +emp 4 +emp 5 +emp 6 - no dept +emp 7 +emp 8 + + +-- !query +SELECT * +FROM emp +WHERE EXISTS (SELECT dept.dept_name + FROM dept + WHERE emp.dept_id = dept.dept_id) + OR emp.dept_id IS NULL +-- !query schema +struct +-- !query output +100 emp 1 2005-01-01 100.0 10 +100 emp 1 2005-01-01 100.0 10 +200 emp 2 2003-01-01 200.0 10 +300 emp 3 2002-01-01 300.0 20 +400 emp 4 2005-01-01 400.0 30 +500 emp 5 2001-01-01 400.0 NULL +800 emp 8 2016-01-01 150.0 70 + + +-- !query +SELECT emp.emp_name +FROM emp +WHERE EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id + AND dept.dept_id = 20) + OR EXISTS (SELECT dept.state + FROM dept + WHERE emp.dept_id = dept.dept_id + AND dept.dept_id = 30) +-- !query schema +struct +-- !query output +emp 3 +emp 4 + + +-- !query +SELECT * +FROM bonus +WHERE ( NOT EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary) + OR EXISTS (SELECT * + FROM emp + WHERE emp.emp_name = emp_name + OR bonus_amt < emp.salary) ) +-- !query schema +struct +-- !query output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 2 300.0 +emp 3 300.0 +emp 4 100.0 +emp 5 1000.0 +emp 6 - no dept 500.0 + + +-- !query +SELECT * FROM bonus WHERE NOT EXISTS +( + SELECT * + FROM emp + WHERE emp.emp_name = emp_name + AND bonus_amt > emp.salary) +AND +emp_name IN +( + SELECT emp_name + FROM emp + WHERE bonus_amt < emp.salary) +-- !query schema +struct +-- !query output +emp 1 10.0 +emp 1 20.0 +emp 2 100.0 +emp 4 100.0 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-basic.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-basic.sql.out new file mode 100644 index 000000000000..3a07dbdbf00e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-basic.sql.out @@ -0,0 +1,76 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view tab_a as select * from values (1, 1) as tab_a(a1, b1) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view tab_b as select * from values (1, 1) as tab_b(a2, b2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view struct_tab as select struct(col1 as a, col2 as b) as record from + values (1, 1), (1, 2), (2, 1), (2, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +select 1 from tab_a where (a1, b1) not in (select a2, b2 from tab_b) +-- !query schema +struct<1:int> +-- !query output + + + +-- !query +select 1 from tab_a where (a1, b1) not in (select (a2, b2) from tab_b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_LENGTH_MISMATCH", + "sqlState" : "42K09", + "messageParameters" : { + "leftColumns" : "\"a1\", \"b1\"", + "leftLength" : "2", + "rightColumns" : "\"named_struct(a2, a2, b2, b2)\"", + "rightLength" : "1", + "sqlExpr" : "\"(named_struct('a1', a1, 'b1', b1) IN (listquery()))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 70, + "fragment" : "not in (select (a2, b2) from tab_b)" + } ] +} + + +-- !query +select count(*) from struct_tab where record in + (select (a2 as a, b2 as b) from tab_b) +-- !query schema +struct +-- !query output +1 + + +-- !query +select count(*) from struct_tab where record not in + (select (a2 as a, b2 as b) from tab_b) +-- !query schema +struct +-- !query output +3 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-count-bug.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-count-bug.sql.out new file mode 100644 index 000000000000..147f023ed8ea --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-count-bug.sql.out @@ -0,0 +1,182 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1(c1, c2) as values (0, 1), (1, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2(c1, c2) as values (0, 2), (0, 3) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t3(c1, c2) as values (0, 3), (1, 4), (2, 5) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from t1 where c1 in (select count(*) + 1 from t2 where t2.c1 = t1.c1) +-- !query schema +struct +-- !query output +1 2 + + +-- !query +select *, c1 in (select count(*) + 1 from t2 where t2.c1 = t1.c1) +from t1 +-- !query schema +struct +-- !query output +0 1 false +1 2 true + + +-- !query +select *, c1 not in (select count(*) + 1 from t2 where t2.c1 = t1.c1) +from t1 +-- !query schema +struct +-- !query output +0 1 true +1 2 false + + +-- !query +select * from t1 where + c1 in (select count(*) + 1 from t2 where t2.c1 = t1.c1) OR + c2 in (select count(*) - 1 from t2 where t2.c1 = t1.c1) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +select * from t1 where + (c1 in (select count(*) + 1 from t2 where t2.c1 = t1.c1) OR + c2 in (select count(*) - 1 from t2 where t2.c1 = t1.c1)) AND + c1 NOT in (select count(*) from t2 where t2.c1 = t1.c2) +-- !query schema +struct +-- !query output +1 2 + + +-- !query +select * from t1 where c1 in (select 1 from t2 where t2.c1 = t1.c1 having count(*) = 0) +-- !query schema +struct +-- !query output +1 2 + + +-- !query +select * from t1 where c1 not in (select 1 from t2 where t2.c1 = t1.c1 having count(*) = 0) +-- !query schema +struct +-- !query output +0 1 + + +-- !query +select * from t1 where c1 in (select count(*) from t1 join t3 using (c1) where t3.c1 = t1.c2) +-- !query schema +struct +-- !query output +0 1 + + +-- !query +select * from t1 where c1 not in (select count(*) + 1 from t1 join t3 using (c1) where t3.c1 = t1.c2) +-- !query schema +struct +-- !query output +0 1 + + +-- !query +set spark.sql.optimizer.decorrelateExistsSubqueryLegacyIncorrectCountHandling.enabled = true +-- !query schema +struct +-- !query output +spark.sql.optimizer.decorrelateExistsSubqueryLegacyIncorrectCountHandling.enabled true + + +-- !query +select * from t1 where c1 in (select count(*) + 1 from t2 where t2.c1 = t1.c1) +-- !query schema +struct +-- !query output + + + +-- !query +select *, c1 in (select count(*) + 1 from t2 where t2.c1 = t1.c1) +from t1 +-- !query schema +struct +-- !query output +0 1 false +1 2 false + + +-- !query +select *, c1 not in (select count(*) + 1 from t2 where t2.c1 = t1.c1) +from t1 +-- !query schema +struct +-- !query output +0 1 true +1 2 true + + +-- !query +select * from t1 where c1 in (select 1 from t2 where t2.c1 = t1.c1 having count(*) = 0) +-- !query schema +struct +-- !query output + + + +-- !query +select * from t1 where c1 not in (select 1 from t2 where t2.c1 = t1.c1 having count(*) = 0) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +select * from t1 where c1 in (select count(*) from t1 join t3 using (c1) where t3.c1 = t1.c2) +-- !query schema +struct +-- !query output +0 1 + + +-- !query +select * from t1 where c1 not in (select count(*) + 1 from t1 join t3 using (c1) where t3.c1 = t1.c2) +-- !query schema +struct +-- !query output +0 1 + + +-- !query +set spark.sql.optimizer.decorrelateExistsSubqueryLegacyIncorrectCountHandling.enabled = false +-- !query schema +struct +-- !query output +spark.sql.optimizer.decorrelateExistsSubqueryLegacyIncorrectCountHandling.enabled false diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-group-by.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-group-by.sql.out new file mode 100644 index 000000000000..42abf286d45d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-group-by.sql.out @@ -0,0 +1,487 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("t1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("t1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2 as select * from values + ("t2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("t1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("t2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("t1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("t1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t3 as select * from values + ("t3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("t3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("t3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("t3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("t1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("t3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1a, + Avg(t1b) +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2) +GROUP BY t1a +-- !query schema +struct +-- !query output +t1b 8.0 +t1c 8.0 +t1e 10.0 + + +-- !query +SELECT t1a, + Max(t1b) +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a) +GROUP BY t1a, + t1d +-- !query schema +struct +-- !query output +t1b 8 + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) +GROUP BY t1a, + t1b +-- !query schema +struct +-- !query output +t1b 8 +t1c 8 + + +-- !query +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) + OR t1c IN (SELECT t3c + FROM t3 + WHERE t1a = t3a) +GROUP BY t1a, + t1c +-- !query schema +struct +-- !query output +t1b 8 +t1c 8 + + +-- !query +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) + AND t1c IN (SELECT t3c + FROM t3 + WHERE t1a = t3a) +GROUP BY t1a, + t1c +-- !query schema +struct +-- !query output +t1b 8 + + +-- !query +SELECT t1a, + Count(DISTINCT( t1b )) +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) +GROUP BY t1a, + t1c +HAVING t1a = "t1b" +-- !query schema +struct +-- !query output +t1b 1 + + +-- !query +SELECT * +FROM t1 +WHERE t1b IN (SELECT Max(t2b) + FROM t2 + GROUP BY t2a) +-- !query schema +struct +-- !query output +t1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +t1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 +t1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +t1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 +t1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 +t1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +t1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +t1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 + + +-- !query +SELECT * +FROM (SELECT t2a, + t2b + FROM t2 + WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b) + GROUP BY t2a, + t2b) t2 +-- !query schema +struct +-- !query output +t1b 8 + + +-- !query +SELECT Count(DISTINCT * ) +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + AND t1c = t2c + GROUP BY t2a) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT Max(t2c) + FROM t2 + WHERE t1a = t2a + GROUP BY t2a, + t2c + HAVING t2c > 8) +-- !query schema +struct +-- !query output +t1b 8 +t1c 8 + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2a IN (SELECT Min(t3a) + FROM t3 + WHERE t3a = t2a + GROUP BY t3b) + GROUP BY t2c) +-- !query schema +struct +-- !query output +t1a 16 +t1a 16 +t1b 8 +t1c 8 +t1d NULL +t1d NULL + + +-- !query +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a) +GROUP BY t1a +-- !query schema +struct +-- !query output +t1b 8 +t1c 8 + + +-- !query +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b IN (SELECT Min(t3b) + FROM t3 + WHERE t2a = t3a + GROUP BY t3a) + GROUP BY t2c) +GROUP BY t1a, + t1d +-- !query schema +struct +-- !query output +t1b 8 +t1c 8 +t1d NULL +t1d NULL + + +-- !query +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a) + AND t1d IN (SELECT t3d + FROM t3 + WHERE t1c = t3c + GROUP BY t3d) +GROUP BY t1a +-- !query schema +struct +-- !query output +t1b 8 +t1c 8 + + +-- !query +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a) + OR t1d IN (SELECT t3d + FROM t3 + WHERE t1c = t3c + GROUP BY t3d) +GROUP BY t1a +-- !query schema +struct +-- !query output +t1a 16 +t1b 8 +t1c 8 +t1d NULL + + +-- !query +SELECT t1a, + Min(t1b) +FROM t1 +WHERE t1c IN (SELECT Min(t2c) + FROM t2 + WHERE t2b = t1b + GROUP BY t2a + HAVING t2a > t1a) + OR t1d IN (SELECT t3d + FROM t3 + WHERE t1c = t3c + GROUP BY t3d + HAVING t3d = t1d) +GROUP BY t1a +HAVING Min(t1b) IS NOT NULL +-- !query schema +struct +-- !query output +t1a 16 +t1b 8 +t1c 8 + + +-- !query +select t1a +from t1 +where t1f IN (SELECT RANK() OVER (partition by t3c order by t2b) as s + FROM t2, t3 where t2.t2c = t3.t3c and t2.t2a < t1.t1a) +-- !query schema +struct +-- !query output +t1d +t1d +t1d +t1e +t1e +t1e + + +-- !query +SELECT + t1.t1a, + t1.t1a IN (SELECT t2a FROM t2) as v1 +FROM t1 +GROUP BY t1.t1a ORDER BY t1.t1a +-- !query schema +struct +-- !query output +t1a false +t1b true +t1c true +t1d false +t1e true + + +-- !query +SELECT + count(cast(t1.t1a IN (SELECT t2a FROM t2) as INT)), + sum(cast(t1.t1b NOT IN (SELECT t2b FROM t2) as INT)) +FROM t1 +-- !query schema +struct +-- !query output +12 0 + + +-- !query +SELECT + agg_results.t1a, + COUNT(*) + FROM (SELECT t1.t1a FROM t1 WHERE t1.t1a IN (SELECT t2a FROM t2)) AS agg_results +GROUP BY agg_results.t1a ORDER BY agg_results.t1a +-- !query schema +struct +-- !query output +t1b 1 +t1c 1 +t1e 3 + + +-- !query +SELECT + t1.t1a, + CASE + WHEN t1.t1a IN (SELECT t2a FROM t2) THEN 10 + ELSE -10 + END AS v1 +FROM t1 +GROUP BY t1.t1a +ORDER BY t1.t1a +-- !query schema +struct +-- !query output +t1a -10 +t1b 10 +t1c 10 +t1d -10 +t1e 10 + + +-- !query +SELECT + t1.t1c, + SUM(CASE + WHEN t1.t1c IN (SELECT t2c FROM t2) THEN 10 + ELSE -10 + END) AS v1, + SUM(CASE + WHEN t1.t1d IN (SELECT t2c FROM t2) THEN 10 + ELSE -10 + END) AS v2, + t1.t1c + 10 IN (SELECT t2c + 2 FROM t2) AS v3, + count(t1.t1c) as ct, + count(t1.t1d) +FROM t1 +GROUP BY t1.t1c +ORDER BY t1.t1c +-- !query schema +struct +-- !query output +NULL -40 -20 false 0 4 +8 -20 -20 true 2 2 +12 20 -20 false 2 2 +16 40 -40 false 4 4 + + +-- !query +SELECT + SUM(CASE + WHEN t1.t1c IN (SELECT t2c FROM t2) THEN 10 + ELSE -10 + END) AS v1, + count(t1.t1c) as ct +FROM t1 +-- !query schema +struct +-- !query output +0 8 + + +-- !query +SELECT + cast(t1a in (select t2a from t2) as int) + 1 as groupExpr, + sum(cast(t1a in (select t2a from t2) as int) + 1) as aggExpr, + cast(t1a in (select t2a from t2) as int) + 1 + cast(exists (select t2a from t2) as int) + as complexExpr +FROM t1 +GROUP BY + cast(t1a in (select t2a from t2) as int) + 1 +-- !query schema +struct +-- !query output +1 7 2 +2 10 3 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-having.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-having.sql.out new file mode 100644 index 000000000000..e4b5db98c560 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-having.sql.out @@ -0,0 +1,214 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1a, + t1b, + t1h +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + GROUP BY t2b + HAVING t2b < 10) +-- !query schema +struct +-- !query output +val1a 6 2014-04-04 01:00:00 +val1a 6 2014-04-04 01:02:00.001 +val1b 8 2014-05-04 01:01:00 +val1c 8 2014-05-04 01:02:00.001 + + +-- !query +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2b + HAVING t2b > 1) +-- !query schema +struct +-- !query output +val1b 8 16 + + +-- !query +SELECT t1a, t1b, t1c +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1c < t2c) +GROUP BY t1a, t1b, t1c +HAVING t1b < 10 +-- !query schema +struct +-- !query output +val1a 6 8 + + +-- !query +SELECT t1a, t1b, t1c +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1c = t2c) +GROUP BY t1a, t1b, t1c +HAVING COUNT (DISTINCT t1b) < 10 +-- !query schema +struct +-- !query output +val1b 8 16 +val1c 8 16 + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a + GROUP BY t2c + HAVING t2c > 10) +GROUP BY t1b +HAVING t1b >= 8 +-- !query schema +struct +-- !query output +2 8 + + +-- !query +SELECT t1a, + Max(t1b) +FROM t1 +WHERE t1b > 0 +GROUP BY t1a +HAVING t1a IN (SELECT t2a + FROM t2 + WHERE t2b IN (SELECT t3b + FROM t3 + WHERE t2c = t3c) + ) +-- !query schema +struct +-- !query output +val1b 8 + + +-- !query +SELECT t1a, + t1c, + Min(t1d) +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + GROUP BY t2a + HAVING t2a > 'val2a') +GROUP BY t1a, t1c +HAVING Min(t1d) > t1c +-- !query schema +struct +-- !query output +val1a 8 10 +val1b 16 19 +val1c 16 19 +val1d 16 19 + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + WHERE t1a = t2a + GROUP BY t2c, t2d + HAVING t2c > 8) +GROUP BY t1a, t1b +HAVING t1b < 10 +-- !query schema +struct +-- !query output +val1a 6 + + +-- !query +SELECT t1a, + Max(t1b) +FROM t1 +WHERE t1b > 0 +GROUP BY t1a +HAVING t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b > 3) +-- !query schema +struct +-- !query output +val1a 16 +val1d 10 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-joins.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-joins.sql.out new file mode 100644 index 000000000000..9df4f1ca0875 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-joins.sql.out @@ -0,0 +1,593 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view s1 as select * from values + (1), (3), (5), (7), (9) + as s1(id) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view s2 as select * from values + (1), (3), (4), (6), (9) + as s2(id) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view s3 as select * from values + (3), (4), (6), (9) + as s3(id) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1a, t1b, t1c, t3a, t3b, t3c +FROM t1 natural JOIN t3 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1a = t2a) + AND t1b = t3b + AND t1a = t3a +ORDER BY t1a, + t1b, + t1c DESC nulls first +-- !query schema +struct +-- !query output +val1b 8 16 val1b 8 16 +val1b 8 16 val1b 8 16 + + +-- !query +SELECT Count(DISTINCT(t1a)), + t1b, + t3a, + t3b, + t3c +FROM t1 natural left JOIN t3 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1d = t2d) +AND t1b > t3b +GROUP BY t1a, + t1b, + t3a, + t3b, + t3c +ORDER BY t1a DESC, t3b DESC, t3c ASC +-- !query schema +struct +-- !query output +1 10 val3b 8 NULL +1 10 val1b 8 16 +1 10 val3a 6 12 +1 8 val3a 6 12 +1 8 val3a 6 12 + + +-- !query +SELECT Count(DISTINCT(t1a)) +FROM t1 natural right JOIN t3 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1b = t2b) +AND t1d IN + ( + SELECT t2d + FROM t2 + WHERE t1c > t2c) +AND t1a = t3a +GROUP BY t1a +ORDER BY t1a +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT t1a, + t1b, + t1c, + t3a, + t3b, + t3c +FROM t1 FULL OUTER JOIN t3 +where t1a IN + ( + SELECT t2a + FROM t2 + WHERE t2c IS NOT NULL) +AND t1b != t3b +AND t1a = 'val1b' +ORDER BY t1a +-- !query schema +struct +-- !query output +val1b 8 16 val3a 6 12 +val1b 8 16 val3a 6 12 +val1b 8 16 val1b 10 12 +val1b 8 16 val1b 10 12 +val1b 8 16 val3c 17 16 +val1b 8 16 val3c 17 16 + + +-- !query +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 RIGHT JOIN t3 +where t1a IN + ( + SELECT t2a + FROM t2 + WHERE t2h > t3h) +AND t3a IN + ( + SELECT t2a + FROM t2 + WHERE t2c > t3c) +AND t1h >= t3h +GROUP BY t1a, + t1b +HAVING t1b > 8 +ORDER BY t1a +-- !query schema +struct +-- !query output +1 10 + + +-- !query +SELECT Count(DISTINCT(t1a)) +FROM t1 LEFT OUTER +JOIN t3 +ON t1a = t3a +WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1h < t2h ) +GROUP BY t1a +ORDER BY t1a +-- !query schema +struct +-- !query output +1 +1 +1 + + +-- !query +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 INNER JOIN t2 +ON t1a > t2a +WHERE t1b IN + ( + SELECT t2b + FROM t2 + WHERE t2h > t1h) +OR t1a IN + ( + SELECT t2a + FROM t2 + WHERE t2h < t1h) +GROUP BY t1b +HAVING t1b > 6 +-- !query schema +struct +-- !query output +1 10 +1 8 + + +-- !query +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + JOIN t1 + WHERE t2b <> t1b) +AND t1h IN + ( + SELECT t2h + FROM t2 + RIGHT JOIN t3 + where t2b = t3b) +GROUP BY t1b +HAVING t1b > 8 +-- !query schema +struct +-- !query output +1 10 + + +-- !query +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 +WHERE t1a IN + ( + SELECT t2a + FROM t2 + JOIN t1 + WHERE t2b <> t1b) +AND t1h IN + ( + SELECT t2h + FROM t2 + RIGHT JOIN t3 + where t2b = t3b) +AND t1b IN + ( + SELECT t2b + FROM t2 + FULL OUTER JOIN t3 + where t2b = t3b) + +GROUP BY t1b +HAVING t1b > 8 +-- !query schema +struct +-- !query output +1 10 + + +-- !query +SELECT Count(DISTINCT(t1a)), + t1b +FROM t1 +INNER JOIN t2 on t1b = t2b +RIGHT JOIN t3 ON t1a = t3a +where t1a IN + ( + SELECT t2a + FROM t2 + FULL OUTER JOIN t3 + WHERE t2b > t3b) +AND t1c IN + ( + SELECT t3c + FROM t3 + LEFT OUTER JOIN t2 + ON t3a = t2a ) +AND t1b IN + ( + SELECT t3b + FROM t3 LEFT OUTER + JOIN t1 + WHERE t3c = t1c) + +AND t1a = t2a +GROUP BY t1b +ORDER BY t1b DESC +-- !query schema +struct +-- !query output +1 8 + + +-- !query +SELECT t1a, + t1b, + t1c, + count(distinct(t2a)), + t2b, + t2c +FROM t1 +FULL JOIN t2 on t1a = t2a +RIGHT JOIN t3 on t1a = t3a +where t1a IN + ( + SELECT t2a + FROM t2 INNER + JOIN t3 + ON t2b < t3b + WHERE t2c IN + ( + SELECT t1c + FROM t1 + WHERE t1a = t2a)) +and t1a = t2a +Group By t1a, t1b, t1c, t2a, t2b, t2c +HAVING t2c IS NOT NULL +ORDER By t2b DESC nulls last +-- !query schema +struct +-- !query output +val1b 8 16 1 10 12 +val1b 8 16 1 8 16 +val1b 8 16 1 NULL 16 + + +-- !query +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id IN (SELECT 9) +-- !query schema +struct +-- !query output +9 + + +-- !query +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id NOT IN (SELECT 9) +-- !query schema +struct +-- !query output +1 +3 + + +-- !query +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3) +-- !query schema +struct +-- !query output +3 +9 + + +-- !query +SELECT s1.id AS id2 FROM s1 +LEFT SEMI JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3) +-- !query schema +struct +-- !query output +3 +9 + + +-- !query +SELECT s1.id as id2 FROM s1 +LEFT ANTI JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3) +-- !query schema +struct +-- !query output +1 +5 +7 + + +-- !query +SELECT s1.id, s2.id as id2 FROM s1 +LEFT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3) +-- !query schema +struct +-- !query output +1 NULL +3 3 +5 NULL +7 NULL +9 9 + + +-- !query +SELECT s1.id, s2.id as id2 FROM s1 +RIGHT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3) +-- !query schema +struct +-- !query output +3 3 +9 9 +NULL 1 +NULL 4 +NULL 6 + + +-- !query +SELECT s1.id, s2.id AS id2 FROM s1 +FULL OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id IN (SELECT id FROM s3) +-- !query schema +struct +-- !query output +1 NULL +3 3 +5 NULL +7 NULL +9 9 +NULL 1 +NULL 4 +NULL 6 + + +-- !query +SELECT s1.id FROM s1 +JOIN s2 ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT s1.id AS id2 FROM s1 +LEFT SEMI JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT s1.id AS id2 FROM s1 +LEFT ANTI JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3) +-- !query schema +struct +-- !query output +3 +5 +7 +9 + + +-- !query +SELECT s1.id, s2.id AS id2 FROM s1 +LEFT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3) +-- !query schema +struct +-- !query output +1 1 +3 NULL +5 NULL +7 NULL +9 NULL + + +-- !query +SELECT s1.id, s2.id AS id2 FROM s1 +RIGHT OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3) +-- !query schema +struct +-- !query output +1 1 +NULL 3 +NULL 4 +NULL 6 +NULL 9 + + +-- !query +SELECT s1.id, s2.id AS id2 FROM s1 +FULL OUTER JOIN s2 +ON s1.id = s2.id +AND s1.id NOT IN (SELECT id FROM s3) +-- !query schema +struct +-- !query output +1 1 +3 NULL +5 NULL +7 NULL +9 NULL +NULL 3 +NULL 4 +NULL 6 +NULL 9 + + +-- !query +DROP VIEW s1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW s2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW s3 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-limit.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-limit.sql.out new file mode 100644 index 000000000000..d501c93973a3 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-limit.sql.out @@ -0,0 +1,624 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +LIMIT 2 +-- !query schema +struct +-- !query output +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d + LIMIT 10) +LIMIT 2 +-- !query schema +struct +-- !query output +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +LIMIT 2 +OFFSET 1 +-- !query schema +struct +-- !query output +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d + LIMIT 10 + OFFSET 2) +LIMIT 2 +OFFSET 1 +-- !query schema +struct +-- !query output +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +OFFSET 1 +-- !query schema +struct +-- !query output +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d + OFFSET 2) +OFFSET 1 +-- !query schema +struct +-- !query output +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b >= 8 + LIMIT 2) +LIMIT 4 +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + ORDER BY t2c, t2d + LIMIT 2) +GROUP BY t1b +ORDER BY t1b DESC NULLS FIRST +LIMIT 1 +-- !query schema +struct +-- !query output +1 NULL + + +-- !query +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b <= t1d + LIMIT 2) +LIMIT 4 +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b > 6 + LIMIT 2) +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + ORDER BY t2b DESC nulls first, t2d + LIMIT 1) +GROUP BY t1b +ORDER BY t1b NULLS last +LIMIT 1 +-- !query schema +struct +-- !query output +1 6 + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + WHERE t2b <= t1d + ORDER BY t2c, t2d + LIMIT 2) +GROUP BY t1b +ORDER BY t1b DESC NULLS FIRST +LIMIT 1 +-- !query schema +struct +-- !query output +1 NULL + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +LIMIT 2 +OFFSET 2 +-- !query schema +struct +-- !query output +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d > t2d + ORDER BY t2a DESC + LIMIT 3) +LIMIT 2 +OFFSET 2 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b >= 8 + LIMIT 2 + OFFSET 2) +LIMIT 4 +OFFSET 2 +-- !query schema +struct +-- !query output +val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL +val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + ORDER BY t2c, t2d + LIMIT 2) +GROUP BY t1b +ORDER BY t1b DESC NULLS FIRST +LIMIT 1 +OFFSET 1 +-- !query schema +struct +-- !query output +1 10 + + +-- !query +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b > 6 + LIMIT 2 + OFFSET 2) +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 +val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b <= t1d + LIMIT 2) +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 + + +-- !query +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b = t1b + LIMIT 2 + OFFSET 2) +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 +val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 +val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL +val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + ORDER BY t2b DESC nulls first, t2d + LIMIT 1 + OFFSET 1) +GROUP BY t1b +ORDER BY t1b NULLS last +LIMIT 1 +OFFSET 1 +-- !query schema +struct +-- !query output +2 8 + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + WHERE t2b > t1b + ORDER BY t2b DESC nulls first, t2d + LIMIT 1 + OFFSET 1) +GROUP BY t1b +ORDER BY t1b NULLS last +LIMIT 1 +OFFSET 1 +-- !query schema +struct +-- !query output +2 8 + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d) +OFFSET 2 +-- !query schema +struct +-- !query output +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t1d = t2d + ORDER BY t2a + OFFSET 2) +OFFSET 2 +-- !query schema +struct +-- !query output +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b >= 8 + OFFSET 2) +OFFSET 4 +-- !query schema +struct +-- !query output +val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL +val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL + + +-- !query +SELECT * +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t2b < t1b + ORDER BY t2c + OFFSET 2) +OFFSET 1 +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + ORDER BY t2c, t2d + OFFSET 2) +GROUP BY t1b +ORDER BY t1b DESC NULLS FIRST +OFFSET 1 +-- !query schema +struct +-- !query output +1 10 +2 8 + + +-- !query +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b > 6 + OFFSET 2) +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 + + +-- !query +SELECT count(*) +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b < t1b + OFFSET 2) +-- !query schema +struct +-- !query output +12 + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + ORDER BY t2b DESC nulls first, t2d + OFFSET 1) +GROUP BY t1b +ORDER BY t1b NULLS last +OFFSET 1 +-- !query schema +struct +-- !query output +2 10 +1 16 +1 NULL + + +-- !query +SELECT COUNT(DISTINCT(t1a)) +FROM t1 +WHERE t1d IN (SELECT t2d + FROM t2 + WHERE t1a IS NOT NULL + LIMIT 10) +-- !query schema +struct +-- !query output +4 + + +-- !query +SELECT COUNT(DISTINCT(t1a)) +FROM t1 +WHERE t1d IN (SELECT MAX(t2d) + FROM t2 + WHERE t1a IS NOT NULL + LIMIT 10) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT COUNT(DISTINCT(t1a)) +FROM t1 +WHERE t1d IN (SELECT DISTINCT t2d + FROM t2 + WHERE t1a IS NOT NULL + LIMIT 10) +-- !query schema +struct +-- !query output +4 + + +-- !query +set spark.sql.optimizer.decorrelateExistsIn.enabled = false +-- !query schema +struct +-- !query output +spark.sql.optimizer.decorrelateExistsIn.enabled false + + +-- !query +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT t2b + FROM t2 + WHERE t2b = t1b + LIMIT 2 + OFFSET 2) +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 +val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 +val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL +val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 + + +-- !query +set spark.sql.optimizer.decorrelateExistsIn.enabled = true +-- !query schema +struct +-- !query output +spark.sql.optimizer.decorrelateExistsIn.enabled true diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-multiple-columns.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-multiple-columns.sql.out new file mode 100644 index 000000000000..d37dec0dfbc8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-multiple-columns.sql.out @@ -0,0 +1,175 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1a, + t1b, + t1h +FROM t1 +WHERE ( t1a, t1h ) NOT IN (SELECT t2a, + t2h + FROM t2 + WHERE t2a = t1a + ORDER BY t2a) +AND t1a = 'val1a' +-- !query schema +struct +-- !query output +val1a 16 2014-06-04 01:02:00.001 +val1a 16 2014-07-04 01:01:00 +val1a 6 2014-04-04 01:00:00 +val1a 6 2014-04-04 01:02:00.001 + + +-- !query +SELECT t1a, + t1b, + t1d +FROM t1 +WHERE ( t1b, t1d ) IN (SELECT t2b, + t2d + FROM t2 + WHERE t2i IN (SELECT t3i + FROM t3 + WHERE t2b > t3b)) +-- !query schema +struct +-- !query output +val1e 10 19 +val1e 10 19 + + +-- !query +SELECT t1a, + t1b, + t1d +FROM t1 +WHERE ( t1b, t1d ) NOT IN (SELECT t2b, + t2d + FROM t2 + WHERE t2h IN (SELECT t3h + FROM t3 + WHERE t2b > t3b)) +AND t1a = 'val1a' +-- !query schema +struct +-- !query output +val1a 16 10 +val1a 16 21 +val1a 6 10 +val1a 6 10 + + +-- !query +SELECT t2a +FROM (SELECT t2a + FROM t2 + WHERE ( t2a, t2b ) IN (SELECT t1a, + t1b + FROM t1) + UNION ALL + SELECT t2a + FROM t2 + WHERE ( t2a, t2b ) IN (SELECT t1a, + t1b + FROM t1) + UNION DISTINCT + SELECT t2a + FROM t2 + WHERE ( t2a, t2b ) IN (SELECT t3a, + t3b + FROM t3)) AS t4 +-- !query schema +struct +-- !query output +val1b + + +-- !query +WITH cte1 AS +( + SELECT t1a, + t1b + FROM t1 + WHERE ( + t1b, t1d) IN + ( + SELECT t2b, + t2d + FROM t2 + WHERE t1c = t2c)) +SELECT * +FROM ( + SELECT * + FROM cte1 + JOIN cte1 cte2 + on cte1.t1b = cte2.t1b) s +-- !query schema +struct +-- !query output +val1b 8 val1b 8 +val1b 8 val1c 8 +val1c 8 val1b 8 +val1c 8 val1c 8 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-null-semantics.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-null-semantics.sql.out new file mode 100644 index 000000000000..1337111d6a01 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-null-semantics.sql.out @@ -0,0 +1,298 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temp view v (c) as values (1), (null) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temp view v_empty (e) as select 1 where false +-- !query schema +struct<> +-- !query output + + + +-- !query +create table t(c int) using json +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t values (1), (null) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table t2(d int) using json +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into t2 values (2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table t_empty(e int) using json +-- !query schema +struct<> +-- !query output + + + +-- !query +set spark.sql.legacy.nullInEmptyListBehavior = false +-- !query schema +struct +-- !query output +spark.sql.legacy.nullInEmptyListBehavior false + + +-- !query +select c, c in (select e from t_empty) from t +-- !query schema +struct +-- !query output +1 false +NULL false + + +-- !query +select c, c in (select e from v_empty) from v +-- !query schema +struct +-- !query output +1 false +NULL false + + +-- !query +select c, c not in (select e from t_empty) from t +-- !query schema +struct +-- !query output +1 true +NULL true + + +-- !query +select c, c not in (select e from v_empty) from v +-- !query schema +struct +-- !query output +1 true +NULL true + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding + + +-- !query +select null in (select e from t_empty) +-- !query schema +struct<(NULL IN (listquery())):boolean> +-- !query output +false + + +-- !query +select null in (select e from v_empty) +-- !query schema +struct<(NULL IN (listquery())):boolean> +-- !query output +false + + +-- !query +select null not in (select e from t_empty) +-- !query schema +struct<(NOT (NULL IN (listquery()))):boolean> +-- !query output +true + + +-- !query +select null not in (select e from v_empty) +-- !query schema +struct<(NOT (NULL IN (listquery()))):boolean> +-- !query output +true + + +-- !query +select * from t left join t2 on (t.c in (select e from t_empty)) is null +-- !query schema +struct +-- !query output +1 NULL +NULL NULL + + +-- !query +select * from t left join t2 on (t.c not in (select e from t_empty)) is null +-- !query schema +struct +-- !query output +1 NULL +NULL NULL + + +-- !query +set spark.sql.optimizer.optimizeUncorrelatedInSubqueriesInJoinCondition.enabled=true +-- !query schema +struct +-- !query output +spark.sql.optimizer.optimizeUncorrelatedInSubqueriesInJoinCondition.enabled true + + +-- !query +select * from t left join t2 on (t.c in (select e from t_empty)) is null +-- !query schema +struct +-- !query output +1 NULL +NULL NULL + + +-- !query +select * from t left join t2 on (t.c not in (select e from t_empty)) is null +-- !query schema +struct +-- !query output +1 NULL +NULL NULL + + +-- !query +set spark.sql.legacy.nullInEmptyListBehavior = true +-- !query schema +struct +-- !query output +spark.sql.legacy.nullInEmptyListBehavior true + + +-- !query +set spark.sql.optimizer.optimizeUncorrelatedInSubqueriesInJoinCondition.enabled=false +-- !query schema +struct +-- !query output +spark.sql.optimizer.optimizeUncorrelatedInSubqueriesInJoinCondition.enabled false + + +-- !query +select null in (select e from t_empty) +-- !query schema +struct<(NULL IN (listquery())):boolean> +-- !query output +NULL + + +-- !query +select null in (select e from v_empty) +-- !query schema +struct<(NULL IN (listquery())):boolean> +-- !query output +NULL + + +-- !query +select null not in (select e from t_empty) +-- !query schema +struct<(NOT (NULL IN (listquery()))):boolean> +-- !query output +NULL + + +-- !query +select null not in (select e from v_empty) +-- !query schema +struct<(NOT (NULL IN (listquery()))):boolean> +-- !query output +NULL + + +-- !query +set spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation +-- !query schema +struct +-- !query output +spark.sql.optimizer.excludedRules org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation,org.apache.spark.sql.catalyst.optimizer.ConstantFolding,org.apache.spark.sql.catalyst.optimizer.NullPropagation + + +-- !query + +select * from t left join t2 on (t.c in (select e from t_empty)) is null +-- !query schema +struct +-- !query output +1 NULL +NULL 2 + + +-- !query +select * from t left join t2 on (t.c not in (select e from t_empty)) is null +-- !query schema +struct +-- !query output +1 NULL +NULL 2 + + +-- !query +reset spark.sql.legacy.nullInEmptyListBehavior +-- !query schema +struct<> +-- !query output + + + +-- !query +reset spark.sql.optimizer.optimizeUncorrelatedInSubqueriesInJoinCondition.enabled +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t_empty +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-nullability.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-nullability.sql.out new file mode 100644 index 000000000000..4c8e9b300b82 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-nullability.sql.out @@ -0,0 +1,71 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temp view t0 as select 1 as a_nonnullable +-- !query schema +struct<> +-- !query output + + + +-- !query +create temp view t1 as select cast(null as int) as b_nullable +-- !query schema +struct<> +-- !query output + + + +-- !query +create temp view t2 as select 2 as c +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from t0 where a_nonnullable in (select b_nullable from t1) +-- !query schema +struct +-- !query output + + + +-- !query +select * from t0 where (a_nonnullable in (select b_nullable from t1)) <=> true +-- !query schema +struct +-- !query output + + + +-- !query +select * from t0 where a_nonnullable not in (select b_nullable from t1) +-- !query schema +struct +-- !query output + + + +-- !query +select * from t0 where (a_nonnullable not in (select b_nullable from t1)) <=> true +-- !query schema +struct +-- !query output + + + +-- !query +select * from t0 left join t2 on (a_nonnullable IN (select b_nullable from t1)) is null +-- !query schema +struct +-- !query output +1 2 + + +-- !query +select * from t0 left join t2 on (a_nonnullable IN (select b_nullable from t1)) <=> true +-- !query schema +struct +-- !query output +1 NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-order-by.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-order-by.sql.out new file mode 100644 index 000000000000..d687b5938834 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-order-by.sql.out @@ -0,0 +1,325 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2) +ORDER BY t1a +-- !query schema +struct +-- !query output +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 +val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 + + +-- !query +SELECT t1a +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a) +ORDER BY t1b DESC +-- !query schema +struct +-- !query output +val1b + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a) +ORDER BY 2 DESC nulls last +-- !query schema +struct +-- !query output +val1b 8 +val1c 8 + + +-- !query +SELECT Count(DISTINCT( t1a )) +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a) +ORDER BY Count(DISTINCT( t1a )) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * +FROM t1 +WHERE t1b IN (SELECT t2c + FROM t2 + ORDER BY t2d) +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1b = t2b + ORDER BY Min(t2b)) +ORDER BY t1c DESC nulls first, t1a DESC, t1d DESC, t1h +-- !query schema +struct +-- !query output +val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 + + +-- !query +SELECT t1a, + t1b, + t1h +FROM t1 +WHERE t1c IN (SELECT t2c + FROM t2 + WHERE t1a = t2a + ORDER BY t2b DESC nulls first) + OR t1h IN (SELECT t2h + FROM t2 + WHERE t1h > t2h) +ORDER BY t1h DESC nulls last +-- !query schema +struct +-- !query output +val1c 8 2014-05-04 01:02:00.001 +val1b 8 2014-05-04 01:01:00 + + +-- !query +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2) +ORDER BY t1a +-- !query schema +struct +-- !query output +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 +val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL +val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL +val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t1a = t2a) +ORDER BY t1b DESC nulls last +-- !query schema +struct +-- !query output +val1a 16 +val1a 16 +val1d 10 +val1a 6 +val1a 6 +val1d NULL +val1d NULL + + +-- !query +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + ORDER BY t2a DESC nulls first) + and t1c IN (SELECT t2c + FROM t2 + ORDER BY t2b DESC nulls last) +ORDER BY t1c DESC nulls last +-- !query schema +struct +-- !query output +val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL +val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + GROUP BY t2a + ORDER BY t2a DESC) +-- !query schema +struct +-- !query output +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 + + +-- !query +SELECT t1a, + Count(DISTINCT( t1b )) +FROM t1 +WHERE t1b IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2a + ORDER BY t2a) +GROUP BY t1a, + t1h +ORDER BY t1a +-- !query schema +struct +-- !query output +val1b 1 + + +-- !query +SELECT * +FROM t1 +WHERE t1b NOT IN (SELECT Min(t2b) + FROM t2 + GROUP BY t2a + ORDER BY t2a) +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 + + +-- !query +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1b NOT IN (SELECT Min(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2c + ORDER BY t2c DESC nulls last) +GROUP BY t1a +-- !query schema +struct +-- !query output +val1a 22 +val1c 8 +val1d 10 +val1e 10 + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b +FROM t1 +WHERE t1h NOT IN (SELECT t2h + FROM t2 + where t1a = t2a + order by t2d DESC nulls first + ) +GROUP BY t1a, + t1b +ORDER BY t1b DESC nulls last +-- !query schema +struct +-- !query output +1 16 +1 10 +1 10 +1 8 +1 6 +1 NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out new file mode 100644 index 000000000000..13e1d8b56257 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-set-operations.sql.out @@ -0,0 +1,834 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t2a, + t2b, + t2c, + t2h, + t2i +FROM (SELECT * + FROM t2 + WHERE t2a IN (SELECT t1a + FROM t1) + UNION ALL + SELECT * + FROM t3 + WHERE t3a IN (SELECT t1a + FROM t1)) AS t3 +WHERE t2i IS NOT NULL AND + 2 * t2b = t2c +ORDER BY t2c DESC nulls first +-- !query schema +struct +-- !query output +val1b 8 16 2015-05-04 01:01:00 2015-05-04 +val1b 8 16 2014-07-04 01:01:00 2014-07-04 +val1b 8 16 2014-06-04 01:02:00 2014-06-04 +val1b 8 16 2014-07-04 01:02:00 2014-07-04 + + +-- !query +SELECT t2a, + t2b, + t2d, + Count(DISTINCT( t2h )), + t2i +FROM (SELECT * + FROM t2 + WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t2b = t1b) + UNION + SELECT * + FROM t1 + WHERE t1a IN (SELECT t3a + FROM t3 + WHERE t1c = t3c)) AS t3 +GROUP BY t2a, + t2b, + t2d, + t2i +ORDER BY t2d DESC +-- !query schema +struct +-- !query output +val1b 8 119 1 2015-05-04 +val1b 8 19 1 2014-07-04 +val1b 8 19 1 2014-05-04 + + +-- !query +SELECT t2a, + t2b, + t2c, + Min(t2d) +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b) +GROUP BY t2a, t2b, t2c +UNION ALL +SELECT t2a, + t2b, + t2c, + Max(t2d) +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t2c = t1c) +GROUP BY t2a, t2b, t2c +UNION +SELECT t3a, + t3b, + t3c, + Min(t3d) +FROM t3 +WHERE t3a IN (SELECT t2a + FROM t2 + WHERE t3c = t2c) +GROUP BY t3a, t3b, t3c +UNION DISTINCT +SELECT t1a, + t1b, + t1c, + Max(t1d) +FROM t1 +WHERE t1a IN (SELECT t3a + FROM t3 + WHERE t3d = t1d) +GROUP BY t1a, t1b, t1c +-- !query schema +struct +-- !query output +val1b 10 12 19 +val1b 8 16 119 +val1b 8 16 19 +val1b NULL 16 19 +val1b NULL 16 319 +val1c 12 16 219 + + +-- !query +SELECT DISTINCT( t2a ), + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i +UNION +SELECT DISTINCT( t2a ), + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t2c = t1c) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i +HAVING t2b IS NOT NULL +-- !query schema +struct +-- !query output +val1b 8 1 119 2015-05-04 01:01:00 2015-05-04 +val1b 8 1 19 2014-07-04 01:01:00 2014-07-04 +val1c 12 1 19 2014-08-04 01:01:00 2014-08-05 +val1c 12 1 219 2016-05-04 01:01:00 2016-05-04 + + +-- !query +SELECT t2a, + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2a IN (SELECT DISTINCT(t1a) + FROM t1 + WHERE t1b = t2b) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i + +UNION +SELECT DISTINCT( t2a ), + t2b, + Count(t2c), + t2d, + t2h, + t2i +FROM t2 +WHERE t2b IN (SELECT Max(t1b) + FROM t1 + WHERE t2c = t1c) +GROUP BY t2a, + t2b, + t2c, + t2d, + t2h, + t2i +HAVING t2b IS NOT NULL +UNION DISTINCT +SELECT t2a, + t2b, + t2c, + t2d, + t2h, + t2i +FROM t2 +WHERE t2d IN (SELECT min(t1d) + FROM t1 + WHERE t2c = t1c) +-- !query schema +struct +-- !query output +val1b 8 1 119 2015-05-04 01:01:00 2015-05-04 +val1b 8 1 19 2014-07-04 01:01:00 2014-07-04 +val1b 8 16 19 2014-07-04 01:01:00 2014-07-04 +val1b NULL 16 19 2014-05-04 01:01:00 NULL +val1c 12 16 19 2014-08-04 01:01:00 2014-08-05 + + +-- !query +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2a IN (SELECT t1a + FROM t1 + WHERE t1b = t2b AND + t1d < t2d) +INTERSECT +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2b IN (SELECT Max(t1b) + FROM t1 + WHERE t2c = t1c) +EXCEPT +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2d IN (SELECT Min(t3d) + FROM t3 + WHERE t2c = t3c) +UNION ALL +SELECT t2a, + t2b, + t2c, + t2d +FROM t2 +WHERE t2c IN (SELECT Max(t1c) + FROM t1 + WHERE t1d = t2d) +-- !query schema +struct +-- !query output +val1b 8 16 119 +val1b 8 16 19 +val1b NULL 16 19 +val1c 12 16 19 + + +-- !query +SELECT DISTINCT(t1a), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a IN (SELECT t3a + FROM (SELECT t2a t3a + FROM t2 + UNION ALL + SELECT t2a t3a + FROM t2) AS t3 + UNION + SELECT t2a + FROM (SELECT t2a + FROM t2 + WHERE t2b > 6 + UNION + SELECT t2a + FROM t2 + WHERE t2b > 6) AS t4 + UNION DISTINCT + SELECT t2a + FROM (SELECT t2a + FROM t2 + WHERE t2b > 6 + UNION DISTINCT + SELECT t1a + FROM t1 + WHERE t1b > 6) AS t5) +GROUP BY t1a, t1b, t1c, t1d +HAVING t1c IS NOT NULL AND t1b IS NOT NULL +ORDER BY t1c DESC, t1a DESC +-- !query schema +struct +-- !query output +val1c 8 16 19 +val1b 8 16 19 +val1a 16 12 21 +val1a 16 12 10 +val1a 6 8 10 + + +-- !query +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN (SELECT t2b + FROM (SELECT t2b + FROM t2 + WHERE t2b > 6 + INTERSECT + SELECT t1b + FROM t1 + WHERE t1b > 6) AS t3 + WHERE t2b = t1b) +-- !query schema +struct +-- !query output +val1b 8 16 +val1c 8 16 +val1d 10 NULL +val1e 10 NULL +val1e 10 NULL +val1e 10 NULL + + +-- !query +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1h IN (SELECT t2h + FROM (SELECT t2h + FROM t2 + EXCEPT + SELECT t3h + FROM t3) AS t3) +ORDER BY t1b DESC NULLs first, t1c DESC NULLs last +-- !query schema +struct +-- !query output +val1d NULL 16 +val1a 16 12 +val1e 10 NULL +val1d 10 NULL +val1e 10 NULL +val1b 8 16 + + +-- !query +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN + ( + SELECT t2b + FROM ( + SELECT t2b + FROM t2 + WHERE t2b > 6 + INTERSECT + SELECT t1b + FROM t1 + WHERE t1b > 6) AS t3) +UNION DISTINCT +SELECT t1a, + t1b, + t1c +FROM t1 +WHERE t1b IN + ( + SELECT t2b + FROM ( + SELECT t2b + FROM t2 + WHERE t2b > 6 + EXCEPT + SELECT t1b + FROM t1 + WHERE t1b > 6) AS t4 + WHERE t2b = t1b) +ORDER BY t1c DESC NULLS last, t1a DESC +-- !query schema +struct +-- !query output +val1c 8 16 +val1b 8 16 +val1e 10 NULL +val1d 10 NULL + + +-- !query +SELECT * +FROM (SELECT * + FROM (SELECT * + FROM t2 + WHERE t2h IN (SELECT t1h + FROM t1 + WHERE t1a = t2a) + UNION DISTINCT + SELECT * + FROM t1 + WHERE t1h IN (SELECT t3h + FROM t3 + UNION + SELECT t1h + FROM t1) + UNION + SELECT * + FROM t3 + WHERE t3a IN (SELECT t2a + FROM t2 + UNION ALL + SELECT t1a + FROM t1 + WHERE t1b > 0) + INTERSECT + SELECT * + FROM T1 + WHERE t1b IN (SELECT t3b + FROM t3 + UNION DISTINCT + SELECT t2b + FROM t2 + ) + EXCEPT + SELECT * + FROM t2 + WHERE t2h IN (SELECT t1i + FROM t1)) t4 + WHERE t4.t2b IN (SELECT Min(t3b) + FROM t3 + WHERE t4.t2a = t3a)) +-- !query schema +struct +-- !query output +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 + + +-- !query +SELECT t2a, + t2b, + t2c, + t2i +FROM (SELECT * + FROM t2 + WHERE t2a NOT IN (SELECT t1a + FROM t1 + UNION + SELECT t3a + FROM t3) + UNION ALL + SELECT * + FROM t2 + WHERE t2a NOT IN (SELECT t1a + FROM t1 + INTERSECT + SELECT t2a + FROM t2)) AS t3 +WHERE t3.t2a NOT IN (SELECT t1a + FROM t1 + INTERSECT + SELECT t2a + FROM t2) + AND t2c IS NOT NULL +ORDER BY t2a +-- !query schema +struct +-- !query output +val2a 6 12 2014-04-04 +val2a 6 12 2014-04-04 + + +-- !query +SELECT Count(DISTINCT(t1a)), + t1b, + t1c, + t1i +FROM t1 +WHERE t1b NOT IN + ( + SELECT t2b + FROM ( + SELECT t2b + FROM t2 + WHERE t2b NOT IN + ( + SELECT t1b + FROM t1) + UNION + SELECT t1b + FROM t1 + WHERE t1b NOT IN + ( + SELECT t3b + FROM t3) + UNION + distinct SELECT t3b + FROM t3 + WHERE t3b NOT IN + ( + SELECT t2b + FROM t2)) AS t3 + WHERE t2b = t1b) +GROUP BY t1a, + t1b, + t1c, + t1i +HAVING t1b NOT IN + ( + SELECT t2b + FROM t2 + WHERE t2c IS NULL + EXCEPT + SELECT t3b + FROM t3) +ORDER BY t1c DESC NULLS LAST, t1i +-- !query schema +struct +-- !query output +1 8 16 2014-05-04 +1 8 16 2014-05-05 +1 16 12 2014-06-04 +1 16 12 2014-07-04 +1 6 8 2014-04-04 +1 10 NULL 2014-05-04 +1 10 NULL 2014-08-04 +1 10 NULL 2014-09-04 +1 10 NULL 2015-05-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + UNION ALL + SELECT t3a + FROM t3) +-- !query schema +struct +-- !query output +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + UNION DISTINCT + SELECT t3a + FROM t3) +-- !query schema +struct +-- !query output +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + INTERSECT ALL + SELECT t3a + FROM t3) +-- !query schema +struct +-- !query output +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + INTERSECT DISTINCT + SELECT t3a + FROM t3) +-- !query schema +struct +-- !query output +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + EXCEPT ALL + SELECT t3a + FROM t3) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + EXCEPT DISTINCT + SELECT t3a + FROM t3) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + UNION ALL + SELECT t3a + FROM t3) +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 +val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 +val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL +val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + UNION DISTINCT + SELECT t3a + FROM t3) +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 +val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 +val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL +val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + INTERSECT ALL + SELECT t3a + FROM t3) +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 +val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 +val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL +val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + INTERSECT DISTINCT + SELECT t3a + FROM t3) +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 +val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 +val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL +val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + EXCEPT ALL + SELECT t3a + FROM t3) +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 +val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 +val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL +val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + WHERE t2b = t1b + EXCEPT DISTINCT + SELECT t3a + FROM t3) +-- !query schema +struct +-- !query output +val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 +val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 +val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 +val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 +val1d 10 NULL 12 17.0 25.0 2600 2015-05-04 01:01:00 2015-05-04 +val1d NULL 16 19 17.0 25.0 2600 2014-07-04 01:02:00.001 NULL +val1d NULL 16 22 17.0 25.0 2600 2014-06-04 01:01:00 NULL +val1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +val1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +val1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-subquery-in-join-condition.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-subquery-in-join-condition.sql.out new file mode 100644 index 000000000000..13af4c81173a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-subquery-in-join-condition.sql.out @@ -0,0 +1,466 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMP VIEW x(x1, x2) AS VALUES + (2, 1), + (1, 1), + (3, 4) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMP VIEW y(y1, y2) AS VALUES + (0, 2), + (1, 4), + (4, 11) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMP VIEW z(z1, z2) AS VALUES + (4, 2), + (3, 3), + (8, 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from x inner join y on x1 = y1 and x2 IN (select z1 from z where z2 = x2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output + + + +-- !query +select * from x inner join y on x1 = y1 and x2 not IN (select z1 from z where z2 = x2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 + + +-- !query +select * from x inner join y on x1 = y1 and y2 IN (select z1 from z where z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output + + + +-- !query +select * from x inner join y on x1 = y1 and y2 not IN (select z1 from z where z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 + + +-- !query +select * from x left join y on x1 = y1 and x2 IN (select z1 from z where z2 = x2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 NULL NULL +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x left join y on x1 = y1 and x2 not IN (select z1 from z where z2 = x2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x left join y on x1 = y1 and y2 IN (select z1 from z where z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 NULL NULL +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x left join y on x1 = y1 and y2 not IN (select z1 from z where z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x right join y on x1 = y1 and x2 IN (select z1 from z where z2 = x2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 1 4 +NULL NULL 4 11 + + +-- !query +select * from x right join y on x1 = y1 and x2 not IN (select z1 from z where z2 = x2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 4 11 +1 1 1 4 + + +-- !query +select * from x right join y on x1 = y1 and y2 IN (select z1 from z where z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 1 4 +NULL NULL 4 11 + + +-- !query +select * from x right join y on x1 = y1 and y2 not IN (select z1 from z where z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 4 11 +1 1 1 4 + + +-- !query +select * from x left semi join y on x1 = y1 and x2 IN (select z1 from z where z2 = x2) order by x1, x2 +-- !query schema +struct +-- !query output + + + +-- !query +select * from x left semi join y on x1 = y1 and x2 not IN (select z1 from z where z2 = x2) order by x1, x2 +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select * from x left semi join y on x1 = y1 and y2 IN (select z1 from z where z2 = y2) order by x1, x2 +-- !query schema +struct +-- !query output + + + +-- !query +select * from x left semi join y on x1 = y1 and y2 not IN (select z1 from z where z2 = y2) order by x1, x2 +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select * from x left anti join y on x1 = y1 and x2 IN (select z1 from z where z2 = x2) order by x1, x2 +-- !query schema +struct +-- !query output +1 1 +2 1 +3 4 + + +-- !query +select * from x left anti join y on x1 = y1 and x2 not IN (select z1 from z where z2 = x2) order by x1, x2 +-- !query schema +struct +-- !query output +2 1 +3 4 + + +-- !query +select * from x left anti join y on x1 = y1 and y2 IN (select z1 from z where z2 = y2) order by x1, x2 +-- !query schema +struct +-- !query output +1 1 +2 1 +3 4 + + +-- !query +select * from x left anti join y on x1 = y1 and y2 not IN (select z1 from z where z2 = y2) order by x1, x2 +-- !query schema +struct +-- !query output +2 1 +3 4 + + +-- !query +select * from x full outer join y on x1 = y1 and x2 IN (select z1 from z where z2 = x2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 1 4 +NULL NULL 4 11 +1 1 NULL NULL +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x full outer join y on x1 = y1 and x2 not IN (select z1 from z where z2 = x2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 4 11 +1 1 1 4 +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x full outer join y on x1 = y1 and y2 IN (select z1 from z where z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 1 4 +NULL NULL 4 11 +1 1 NULL NULL +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x full outer join y on x1 = y1 and y2 not IN (select z1 from z where z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +NULL NULL 0 2 +NULL NULL 4 11 +1 1 1 4 +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x inner join y on x1 = y1 or x2 IN (select z1 from z where z1 = x1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 + + +-- !query +select * from x inner join y on x1 = y1 or x2 not IN (select z1 from z where z1 = x1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 0 2 +1 1 1 4 +1 1 4 11 +2 1 0 2 +2 1 1 4 +2 1 4 11 +3 4 0 2 +3 4 1 4 +3 4 4 11 + + +-- !query +select * from x left join y on x1 = y1 or x2 IN (select z1 from z where z1 = x1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x left join y on x1 = y1 or x2 not IN (select z1 from z where z1 = x1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 0 2 +1 1 1 4 +1 1 4 11 +2 1 0 2 +2 1 1 4 +2 1 4 11 +3 4 0 2 +3 4 1 4 +3 4 4 11 + + +-- !query +select * from x inner join y on x1 = y1 or y2 IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 + + +-- !query +select * from x inner join y on x1 = y1 or y2 not IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 0 2 +1 1 1 4 +1 1 4 11 +2 1 0 2 +2 1 1 4 +2 1 4 11 +3 4 0 2 +3 4 1 4 +3 4 4 11 + + +-- !query +select * from x left join y on x1 = y1 or y2 IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x left join y on x1 = y1 or y2 not IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 0 2 +1 1 1 4 +1 1 4 11 +2 1 0 2 +2 1 1 4 +2 1 4 11 +3 4 0 2 +3 4 1 4 +3 4 4 11 + + +-- !query +select * from x inner join y on x1 = y1 and x2 IN (select z1 from z where z1 = x1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output + + + +-- !query +select * from x inner join y on x1 = y1 and x2 not IN (select z1 from z where z1 = x1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 + + +-- !query +select * from x left join y on x1 = y1 and x2 IN (select z1 from z where z1 = x1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 NULL NULL +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x left join y on x1 = y1 and x2 not IN (select z1 from z where z1 = x1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x inner join y on x1 = y1 and y2 IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output + + + +-- !query +select * from x inner join y on x1 = y1 and y2 not IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 + + +-- !query +select * from x left join y on x1 = y1 and y2 IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 NULL NULL +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x left join y on x1 = y1 and y2 not IN (select z1 from z where z1 = y1) order by x1, x2, y1, y2 +-- !query schema +struct +-- !query output +1 1 1 4 +2 1 NULL NULL +3 4 NULL NULL + + +-- !query +select * from x left join y on x1 = y1 and x2 IN (select z1 from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_EXPRESSION_IN_JOIN_CONDITION", + "sqlState" : "0A000", + "messageParameters" : { + "subqueryExpression" : "(x.x2 IN (listquery(x.x2, y.y2, (z.z2 = x.x2), (z.z2 = y.y2))))" + } +} + + +-- !query +select * from x left join y on x1 = y1 and x2 not IN (select z1 from z where z2 = x2 AND z2 = y2) order by x1, x2, y1, y2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_EXPRESSION_IN_JOIN_CONDITION", + "sqlState" : "0A000", + "messageParameters" : { + "subqueryExpression" : "(x.x2 IN (listquery(x.x2, y.y2, (z.z2 = x.x2), (z.z2 = y.y2))))" + } +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-with-cte.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-with-cte.sql.out new file mode 100644 index 000000000000..562b8302e1c3 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/in-with-cte.sql.out @@ -0,0 +1,361 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1a = "val1a") +SELECT t1a, + t1b, + t1c, + t1d, + t1h +FROM t1 +WHERE t1b IN (SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b > 0) +-- !query schema +struct +-- !query output +val1a 16 12 10 2014-07-04 01:01:00 +val1a 16 12 21 2014-06-04 01:02:00.001 +val1a 6 8 10 2014-04-04 01:00:00 +val1a 6 8 10 2014-04-04 01:02:00.001 + + +-- !query +WITH cte1 AS +( + SELECT t1a, + t1b + FROM t1) +SELECT count(distinct(t1a)), t1b, t1c +FROM t1 +WHERE t1b IN + ( + SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b > 0 + UNION + SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b > 5 + UNION ALL + SELECT cte1.t1b + FROM cte1 + INTERSECT + SELECT cte1.t1b + FROM cte1 + UNION + SELECT cte1.t1b + FROM cte1 ) +GROUP BY t1a, t1b, t1c +HAVING t1c IS NOT NULL +-- !query schema +struct +-- !query output +1 16 12 +1 6 8 +1 8 16 +1 8 16 + + +-- !query +WITH cte1 AS +( + SELECT t1a, + t1b, + t1c, + t1d, + t1e + FROM t1) +SELECT t1a, + t1b, + t1c, + t1h +FROM t1 +WHERE t1c IN + ( + SELECT cte1.t1c + FROM cte1 + JOIN cte1 cte2 + on cte1.t1b > cte2.t1b + FULL OUTER JOIN cte1 cte3 + ON cte1.t1c = cte3.t1c + LEFT JOIN cte1 cte4 + ON cte1.t1d = cte4.t1d + INNER JOIN cte1 cte5 + ON cte1.t1b < cte5.t1b + LEFT OUTER JOIN cte1 cte6 + ON cte1.t1d > cte6.t1d) +-- !query schema +struct +-- !query output +val1b 8 16 2014-05-04 01:01:00 +val1c 8 16 2014-05-04 01:02:00.001 +val1d NULL 16 2014-06-04 01:01:00 +val1d NULL 16 2014-07-04 01:02:00.001 + + +-- !query +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1b IN (SELECT t2b + FROM t2 + RIGHT JOIN t1 + ON t1c = t2c + LEFT JOIN t3 + ON t2d = t3d) + AND t1a = "val1b") +SELECT * +FROM (SELECT * + FROM cte1 + JOIN cte1 cte2 + ON cte1.t1b > 5 + AND cte1.t1a = cte2.t1a + FULL OUTER JOIN cte1 cte3 + ON cte1.t1a = cte3.t1a + INNER JOIN cte1 cte4 + ON cte1.t1b = cte4.t1b) s +-- !query schema +struct +-- !query output +val1b 8 val1b 8 val1b 8 val1b 8 + + +-- !query +WITH cte1 AS +( + SELECT t1a, + t1b, + t1h + FROM t1 + WHERE t1a IN + ( + SELECT t2a + FROM t2 + WHERE t1b < t2b)) +SELECT Count(DISTINCT t1a), + t1b +FROM ( + SELECT cte1.t1a, + cte1.t1b + FROM cte1 + JOIN cte1 cte2 + on cte1.t1h >= cte2.t1h) s +WHERE t1b IN + ( + SELECT t1b + FROM t1) +GROUP BY t1b +-- !query schema +struct +-- !query output +2 8 + + +-- !query +WITH cte1 AS +( + SELECT t1a, + t1b, + t1c + FROM t1 + WHERE t1b IN + ( + SELECT t2b + FROM t2 FULL OUTER JOIN T3 on t2a = t3a + WHERE t1c = t2c) AND + t1a = "val1b") +SELECT * +FROM ( + SELECT * + FROM cte1 + INNER JOIN cte1 cte2 ON cte1.t1a = cte2.t1a + RIGHT OUTER JOIN cte1 cte3 ON cte1.t1b = cte3.t1b + LEFT OUTER JOIN cte1 cte4 ON cte1.t1c = cte4.t1c + ) s +-- !query schema +struct +-- !query output +val1b 8 16 val1b 8 16 val1b 8 16 val1b 8 16 + + +-- !query +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1c = t2c)) +SELECT Count(DISTINCT( s.t1a )), + s.t1b +FROM (SELECT cte1.t1a, + cte1.t1b + FROM cte1 + RIGHT OUTER JOIN cte1 cte2 + ON cte1.t1a = cte2.t1a) s +GROUP BY s.t1b +-- !query schema +struct +-- !query output +2 8 + + +-- !query +WITH cte1 AS +( + SELECT t1a, + t1b + FROM t1 + WHERE t1b IN + ( + SELECT t2b + FROM t2 + WHERE t1c = t2c)) +SELECT DISTINCT(s.t1b) +FROM ( + SELECT cte1.t1b + FROM cte1 + LEFT OUTER JOIN cte1 cte2 + ON cte1.t1b = cte2.t1b) s +WHERE s.t1b IN + ( + SELECT t1.t1b + FROM t1 INNER + JOIN cte1 + ON t1.t1a = cte1.t1a) +-- !query schema +struct +-- !query output +8 + + +-- !query +WITH cte1 + AS (SELECT t1a, + t1b + FROM t1 + WHERE t1a = "val1d") +SELECT t1a, + t1b, + t1c, + t1h +FROM t1 +WHERE t1b NOT IN (SELECT cte1.t1b + FROM cte1 + WHERE cte1.t1b < 0) AND + t1c > 10 +-- !query schema +struct +-- !query output +val1a 16 12 2014-06-04 01:02:00.001 +val1a 16 12 2014-07-04 01:01:00 +val1b 8 16 2014-05-04 01:01:00 +val1c 8 16 2014-05-04 01:02:00.001 +val1d NULL 16 2014-06-04 01:01:00 +val1d NULL 16 2014-07-04 01:02:00.001 + + +-- !query +WITH cte1 AS +( + SELECT t1a, + t1b, + t1c, + t1d, + t1h + FROM t1 + WHERE t1d NOT IN + ( + SELECT t2d + FROM t2 + FULL OUTER JOIN t3 ON t2a = t3a + JOIN t1 on t1b = t2b)) +SELECT t1a, + t1b, + t1c, + t1d, + t1h +FROM t1 +WHERE t1b NOT IN + ( + SELECT cte1.t1b + FROM cte1 INNER + JOIN cte1 cte2 ON cte1.t1a = cte2.t1a + RIGHT JOIN cte1 cte3 ON cte1.t1b = cte3.t1b + JOIN cte1 cte4 ON cte1.t1c = cte4.t1c) AND + t1c IS NOT NULL +ORDER BY t1c DESC +-- !query schema +struct +-- !query output +val1b 8 16 19 2014-05-04 01:01:00 +val1c 8 16 19 2014-05-04 01:02:00.001 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/nested-not-in.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/nested-not-in.sql.out new file mode 100644 index 000000000000..874e7a602e76 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/nested-not-in.sql.out @@ -0,0 +1,329 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", 10), + (200, "emp 2", NULL), + (300, "emp 3", 20), + (400, "emp 4", 30), + (500, "emp 5", NULL), + (600, "emp 6", 100), + (800, "emp 8", 70) +AS EMP(id, emp_name, dept_id) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES + ("emp 1", 10.00D), + ("emp 1", 20.00D), + ("emp 2", 300.00D), + ("emp 2", 100.00D), + ("emp 3", 300.00D), + ("emp 4", 100.00D), + ("emp 5", 1000.00D), + ("emp 6 - no dept", 500.00D) +AS BONUS(emp_name, bonus_amt) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW ADDRESS AS SELECT * FROM VALUES + (100, "emp 1", "addr1"), + (200, null, "addr2"), + (null, "emp 3", "addr3"), + (null, null, "addr4"), + (600, "emp 6", "addr6"), + (800, "emp 8", "addr8") +AS ADDRESS(id, emp_name, address) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW S1 AS SELECT * FROM VALUES + (null, null), (5, 5), (8, 8), (11, 11) AS s1(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW S2 AS SELECT * FROM VALUES + (7, 7), (8, 8), (11, 11), (null, null) AS s2(c, d) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT id, + dept_id +FROM emp +WHERE id = 600 + OR id = 500 + OR dept_id NOT IN (SELECT dept_id + FROM emp) +-- !query schema +struct +-- !query output +500 NULL +600 100 + + +-- !query +SELECT id, + dept_id +FROM emp +WHERE id = 800 + OR (dept_id IS NOT NULL + AND dept_id NOT IN (SELECT dept_id + FROM emp)) +-- !query schema +struct +-- !query output +800 70 + + +-- !query +SELECT id, + dept_id +FROM emp +WHERE id = 100 + OR dept_id NOT IN (SELECT dept_id + FROM emp + WHERE dept_id IS NOT NULL) +-- !query schema +struct +-- !query output +100 10 + + +-- !query +SELECT id, + dept_id +FROM emp +WHERE id = 200 + OR (dept_id IS NOT NULL + AND dept_id + 100 NOT IN (SELECT dept_id + FROM emp + WHERE dept_id IS NOT NULL)) +-- !query schema +struct +-- !query output +100 10 +200 NULL +300 20 +400 30 +600 100 +800 70 + + +-- !query +SELECT id, + dept_id, + emp_name +FROM emp +WHERE emp_name IN (SELECT emp_name + FROM bonus) + OR (dept_id IS NOT NULL + AND dept_id NOT IN (SELECT dept_id + FROM dept)) +-- !query schema +struct +-- !query output +100 10 emp 1 +200 NULL emp 2 +300 20 emp 3 +400 30 emp 4 +500 NULL emp 5 +600 100 emp 6 + + +-- !query +SELECT id, + dept_id, + emp_name +FROM emp +WHERE EXISTS (SELECT emp_name + FROM bonus + WHERE emp.emp_name = bonus.emp_name) + OR (dept_id IS NOT NULL + AND dept_id NOT IN (SELECT dept_id + FROM dept)) +-- !query schema +struct +-- !query output +100 10 emp 1 +200 NULL emp 2 +300 20 emp 3 +400 30 emp 4 +500 NULL emp 5 +600 100 emp 6 + + +-- !query +SELECT id, + dept_id, + emp_name +FROM emp +WHERE dept_id = 10 +OR (id, emp_name) NOT IN (SELECT id, emp_name FROM address) +-- !query schema +struct +-- !query output +100 10 emp 1 + + +-- !query +SELECT id, + dept_id, + emp_name +FROM emp +WHERE dept_id = 10 + OR (( id, emp_name ) NOT IN (SELECT id, + emp_name + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL) + AND id > 400 ) +-- !query schema +struct +-- !query output +100 10 emp 1 +500 NULL emp 5 + + +-- !query +SELECT id, + dept_id, + emp_name +FROM emp +WHERE dept_id = 10 + OR emp_name NOT IN (SELECT emp_name + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL + AND emp.id = address.id) +-- !query schema +struct +-- !query output +100 10 emp 1 +200 NULL emp 2 +300 20 emp 3 +400 30 emp 4 +500 NULL emp 5 + + +-- !query +SELECT id, + dept_id, + emp_name +FROM emp +WHERE id NOT IN (SELECT id + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL + AND id >= 400) + OR emp_name NOT IN (SELECT emp_name + FROM address + WHERE id IS NOT NULL + AND emp_name IS NOT NULL + AND emp.id = address.id + AND id < 400) +-- !query schema +struct +-- !query output +100 10 emp 1 +200 NULL emp 2 +300 20 emp 3 +400 30 emp 4 +500 NULL emp 5 +600 100 emp 6 +800 70 emp 8 + + +-- !query +SELECT * +FROM s1 +WHERE NOT (a NOT IN (SELECT c + FROM s2)) +-- !query schema +struct +-- !query output +11 11 +8 8 + + +-- !query +SELECT * +FROM s1 +WHERE NOT (a > 5 + OR a IN (SELECT c + FROM s2)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM s1 +WHERE NOT (a > 5 + OR a NOT IN (SELECT c + FROM s2)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM s1 +WHERE NOT (a > 5 + AND a IN (SELECT c + FROM s2)) +-- !query schema +struct +-- !query output +5 5 + + +-- !query +SELECT * +FROM s1 +WHERE NOT (a > 5 + AND a NOT IN (SELECT c + FROM s2)) +-- !query schema +struct +-- !query output +11 11 +5 5 +8 8 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/not-in-group-by.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/not-in-group-by.sql.out new file mode 100644 index 000000000000..1b51206950c2 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/not-in-group-by.sql.out @@ -0,0 +1,147 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1a, + Avg(t1b) +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2) +GROUP BY t1a +-- !query schema +struct +-- !query output +val1a 11.0 +val1d 10.0 + + +-- !query +SELECT t1a, + Sum(DISTINCT( t1b )) +FROM t1 +WHERE t1d NOT IN (SELECT t2d + FROM t2 + WHERE t1h < t2h) +GROUP BY t1a +-- !query schema +struct +-- !query output +val1a 22 +val1d 10 +val1e 10 + + +-- !query +SELECT Count(*) +FROM (SELECT * + FROM t2 + WHERE t2a NOT IN (SELECT t3a + FROM t3 + WHERE t3h != t2h)) t2 +WHERE t2b NOT IN (SELECT Min(t2b) + FROM t2 + WHERE t2b = t2b + GROUP BY t2c) +-- !query schema +struct +-- !query output +4 + + +-- !query +SELECT t1a, + max(t1b) +FROM t1 +WHERE t1c NOT IN (SELECT Max(t2b) + FROM t2 + WHERE t1a = t2a + GROUP BY t2a) +GROUP BY t1a +-- !query schema +struct +-- !query output +val1a 16 +val1b 8 +val1c 8 +val1d 10 + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t2a NOT IN (SELECT Min(t3a) + FROM t3 + WHERE t3a = t2a + GROUP BY t3b) order by t2a) +-- !query schema +struct +-- !query output +val1a 16 +val1a 16 +val1a 6 +val1a 6 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/not-in-joins.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/not-in-joins.sql.out new file mode 100644 index 000000000000..9aeff336c6d9 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/not-in-joins.sql.out @@ -0,0 +1,226 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1a", 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("val1a", 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("val1d", null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ("val1d", null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ("val1e", 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("val1d", 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2 as select * from values + ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("val1c", 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("val1b", null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ("val2e", 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1f", 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("val1c", 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("val1e", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t3 as select * from values + ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val1b", 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("val1b", 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("val3c", 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("val3c", 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("val1b", null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1a, + t1b, + t1c, + t3a, + t3b, + t3c +FROM t1 + JOIN t3 +WHERE t1a NOT IN (SELECT t2a + FROM t2) + AND t1b = t3b +-- !query schema +struct +-- !query output +val1a 6 8 val3a 6 12 +val1a 6 8 val3a 6 12 +val1a 6 8 val3a 6 12 +val1a 6 8 val3a 6 12 +val1d 10 NULL val1b 10 12 +val1d 10 NULL val1b 10 12 + + +-- !query +SELECT t1a, + t1b, + t1c, + count(distinct(t3a)), + t3b, + t3c +FROM t1 +FULL OUTER JOIN t3 on t1b != t3b +RIGHT JOIN t2 on t1c = t2c +where t1a NOT IN + ( + SELECT t2a + FROM t2 + WHERE t2c NOT IN + ( + SELECT t1c + FROM t1 + WHERE t1a = t2a)) +AND t1b != t3b +AND t1d = t2d +GROUP BY t1a, t1b, t1c, t3a, t3b, t3c +HAVING count(distinct(t3a)) >= 1 +ORDER BY t1a, t3b +-- !query schema +struct +-- !query output +val1c 8 16 1 6 12 +val1c 8 16 1 10 12 +val1c 8 16 1 17 16 + + +-- !query +SELECT t1a, + t1b, + t1c, + t1d, + t1h +FROM t1 +WHERE t1a NOT IN + ( + SELECT t2a + FROM t2 + LEFT JOIN t3 on t2b = t3b + WHERE t1d = t2d + ) +AND t1d NOT IN + ( + SELECT t2d + FROM t2 + RIGHT JOIN t1 on t2e = t1e + WHERE t1a = t2a) +-- !query schema +struct +-- !query output +val1a 16 12 10 2014-07-04 01:01:00 +val1a 16 12 21 2014-06-04 01:02:00.001 +val1a 6 8 10 2014-04-04 01:00:00 +val1a 6 8 10 2014-04-04 01:02:00.001 +val1d 10 NULL 12 2015-05-04 01:01:00 +val1d NULL 16 22 2014-06-04 01:01:00 +val1e 10 NULL 25 2014-08-04 01:01:00 + + +-- !query +SELECT Count(DISTINCT( t1a )), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2 + JOIN t1 + WHERE t2b <> t1b) +GROUP BY t1b, + t1c, + t1d +HAVING t1d NOT IN (SELECT t2d + FROM t2 + WHERE t1d = t2d) +ORDER BY t1b DESC, t1d ASC +-- !query schema +struct +-- !query output +1 16 12 10 +1 16 12 21 +1 10 NULL 12 +1 6 8 10 +1 NULL 16 22 + + +-- !query +SELECT COUNT(DISTINCT(t1a)), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a NOT IN + ( + SELECT t2a + FROM t2 INNER + JOIN t1 ON t1a = t2a) +GROUP BY t1b, + t1c, + t1d +HAVING t1b < sum(t1c) +-- !query schema +struct +-- !query output +1 6 8 10 + + +-- !query +SELECT COUNT(DISTINCT(t1a)), + t1b, + t1c, + t1d +FROM t1 +WHERE t1a NOT IN + ( + SELECT t2a + FROM t2 INNER + JOIN t1 + ON t1a = t2a) +AND t1d NOT IN + ( + SELECT t2d + FROM t2 + INNER JOIN t3 + ON t2b = t3b ) +GROUP BY t1b, + t1c, + t1d +HAVING t1b < sum(t1c) +-- !query schema +struct +-- !query output +1 6 8 10 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out new file mode 100644 index 000000000000..2dc9d63c42b5 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql.out @@ -0,0 +1,45 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES + (null, null), + (null, 1.0), + (2, 3.0), + (4, 5.0) + AS m(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * +FROM m +WHERE b = 1.0 -- Matches (null, 1.0) + AND (a, b) NOT IN ((2, 3.0)) +-- !query schema +struct +-- !query output +NULL 1.0 + + +-- !query +SELECT * +FROM m +WHERE b = 3.0 -- Matches (2, 3.0) + AND (a, b) NOT IN ((2, 3.0)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM m +WHERE b = 5.0 -- Matches (4, 5.0) + AND (a, b) NOT IN ((2, 3.0)) +-- !query schema +struct +-- !query output +4 5.0 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out new file mode 100644 index 000000000000..525179f7ffcb --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/not-in-unit-tests-multi-column.sql.out @@ -0,0 +1,117 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES + (null, null), + (null, 1.0), + (2, 3.0), + (4, 5.0) + AS m(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW s AS SELECT * FROM VALUES + (null, null), + (0, 1.0), + (2, 3.0), + (4, null) + AS s(c, d) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * +FROM m +WHERE (a, b) NOT IN (SELECT * + FROM s + WHERE d > 5.0) -- Matches no rows +-- !query schema +struct +-- !query output +2 3.0 +4 5.0 +NULL 1.0 +NULL NULL + + +-- !query +SELECT * +FROM m +WHERE (a, b) NOT IN (SELECT * + FROM s + WHERE c IS NULL AND d IS NULL) -- Matches only (null, null) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM m +WHERE a IS NULL AND b IS NULL -- Matches only (null, null) + AND (a, b) NOT IN (SELECT * + FROM s + WHERE c IS NOT NULL) -- Matches (0, 1.0), (2, 3.0), (4, null) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM m +WHERE b = 1.0 -- Matches (null, 1.0) + AND (a, b) NOT IN (SELECT * + FROM s + WHERE c IS NOT NULL) -- Matches (0, 1.0), (2, 3.0), (4, null) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM m +WHERE b = 1.0 -- Matches (null, 1.0) + AND (a, b) NOT IN (SELECT * + FROM s + WHERE c = 2) -- Matches (2, 3.0) +-- !query schema +struct +-- !query output +NULL 1.0 + + +-- !query +SELECT * +FROM m +WHERE b = 3.0 -- Matches (2, 3.0) + AND (a, b) NOT IN (SELECT * + FROM s + WHERE c = 2) -- Matches (2, 3.0) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM m +WHERE b = 5.0 -- Matches (4, 5.0) + AND (a, b) NOT IN (SELECT * + FROM s + WHERE c = 2) -- Matches (2, 3.0) +-- !query schema +struct +-- !query output +4 5.0 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out new file mode 100644 index 000000000000..084a4fdbee28 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column-literal.sql.out @@ -0,0 +1,54 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES + (null, 1.0), + (2, 3.0), + (4, 5.0) + AS m(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * +FROM m +WHERE a NOT IN (null) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM m +WHERE b = 1.0 -- Only matches (null, 1.0) + AND a NOT IN (2) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM m +WHERE b = 3.0 -- Only matches (2, 3.0) + AND a NOT IN (2) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM m +WHERE b = 3.0 -- Only matches (2, 3.0) + AND a NOT IN (6) +-- !query schema +struct +-- !query output +2 3.0 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out new file mode 100644 index 000000000000..fe115fb9753e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/not-in-unit-tests-single-column.sql.out @@ -0,0 +1,128 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW m AS SELECT * FROM VALUES + (null, 1.0), + (2, 3.0), + (4, 5.0) + AS m(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW s AS SELECT * FROM VALUES + (null, 1.0), + (2, 3.0), + (6, 7.0) + AS s(c, d) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * +FROM m +WHERE a NOT IN (SELECT c + FROM s + WHERE d > 10.0) -- (empty subquery) +-- !query schema +struct +-- !query output +2 3.0 +4 5.0 +NULL 1.0 + + +-- !query +SELECT * +FROM m +WHERE a NOT IN (SELECT c + FROM s + WHERE d = 1.0) -- Only matches (null, 1.0) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM m +WHERE b = 1.0 -- Only matches (null, 1.0) + AND a NOT IN (SELECT c + FROM s + WHERE d = 3.0) -- Matches (2, 3.0) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM m +WHERE b = 3.0 -- Only matches (2, 3.0) + AND a NOT IN (SELECT c + FROM s + WHERE d = 3.0) -- Matches (2, 3.0) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM m +WHERE b = 3.0 -- Only matches (2, 3.0) + AND a NOT IN (SELECT c + FROM s + WHERE d = 7.0) -- Matches (6, 7.0) +-- !query schema +struct +-- !query output +2 3.0 + + +-- !query +SELECT * +FROM m +WHERE a NOT IN (SELECT c + FROM s + WHERE d = b + 10) -- Matches no row +-- !query schema +struct +-- !query output +2 3.0 +4 5.0 +NULL 1.0 + + +-- !query +SELECT * +FROM m +WHERE b = 1.0 -- Only matches (null, 1.0) + AND a NOT IN (SELECT c + FROM s + WHERE d = b + 10) -- Matches no row +-- !query schema +struct +-- !query output +NULL 1.0 + + +-- !query +SELECT * +FROM m +WHERE b = 3.0 -- Only matches (2, 3.0) + AND a NOT IN (SELECT c + FROM s + WHERE d = b + 10) -- Matches no row +-- !query schema +struct +-- !query output +2 3.0 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/simple-in.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/simple-in.sql.out new file mode 100644 index 000000000000..c917bba4dbf4 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/in-subquery/simple-in.sql.out @@ -0,0 +1,221 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), + ("t1b", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1a", 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ("t1a", 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ("t1d", null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ("t1d", null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ("t1e", 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ("t1d", 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1a", 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ("t1e", 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2 as select * from values + ("t2a", 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ("t1c", 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ("t1b", null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ("t2e", 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1f", 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ("t1c", 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ("t1e", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ("t1f", 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t3 as select * from values + ("t3a", 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ("t3a", 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t1b", 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ("t1b", 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ("t3c", 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ("t3c", 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ("t1b", null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ("t1b", null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ("t3b", 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ("t3b", 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2) +-- !query schema +struct +-- !query output +t1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +t1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 +t1e 10 NULL 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 +t1e 10 NULL 19 17.0 25.0 2600 2014-09-04 01:02:00.001 2014-09-04 +t1e 10 NULL 25 17.0 25.0 2600 2014-08-04 01:01:00 2014-08-04 + + +-- !query +SELECT * +FROM t1 +WHERE t1b IN (SELECT t2b + FROM t2 + WHERE t1a = t2a) +-- !query schema +struct +-- !query output +t1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t1a != t2a) +-- !query schema +struct +-- !query output +t1a 16 +t1a 16 +t1a 6 +t1a 6 + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t1a = t2a + OR t1b > t2b) +-- !query schema +struct +-- !query output +t1a 16 +t1a 16 + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t2i IN (SELECT t3i + FROM t3 + WHERE t2c = t3c)) +-- !query schema +struct +-- !query output +t1a 6 +t1a 6 + + +-- !query +SELECT t1a, + t1b +FROM t1 +WHERE t1c IN (SELECT t2b + FROM t2 + WHERE t2a IN (SELECT t3a + FROM t3 + WHERE t2c = t3c + AND t2b IS NOT NULL)) +-- !query schema +struct +-- !query output +t1a 6 +t1a 6 + + +-- !query +SELECT DISTINCT( t1a ), + t1b, + t1h +FROM t1 +WHERE t1a NOT IN (SELECT t2a + FROM t2) +-- !query schema +struct +-- !query output +t1a 16 2014-06-04 01:02:00.001 +t1a 16 2014-07-04 01:01:00 +t1a 6 2014-04-04 01:00:00 +t1a 6 2014-04-04 01:02:00.001 +t1d 10 2015-05-04 01:01:00 +t1d NULL 2014-06-04 01:01:00 +t1d NULL 2014-07-04 01:02:00.001 + + +-- !query +create temporary view a as select * from values + (1, 1), (2, 1), (null, 1), (1, 3), (null, 3), (1, null), (null, 2) + as a(a1, a2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view b as select * from values + (1, 1, 2), (null, 3, 2), (1, null, 2), (1, 2, null) + as b(b1, b2, b3) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT a1, a2 +FROM a +WHERE a1 NOT IN (SELECT b.b1 + FROM b + WHERE a.a2 = b.b2) +-- !query schema +struct +-- !query output +1 NULL +2 1 + + +-- !query +SELECT a1, a2 +FROM a +WHERE a1 NOT IN (SELECT b.b1 + FROM b + WHERE a.a2 = b.b2 + AND b.b3 > 1) +-- !query schema +struct +-- !query output +1 NULL +2 1 +NULL 2 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out new file mode 100644 index 000000000000..93c60fd49c58 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out @@ -0,0 +1,208 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES + (1, 2, 3) +AS t1(t1a, t1b, t1c) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES + (1, 0, 1) +AS t2(t2a, t2b, t2c) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES + (3, 1, 2) +AS t3(t3a, t3b, t3c) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1a, t2b +FROM t1, t2 +WHERE t1b = t2c +AND t2b = (SELECT max(avg) + FROM (SELECT t2b, avg(t2b) avg + FROM t2 + WHERE t2a = t1.t1b + ) + ) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 100, + "stopIndex" : 203, + "fragment" : "SELECT t2b, avg(t2b) avg\n FROM t2\n WHERE t2a = t1.t1b" + } ] +} + + +-- !query +SELECT * +FROM t1 +WHERE t1a IN (SELECT min(t2a) + FROM t2 + GROUP BY t2c + HAVING t2c IN (SELECT max(t3c) + FROM t3 + GROUP BY t3b + HAVING t3b > t2b )) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_ATTRIBUTES.RESOLVED_ATTRIBUTE_MISSING_FROM_INPUT", + "sqlState" : "XX000", + "messageParameters" : { + "input" : "\"min(t2a)\", \"t2c\"", + "missingAttributes" : "\"t2b\"", + "operator" : "!Filter t2c#x IN (list#x [t2b#x])" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 35, + "stopIndex" : 298, + "fragment" : "SELECT min(t2a)\n FROM t2\n GROUP BY t2c\n HAVING t2c IN (SELECT max(t3c)\n FROM t3\n GROUP BY t3b\n HAVING t3b > t2b )" + } ] +} + + +-- !query +SELECT t1a +FROM t1 +GROUP BY 1 +HAVING EXISTS (SELECT t2a + FROM t2 + GROUP BY 1 + HAVING t2a < min(t1a + t2a)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "sqlState" : "0A000", + "messageParameters" : { + "function" : "min((outer(t1.t1a) + t2.t2a))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 139, + "stopIndex" : 152, + "fragment" : "min(t1a + t2a)" + } ] +} + + +-- !query +SELECT t1a +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE EXISTS (SELECT 1 + FROM t3 + GROUP BY 1 + HAVING min(t2a + t3a) > 1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "sqlState" : "0A000", + "messageParameters" : { + "function" : "min((outer(t2.t2a) + t3.t3a))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 233, + "stopIndex" : 246, + "fragment" : "min(t2a + t3a)" + } ] +} + + +-- !query +SELECT t1a +FROM t1 +WHERE t1a IN (SELECT t2a + FROM t2 + WHERE EXISTS (SELECT min(t2a) + FROM t3)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"min(t2a) AS `min(outer(t2.t2a))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 105, + "stopIndex" : 160, + "fragment" : "SELECT min(t2a) \n FROM t3" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW t1_copy AS SELECT * FROM VALUES + (1, 2, 3) +AS t1(t1a, t1b, t1c) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1.t1a +FROM t1 +JOIN t1_copy +ON EXISTS (SELECT 1 FROM t2 WHERE t2a > t1a) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t1a`", + "proposal" : "`t2a`, `t2b`, `t2c`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 84, + "stopIndex" : 86, + "fragment" : "t1a" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out new file mode 100644 index 000000000000..52338368f25d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/negative-cases/subq-input-typecheck.sql.out @@ -0,0 +1,195 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES + (1, 2, 3) +AS t1(t1a, t1b, t1c) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES + (1, 0, 1) +AS t2(t2a, t2b, t2c) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES + (3, 1, 2) +AS t3(t3a, t3b, t3c) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES + (CAST(1 AS DOUBLE), CAST(2 AS STRING), CAST(3 AS STRING)) +AS t1(t4a, t4b, t4c) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW t5 AS SELECT * FROM VALUES + (CAST('2011-01-01 01:01:01' AS TIMESTAMP), CAST(2 AS STRING), CAST(3 AS BIGINT)) +AS t1(t5a, t5b, t5c) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + ( SELECT max(t2b), min(t2b) + FROM t2 + WHERE t2.t2b = t1.t1b + GROUP BY t2.t2b + ) +FROM t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_SUBQUERY_EXPRESSION.SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", + "sqlState" : "42823", + "messageParameters" : { + "number" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 11, + "stopIndex" : 101, + "fragment" : "( SELECT max(t2b), min(t2b) \n FROM t2 \n WHERE t2.t2b = t1.t1b\n GROUP BY t2.t2b\n )" + } ] +} + + +-- !query +SELECT + ( SELECT max(t2b), min(t2b) + FROM t2 + WHERE t2.t2b > 0 + GROUP BY t2.t2b + ) +FROM t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_SUBQUERY_EXPRESSION.SCALAR_SUBQUERY_RETURN_MORE_THAN_ONE_OUTPUT_COLUMN", + "sqlState" : "42823", + "messageParameters" : { + "number" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 11, + "stopIndex" : 96, + "fragment" : "( SELECT max(t2b), min(t2b) \n FROM t2 \n WHERE t2.t2b > 0\n GROUP BY t2.t2b\n )" + } ] +} + + +-- !query +SELECT * FROM t1 +WHERE +t1a IN (SELECT t2a, t2b + FROM t2 + WHERE t1a = t2a) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_LENGTH_MISMATCH", + "sqlState" : "42K09", + "messageParameters" : { + "leftColumns" : "\"t1a\"", + "leftLength" : "1", + "rightColumns" : "\"t2a\", \"t2b\"", + "rightLength" : "2", + "sqlExpr" : "\"(t1a IN (listquery(t1a)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 88, + "fragment" : "IN (SELECT t2a, t2b \n FROM t2\n WHERE t1a = t2a)" + } ] +} + + +-- !query +SELECT * FROM T1 +WHERE +(t1a, t1b) IN (SELECT t2a + FROM t2 + WHERE t1a = t2a) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_LENGTH_MISMATCH", + "sqlState" : "42K09", + "messageParameters" : { + "leftColumns" : "\"t1a\", \"t1b\"", + "leftLength" : "2", + "rightColumns" : "\"t2a\"", + "rightLength" : "1", + "sqlExpr" : "\"(named_struct('t1a', t1a, 't1b', t1b) IN (listquery(t1a)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 104, + "fragment" : "IN (SELECT t2a\n FROM t2\n WHERE t1a = t2a)" + } ] +} + + +-- !query +SELECT * FROM t4 +WHERE +(t4a, t4b, t4c) IN (SELECT t5a, + t5b, + t5c + FROM t5) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.IN_SUBQUERY_DATA_TYPE_MISMATCH", + "sqlState" : "42K09", + "messageParameters" : { + "leftType" : "\"DOUBLE\", \"STRING\", \"STRING\"", + "mismatchedColumns" : "(t4.t4a:double, t5.t5a:timestamp), (t4.t4c:string, t5.t5c:bigint)", + "rightType" : "\"TIMESTAMP\", \"STRING\", \"BIGINT\"", + "sqlExpr" : "\"(named_struct('t4a', t4a, 't4b', t4b, 't4c', t4c) IN (listquery()))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 40, + "stopIndex" : 146, + "fragment" : "IN (SELECT t5a,\n t5b,\n t5c\n FROM t5)" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/scalar-subquery/nested-scalar-subquery-count-bug.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/scalar-subquery/nested-scalar-subquery-count-bug.sql.out new file mode 100644 index 000000000000..c524d315bafc --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/scalar-subquery/nested-scalar-subquery-count-bug.sql.out @@ -0,0 +1,125 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE VIEW t1(a1, a2) as values (0, 1), (1, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW t2(b1, b2) as values (0, 2), (0, 3) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW t3(c1, c2) as values (0, 2), (0, 3) +-- !query schema +struct<> +-- !query output + + + +-- !query +set spark.sql.optimizer.decorrelateInnerQuery.enabled=true +-- !query schema +struct +-- !query output +spark.sql.optimizer.decorrelateInnerQuery.enabled true + + +-- !query +set spark.sql.legacy.scalarSubqueryCountBugBehavior=false +-- !query schema +struct +-- !query output +spark.sql.legacy.scalarSubqueryCountBugBehavior false + + +-- !query +select ( select sum(cnt) from (select count(*) cnt from t2 where t1.a1 = t2.b1) ) a from t1 order by a desc +-- !query schema +struct +-- !query output +2 +0 + + +-- !query +select ( select count(*) from (select count(*) cnt from t2 where t1.a1 = t2.b1) ) a from t1 order by a desc +-- !query schema +struct +-- !query output +1 +1 + + +-- !query +select ( + select SUM(l.cnt + r.cnt) + from (select count(*) cnt from t2 where t1.a1 = t2.b1 having cnt = 0) l + join (select count(*) cnt from t3 where t1.a1 = t3.c1 having cnt = 0) r + on l.cnt = r.cnt +) a from t1 order by a desc +-- !query schema +struct +-- !query output +0 +NULL + + +-- !query +select ( + select sum(l.cnt + r.cnt) + from (select count(*) cnt from t2 where t1.a1 = t2.b1) l + join (select count(*) cnt from t3 where t1.a1 = t3.c1) r + on l.cnt = r.cnt +) a from t1 order by a desc +-- !query schema +struct +-- !query output +4 +0 + + +-- !query +reset spark.sql.optimizer.decorrelateInnerQuery.enabled +-- !query schema +struct<> +-- !query output + + + +-- !query +reset spark.sql.legacy.scalarSubqueryCountBugBehavior +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW t3 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/scalar-subquery/scalar-subquery-count-bug.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/scalar-subquery/scalar-subquery-count-bug.sql.out new file mode 100644 index 000000000000..aa295dfaca24 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/scalar-subquery/scalar-subquery-count-bug.sql.out @@ -0,0 +1,382 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temp view l (a, b) +as values + (1, 2.0), + (1, 2.0), + (2, 1.0), + (2, 1.0), + (3, 3.0), + (null, null), + (null, 5.0), + (6, null) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temp view r (c, d) +as values + (2, 3.0), + (2, 3.0), + (3, 2.0), + (4, 1.0), + (null, null), + (null, 5.0), + (6, null) +-- !query schema +struct<> +-- !query output + + + +-- !query +select *, (select count(*) from r where l.a = r.c) from l +-- !query schema +struct +-- !query output +1 2.0 0 +1 2.0 0 +2 1.0 2 +2 1.0 2 +3 3.0 1 +6 NULL 1 +NULL 5.0 0 +NULL NULL 0 + + +-- !query +select *, (select count(*) from r where l.a = r.c group by c) from l +-- !query schema +struct +-- !query output +1 2.0 NULL +1 2.0 NULL +2 1.0 2 +2 1.0 2 +3 3.0 1 +6 NULL 1 +NULL 5.0 NULL +NULL NULL NULL + + +-- !query +select *, (select count(*) from r where l.a = r.c group by 'constant') from l +-- !query schema +struct +-- !query output +1 2.0 NULL +1 2.0 NULL +2 1.0 2 +2 1.0 2 +3 3.0 1 +6 NULL 1 +NULL 5.0 NULL +NULL NULL NULL + + +-- !query +select *, ( + select (count(*)) is null + from r + where l.a = r.c) +from l +-- !query schema +struct +-- !query output +1 2.0 false +1 2.0 false +2 1.0 false +2 1.0 false +3 3.0 false +6 NULL false +NULL 5.0 false +NULL NULL false + + +-- !query +select *, ( + select (count(*)) is null + from r + where l.a = r.c + group by r.c) +from l +-- !query schema +struct +-- !query output +1 2.0 NULL +1 2.0 NULL +2 1.0 false +2 1.0 false +3 3.0 false +6 NULL false +NULL 5.0 NULL +NULL NULL NULL + + +-- !query +select *, (select count(*) from r where l.a = r.c having count(*) <= 1) from l +-- !query schema +struct +-- !query output +1 2.0 0 +1 2.0 0 +2 1.0 NULL +2 1.0 NULL +3 3.0 1 +6 NULL 1 +NULL 5.0 0 +NULL NULL 0 + + +-- !query +select *, (select count(*) from r where l.a = r.c having count(*) >= 2) from l +-- !query schema +struct +-- !query output +1 2.0 NULL +1 2.0 NULL +2 1.0 2 +2 1.0 2 +3 3.0 NULL +6 NULL NULL +NULL 5.0 NULL +NULL NULL NULL + + +-- !query +CREATE TEMPORARY VIEW null_view(a, b) AS SELECT CAST(null AS int), CAST(null as int) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + ( + SELECT + COUNT(null_view.a) AS result + FROM + null_view + WHERE + null_view.a = l.a + ) +FROM + l +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +0 + + +-- !query +SELECT + ( + SELECT + COUNT(null_view.a) AS result + FROM + null_view + WHERE + null_view.a = l.a + having count(*) > -1 + ) +FROM + l +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +0 + + +-- !query +SELECT + ( + SELECT + COUNT(null_view.a) AS result + FROM + null_view + INNER JOIN r + ON r.c = null_view.a + AND r.c IS NOT NULL + WHERE + null_view.a = l.a + ) +FROM + l +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +0 + + +-- !query +SELECT +( + SELECT + COUNT(null_view.a) AS result + FROM + null_view + INNER JOIN r + ON r.c = null_view.a + AND r.c IS NOT NULL + WHERE + null_view.a = l.a + HAVING COUNT(*) > -1 +) +FROM + l +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +0 + + +-- !query +SELECT + ( + SELECT + COUNT(f.a) AS result + FROM + ( + SELECT a, b FROM null_view + INTERSECT + SELECT c, d FROM r WHERE c IS NOT NULL + ) AS f + WHERE + f.a = l.a + ) +FROM + l +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +0 + + +-- !query +SELECT +( + SELECT + COUNT(f.a) AS result + FROM + ( + SELECT a, b FROM null_view + INTERSECT + SELECT c, d FROM r WHERE c IS NOT NULL + ) AS f + WHERE + f.a = l.a + HAVING COUNT(*) > -1 +) +FROM + l +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +0 + + +-- !query +set spark.sql.optimizer.decorrelateSubqueryLegacyIncorrectCountHandling.enabled = true +-- !query schema +struct +-- !query output +spark.sql.optimizer.decorrelateSubqueryLegacyIncorrectCountHandling.enabled true + + +-- !query +select *, (select count(*) from r where l.a = r.c) from l +-- !query schema +struct +-- !query output +1 2.0 0 +1 2.0 0 +2 1.0 2 +2 1.0 2 +3 3.0 1 +6 NULL 1 +NULL 5.0 0 +NULL NULL 0 + + +-- !query +select *, (select count(*) from r where l.a = r.c group by c) from l +-- !query schema +struct +-- !query output +1 2.0 0 +1 2.0 0 +2 1.0 2 +2 1.0 2 +3 3.0 1 +6 NULL 1 +NULL 5.0 0 +NULL NULL 0 + + +-- !query +select *, (select count(*) from r where l.a = r.c group by 'constant') from l +-- !query schema +struct +-- !query output +1 2.0 0 +1 2.0 0 +2 1.0 2 +2 1.0 2 +3 3.0 1 +6 NULL 1 +NULL 5.0 0 +NULL NULL 0 + + +-- !query +reset spark.sql.optimizer.decorrelateSubqueryLegacyIncorrectCountHandling.enabled +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out new file mode 100644 index 000000000000..56932edd4e54 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/scalar-subquery/scalar-subquery-group-by.sql.out @@ -0,0 +1,295 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temp view x (x1, x2) as values (1, 1), (2, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temp view y (y1, y2) as values (2, 0), (3, -1) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temp view z (z1, z2) as values (1, 0), (1, 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from x where (select count(*) from y where y1 = x1 group by y1) = 1 +-- !query schema +struct +-- !query output +2 2 + + +-- !query +select * from x where (select count(*) from y where y1 = x1 group by x1) = 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"x1\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 61, + "stopIndex" : 71, + "fragment" : "group by x1" + } ] +} + + +-- !query +select * from x where (select count(*) from y where y1 > x1 group by x1) = 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"x1\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 61, + "stopIndex" : 71, + "fragment" : "group by x1" + } ] +} + + +-- !query +select *, (select count(*) from y where x1 = y1 and y2 = 1 group by y2) from x +-- !query schema +struct +-- !query output +1 1 NULL +2 2 NULL + + +-- !query +select *, (select count(*) from y where x1 = y1 and y2 = x1 + 1 group by y2) from x +-- !query schema +struct +-- !query output +1 1 NULL +2 2 NULL + + +-- !query +select *, (select count(*) from y where x1 = y1 and cast(y2 as double) = x1 + 1 + group by cast(y2 as double)) from x +-- !query schema +struct +-- !query output +1 1 NULL +2 2 NULL + + +-- !query +select *, (select count(*) from y where y2 + 1 = x1 + x2 group by y2 + 1) from x +-- !query schema +struct +-- !query output +1 1 NULL +2 2 NULL + + +-- !query +set spark.sql.optimizer.scalarSubqueryUseSingleJoin = false +-- !query schema +struct +-- !query output +spark.sql.optimizer.scalarSubqueryUseSingleJoin false + + +-- !query +select * from x where (select count(*) from y where y1 > x1 group by y1) = 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.NON_CORRELATED_COLUMNS_IN_GROUP_BY", + "sqlState" : "0A000", + "messageParameters" : { + "value" : "y1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 72, + "fragment" : "(select count(*) from y where y1 > x1 group by y1)" + } ] +} + + +-- !query +select *, (select count(*) from y where y1 + y2 = x1 group by y1) from x +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.NON_CORRELATED_COLUMNS_IN_GROUP_BY", + "sqlState" : "0A000", + "messageParameters" : { + "value" : "y1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 11, + "stopIndex" : 65, + "fragment" : "(select count(*) from y where y1 + y2 = x1 group by y1)" + } ] +} + + +-- !query +select *, (select count(*) from y where x1 = y1 and y2 + 10 = x1 + 1 group by y2) from x +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.NON_CORRELATED_COLUMNS_IN_GROUP_BY", + "sqlState" : "0A000", + "messageParameters" : { + "value" : "y2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 11, + "stopIndex" : 81, + "fragment" : "(select count(*) from y where x1 = y1 and y2 + 10 = x1 + 1 group by y2)" + } ] +} + + +-- !query +set spark.sql.optimizer.scalarSubqueryUseSingleJoin = true +-- !query schema +struct +-- !query output +spark.sql.optimizer.scalarSubqueryUseSingleJoin true + + +-- !query +select * from x where (select count(*) from y where y1 > x1 group by y1) = 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "SCALAR_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000" +} + + +-- !query +select *, (select count(*) from y where y1 + y2 = x1 group by y1) from x +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "SCALAR_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000" +} + + +-- !query +select *, (select count(*) from y where x1 = y1 and y2 + 10 = x1 + 1 group by y2) from x +-- !query schema +struct +-- !query output +1 1 NULL +2 2 NULL + + +-- !query +select *, (select count(*) from (select * from y where y1 = x1 union all select * from y) sub group by y1) from x +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "SCALAR_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000" +} + + +-- !query +select *, (select count(*) from y left join (select * from z where z1 = x1) sub on y2 = z2 group by z1) from x +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "Filter (z1#x = outer(x1#x))\n+- SubqueryAlias z\n +- View (`z`, [z1#x, z2#x])\n +- Project [cast(col1#x as int) AS z1#x, cast(col2#x as int) AS z2#x]\n +- LocalRelation [col1#x, col2#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 46, + "stopIndex" : 74, + "fragment" : "select * from z where z1 = x1" + } ] +} + + +-- !query +set spark.sql.legacy.scalarSubqueryAllowGroupByNonEqualityCorrelatedPredicate = true +-- !query schema +struct +-- !query output +spark.sql.legacy.scalarSubqueryAllowGroupByNonEqualityCorrelatedPredicate true + + +-- !query +set spark.sql.optimizer.scalarSubqueryUseSingleJoin = false +-- !query schema +struct +-- !query output +spark.sql.optimizer.scalarSubqueryUseSingleJoin false + + +-- !query +select * from x where (select count(*) from y where y1 > x1 group by y1) = 1 +-- !query schema +struct +-- !query output +1 1 +1 1 +2 2 + + +-- !query +reset spark.sql.legacy.scalarSubqueryAllowGroupByNonEqualityCorrelatedPredicate +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out new file mode 100644 index 000000000000..b37fe614e237 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/scalar-subquery/scalar-subquery-predicate.sql.out @@ -0,0 +1,932 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW p AS VALUES (1, 1) AS T(pk, pv) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW c AS VALUES (1, 1) AS T(ck, cv) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT pk, cv +FROM p, c +WHERE p.pk = c.ck +AND c.cv = (SELECT avg(c1.cv) + FROM c c1 + WHERE c1.ck = p.pk) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT pk, cv +FROM p, c +WHERE p.pk = c.ck +AND c.cv = (SELECT max(avg) + FROM (SELECT c1.cv, avg(c1.cv) avg + FROM c c1 + WHERE c1.ck = p.pk + GROUP BY c1.cv)) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +create temporary view t1 as select * from values + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), + ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1a', 16S, 12, 21L, float(15.0), 20D, 20E2BD, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ('val1a', 16S, 12, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 8S, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ('val1d', null, 16, 22L, float(17.0), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', null), + ('val1d', null, 16, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.001', null), + ('val1e', 10S, null, 25L, float(17.0), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ('val1d', 10S, null, 12L, float(17.0), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2 as select * from values + ('val2a', 6S, 12, 14L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 119L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1c', 12S, 16, 219L, float(17), 25D, 26E2BD, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ('val1b', null, 16, 319L, float(17), 25D, 26E2BD, timestamp '2017-05-04 01:01:00.000', null), + ('val2e', 8S, null, 419L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1f', 19S, null, 519L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 12S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ('val1e', 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ('val1f', 19S, null, 19L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ('val1b', null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t3 as select * from values + ('val3a', 6S, 12, 110L, float(15), 20D, 20E2BD, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ('val3a', 6S, 12, 10L, float(15), 20D, 20E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 219L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 319L, float(17), 25D, 26E2BD, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ('val3c', 17S, 16, 519L, float(17), 25D, 26E2BD, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ('val3c', 17S, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ('val1b', null, 16, 419L, float(17), 25D, 26E2BD, timestamp '2014-10-04 01:02:00.000', null), + ('val1b', null, 16, 19L, float(17), 25D, 26E2BD, timestamp '2014-11-04 01:02:00.000', null), + ('val3b', 8S, null, 719L, float(17), 25D, 26E2BD, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val3b', 8S, null, 19L, float(17), 25D, 26E2BD, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t1a, t1b +FROM t1 +WHERE t1c = (SELECT max(t2c) + FROM t2) +-- !query schema +struct +-- !query output +val1b 8 +val1c 8 +val1d NULL +val1d NULL + + +-- !query +SELECT t1a, t1d, t1f +FROM t1 +WHERE t1c = (SELECT max(t2c) + FROM t2) +AND t1b > (SELECT min(t3b) + FROM t3) +-- !query schema +struct +-- !query output +val1b 19 25.0 +val1c 19 25.0 + + +-- !query +SELECT t1a, t1h +FROM t1 +WHERE t1c = (SELECT max(t2c) + FROM t2) +OR t1b = (SELECT min(t3b) + FROM t3 + WHERE t3b > 10) +-- !query schema +struct +-- !query output +val1b 2014-05-04 01:01:00 +val1c 2014-05-04 01:02:00.001 +val1d 2014-06-04 01:01:00 +val1d 2014-07-04 01:02:00.001 + + +-- !query +SELECT t1a, t1b, t2d +FROM t1 LEFT JOIN t2 + ON t1a = t2a +WHERE t1b = (SELECT min(t3b) + FROM t3) +-- !query schema +struct +-- !query output +val1a 6 NULL +val1a 6 NULL + + +-- !query +SELECT t1a, t1b, t1g +FROM t1 +WHERE t1c + 5 = (SELECT max(t2e) + FROM t2) +-- !query schema +struct +-- !query output +val1a 16 2000 +val1a 16 2000 + + +-- !query +SELECT t1a, t1h +FROM t1 +WHERE date(t1h) = (SELECT min(t2i) + FROM t2) +-- !query schema +struct +-- !query output +val1a 2014-04-04 00:00:00 +val1a 2014-04-04 01:02:00.001 + + +-- !query +SELECT t2d, t1a +FROM t1, t2 +WHERE t1b = t2b +AND t2c + 1 = (SELECT max(t2c) + 1 + FROM t2, t1 + WHERE t2b = t1b) +-- !query schema +struct +-- !query output +119 val1b +119 val1c +19 val1b +19 val1c + + +-- !query +SELECT DISTINCT t2a, max_t1g +FROM t2, (SELECT max(t1g) max_t1g, t1a + FROM t1 + GROUP BY t1a) t1 +WHERE t2a = t1a +AND max_t1g = (SELECT max(t1g) + FROM t1) +-- !query schema +struct +-- !query output +val1b 2600 +val1c 2600 +val1e 2600 + + +-- !query +SELECT t3b, t3c +FROM t3 +WHERE (SELECT max(t3c) + FROM t3 + WHERE t3b > 10) >= + (SELECT min(t3b) + FROM t3 + WHERE t3c > 0) +AND (t3b is null or t3c is null) +-- !query schema +struct +-- !query output +8 NULL +8 NULL +NULL 16 +NULL 16 + + +-- !query +SELECT t1a +FROM t1 +WHERE t1a < (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query schema +struct +-- !query output +val1a +val1a +val1b + + +-- !query +SELECT t1a, t1c +FROM t1 +WHERE (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) IS NULL +-- !query schema +struct +-- !query output +val1a 8 +val1a 8 +val1d NULL +val1e NULL +val1e NULL +val1e NULL + + +-- !query +SELECT t1a +FROM t1 +WHERE t1a = (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c + HAVING count(*) >= 0) +OR t1i > '2014-12-31' +-- !query schema +struct +-- !query output +val1c +val1d + + +-- !query +SELECT t1a +FROM t1 +WHERE t1a = (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c + HAVING count(*) >= 1) +OR t1i > '2014-12-31' +-- !query schema +struct +-- !query output +val1c +val1d + + +-- !query +SELECT count(t1a) +FROM t1 RIGHT JOIN t2 +ON t1d = t2d +WHERE t1a < (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query schema +struct +-- !query output +7 + + +-- !query +SELECT t1a +FROM t1 +WHERE t1b <= (SELECT max(t2b) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +AND t1b >= (SELECT min(t2b) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query schema +struct +-- !query output +val1b +val1c + + +-- !query +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +INTERSECT +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query schema +struct +-- !query output +val1b +val1c + + +-- !query +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +UNION ALL +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query schema +struct +-- !query output +val1a +val1a +val1b +val1b +val1c +val1c +val1d +val1d + + +-- !query +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +UNION DISTINCT +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query schema +struct +-- !query output +val1a +val1b +val1c +val1d + + +-- !query +SELECT t1a +FROM t1 +WHERE t1a <= (SELECT max(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +MINUS +SELECT t1a +FROM t1 +WHERE t1a >= (SELECT min(t2a) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query schema +struct +-- !query output +val1a + + +-- !query +SELECT t1a +FROM t1 +GROUP BY t1a, t1c +HAVING max(t1b) <= (SELECT max(t2b) + FROM t2 + WHERE t2c = t1c + GROUP BY t2c) +-- !query schema +struct +-- !query output +val1b +val1c + + +-- !query +SELECT 1 +FROM t1 +WHERE t1b < (SELECT MAX(tmp.s) FROM ( + SELECT SUM(t2b) OVER (partition by t2c order by t2d) as s + FROM t2 WHERE t2.t2d = t1.t1d) as tmp) +-- !query schema +struct<1:int> +-- !query output +1 +1 +1 +1 + + +-- !query +SELECT 1 +FROM t1 +WHERE t1b < (SELECT SUM(t2b) OVER (partition by t2c order by t2d) as s + FROM t2 WHERE t2.t2d = t1.t1d + ORDER BY s DESC + LIMIT 1) +-- !query schema +struct<1:int> +-- !query output +1 +1 +1 +1 + + +-- !query +SELECT 1 +FROM t1 +WHERE t1b < (SELECT MAX(tmp.s) FROM ( + SELECT SUM(t2b) OVER (partition by t2c order by t2d) as s + FROM t2 WHERE t2.t2d <= t1.t1d) as tmp) +-- !query schema +struct<1:int> +-- !query output +1 +1 +1 +1 +1 +1 + + +-- !query +SELECT 1 +FROM t1 +WHERE t1b < (SELECT SUM(t2b) OVER (partition by t2c order by t2d) as s + FROM t2 WHERE t2.t2d <= t1.t1d + ORDER BY s DESC + LIMIT 1) +-- !query schema +struct<1:int> +-- !query output +1 +1 +1 +1 +1 +1 + + +-- !query +SELECT t1b +FROM t1 +WHERE t1b > (SELECT MAX(tmp.s) FROM ( + SELECT RANK() OVER (partition by t3c, t2b order by t3c) as s + FROM t2, t3 where t2.t2c = t3.t3c AND t2.t2a = t1.t1a) as tmp) +-- !query schema +struct +-- !query output +8 +8 + + +-- !query +SELECT t1b +FROM t1 +WHERE t1b > (SELECT MAX(tmp.s) FROM ( + SELECT RANK() OVER (partition by t3c, t3d order by t3c) as s + FROM (SELECT t3b, t3c, max(t3d) as t3d FROM t3 GROUP BY t3b, t3c) as g) as tmp) +ORDER BY t1b +-- !query schema +struct +-- !query output +6 +6 +8 +8 +10 +10 +10 +10 +16 +16 + + +-- !query +SELECT 1 +FROM t1 +WHERE t1b = (SELECT MAX(tmp.s) FROM ( + SELECT SUM(t2c) OVER (partition by t2c order by t1.t1d + t2d) as s + FROM t2) as tmp) +-- !query schema +struct<1:int> +-- !query output + + + +-- !query +SELECT t1a, t1b +FROM t1 +WHERE t1c = (SELECT t2c + FROM t2 + WHERE t2b < t1b + ORDER BY t2d LIMIT 1) +-- !query schema +struct +-- !query output +val1a 16 +val1a 16 + + +-- !query +SELECT t1a, t1b +FROM t1 +WHERE t1c = (SELECT t2c + FROM t2 + WHERE t2c = t1c + ORDER BY t2c LIMIT 1) +-- !query schema +struct +-- !query output +val1a 16 +val1a 16 +val1b 8 +val1c 8 +val1d NULL +val1d NULL + + +-- !query +SELECT t1a, t1b +FROM t1 +WHERE t1c = (SELECT t2c + FROM t2 + WHERE t1b < t1d + ORDER BY t2c LIMIT 1) +-- !query schema +struct +-- !query output +val1a 16 + + +-- !query +SELECT t1a, t1b +FROM t1 +WHERE t1c = (SELECT MAX(t2c) + FROM t2 + WHERE t1b < t1d + ORDER BY min(t2c) LIMIT 1) +-- !query schema +struct +-- !query output +val1b 8 +val1c 8 + + +-- !query +SELECT t1a, t1b +FROM t1 +WHERE t1c = (SELECT DISTINCT t2c + FROM t2 + WHERE t1b < t1d + ORDER BY t2c LIMIT 1) +-- !query schema +struct +-- !query output +val1a 16 + + +-- !query +CREATE OR REPLACE TEMP VIEW t0(t0a, t0b) AS VALUES (1, 1), (2, 0) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMP VIEW t1(t1a, t1b, t1c) AS VALUES (1, 1, 3) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMP VIEW t2(t2a, t2b, t2c) AS VALUES (1, 1, 5), (2, 2, 7) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +-- !query schema +struct +-- !query output +1 1 +2 0 + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + UNION ALL + SELECT t2a - t0a as d + FROM t2) +) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + UNION ALL + SELECT sum(t2a) + t0a as d + FROM t2) +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"sum(t0a) AS d\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 53, + "stopIndex" : 84, + "fragment" : "SELECT sum(t0a) as d\n FROM t1" + } ] +} + + +-- !query +SELECT t0a, t0b FROM t0 +GROUP BY t0a, t0b +HAVING t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select * +from range(1, 3) t1 +where (select t2.id c + from range (1, 2) t2 where t1.id = t2.id + ) is not null +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 JOIN t2 ON (t1a = t0a AND t2b = t1b)) +) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 JOIN t2 ON (t1a < t0a AND t2b >= t1b)) +) +-- !query schema +struct +-- !query output +2 0 + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 LEFT JOIN t2 ON (t1a = t0a AND t2b = t0b)) +) +-- !query schema +struct +-- !query output +1 1 +2 0 + + +-- !query +select * +from range(1, 3) t1 +where (select t2.id c + from range (1, 2) t2 where t1.id = t2.id + ) between 1 and 2 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT * +FROM t1 +WHERE (SELECT max(t2c) + FROM t2 WHERE t1b = t2b + ) between 1 and 2 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM t0 WHERE t0a = (SELECT distinct(t1c) FROM t1 WHERE t1a = t0a) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT MAX(a.col1) +FROM VALUES (1) AS a(col1) +GROUP BY a.col1 +HAVING COUNT(*) = ( + SELECT COUNT(*) + FROM VALUES (1),(1),(2),(2) AS c(col1) + WHERE c.col1 >= a.col1 + LIMIT 1 + ) +-- !query schema +struct +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out new file mode 100644 index 000000000000..85bd9137602a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out @@ -0,0 +1,609 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 00:00:00.000', date '2014-04-04'), + ('val1b', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1a', 16S, 12, 21L, float(15.0), 20D, 20E2, timestamp '2014-06-04 01:02:00.001', date '2014-06-04'), + ('val1a', 16S, 12, 10L, float(15.0), 20D, 20E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:02:00.001', date '2014-05-05'), + ('val1d', null, 16, 22L, float(17.0), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', null), + ('val1d', null, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-07-04 01:02:00.001', null), + ('val1e', 10S, null, 25L, float(17.0), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-09-04 01:02:00.001', date '2014-09-04'), + ('val1d', 10S, null, 12L, float(17.0), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1a', 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), + ('val1e', 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') + as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2 as select * from values + ('val2a', 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 119L, float(17), 25D, 26E2, timestamp '2015-05-04 01:01:00.000', date '2015-05-04'), + ('val1c', 12S, 16, 219L, float(17), 25D, 26E2, timestamp '2016-05-04 01:01:00.000', date '2016-05-04'), + ('val1b', null, 16, 319L, float(17), 25D, 26E2, timestamp '2017-05-04 01:01:00.000', null), + ('val2e', 8S, null, 419L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1f', 19S, null, 519L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-06-04 01:01:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:01:00.000', date '2014-07-04'), + ('val1c', 12S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-08-04 01:01:00.000', date '2014-08-05'), + ('val1e', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:01:00.000', date '2014-09-04'), + ('val1f', 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), + ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) + as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t3 as select * from values + ('val3a', 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), + ('val3a', 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 219L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val1b', 8S, 16, 319L, float(17), 25D, 26E2, timestamp '2014-06-04 01:02:00.000', date '2014-06-04'), + ('val1b', 8S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-07-04 01:02:00.000', date '2014-07-04'), + ('val3c', 17S, 16, 519L, float(17), 25D, 26E2, timestamp '2014-08-04 01:02:00.000', date '2014-08-04'), + ('val3c', 17S, 16, 19L, float(17), 25D, 26E2, timestamp '2014-09-04 01:02:00.000', date '2014-09-05'), + ('val1b', null, 16, 419L, float(17), 25D, 26E2, timestamp '2014-10-04 01:02:00.000', null), + ('val1b', null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-11-04 01:02:00.000', null), + ('val3b', 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), + ('val3b', 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') + as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT (SELECT min(t3d) FROM t3) min_t3d, + (SELECT max(t2h) FROM t2) max_t2h +FROM t1 +WHERE t1a = 'val1c' +-- !query schema +struct +-- !query output +10 2017-05-04 01:01:00 + + +-- !query +SELECT t1a, count(*) +FROM t1 +WHERE t1c IN (SELECT (SELECT min(t3c) FROM t3) + FROM t2 + GROUP BY t2g + HAVING count(*) > 1) +GROUP BY t1a +-- !query schema +struct +-- !query output +val1a 2 + + +-- !query +SELECT (SELECT min(t3d) FROM t3) min_t3d, + null +FROM t1 +WHERE t1a = 'val1c' +UNION +SELECT null, + (SELECT max(t2h) FROM t2) max_t2h +FROM t1 +WHERE t1a = 'val1c' +-- !query schema +struct +-- !query output +10 NULL +NULL 2017-05-04 01:01:00 + + +-- !query +SELECT (SELECT min(t3c) FROM t3) min_t3d +FROM t1 +WHERE t1a = 'val1a' +INTERSECT +SELECT (SELECT min(t2c) FROM t2) min_t2d +FROM t1 +WHERE t1a = 'val1d' +-- !query schema +struct +-- !query output +12 + + +-- !query +SELECT q1.t1a, q2.t2a, q1.min_t3d, q2.avg_t3d +FROM (SELECT t1a, (SELECT min(t3d) FROM t3) min_t3d + FROM t1 + WHERE t1a IN ('val1e', 'val1c')) q1 + FULL OUTER JOIN + (SELECT t2a, (SELECT avg(t3d) FROM t3) avg_t3d + FROM t2 + WHERE t2a IN ('val1c', 'val2a')) q2 +ON q1.t1a = q2.t2a +AND q1.min_t3d < q2.avg_t3d +-- !query schema +struct +-- !query output +NULL val2a NULL 200.83333333333334 +val1c val1c 10 200.83333333333334 +val1c val1c 10 200.83333333333334 +val1e NULL 10 NULL +val1e NULL 10 NULL +val1e NULL 10 NULL + + +-- !query +SELECT (SELECT min(t3d) FROM t3 WHERE t3.t3a = t1.t1a) min_t3d, + (SELECT max(t2h) FROM t2 WHERE t2.t2a = t1.t1a) max_t2h +FROM t1 +WHERE t1a = 'val1b' +-- !query schema +struct +-- !query output +19 2017-05-04 01:01:00 + + +-- !query +SELECT (SELECT min(t3d) FROM t3 WHERE t3a = t1a) min_t3d +FROM t1 +WHERE t1a = 'val1b' +MINUS +SELECT (SELECT min(t3d) FROM t3) abs_min_t3d +FROM t1 +WHERE t1a = 'val1b' +-- !query schema +struct +-- !query output +19 + + +-- !query +SELECT t1a, t1b +FROM t1 +WHERE NOT EXISTS (SELECT (SELECT max(t2b) + FROM t2 LEFT JOIN t1 + ON t2a = t1a + WHERE t2c = t3c) dummy + FROM t3 + WHERE t3b < (SELECT max(t2b) + FROM t2 LEFT JOIN t1 + ON t2a = t1a + WHERE t2c = t3c) + AND t3a = t1a) +-- !query schema +struct +-- !query output +val1a 16 +val1a 16 +val1a 6 +val1a 6 +val1c 8 +val1d 10 +val1d NULL +val1d NULL +val1e 10 +val1e 10 +val1e 10 + + +-- !query +SELECT t1a, + (SELECT count(t2d) FROM t2 WHERE t2a = t1a) count_t2, + (SELECT count_if(t2d > 0) FROM t2 WHERE t2a = t1a) count_if_t2, + (SELECT approx_count_distinct(t2d) FROM t2 WHERE t2a = t1a) approx_count_distinct_t2, + (SELECT collect_list(t2d) FROM t2 WHERE t2a = t1a) collect_list_t2, + (SELECT sort_array(collect_set(t2d)) FROM t2 WHERE t2a = t1a) collect_set_t2, + (SELECT hex(count_min_sketch(t2d, 0.5d, 0.5d, 1)) FROM t2 WHERE t2a = t1a) collect_set_t2 +FROM t1 +-- !query schema +struct,collect_set_t2:array,collect_set_t2:string> +-- !query output +val1a 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1a 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1a 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1a 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1b 6 6 3 [19,119,319,19,19,19] [19,119,319] 0000000100000000000000060000000100000004000000005D8D6AB90000000000000000000000000000000400000000000000010000000000000001 +val1c 2 2 2 [219,19] [19,219] 0000000100000000000000020000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000001 +val1d 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1d 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1d 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1e 1 1 1 [19] [19] 0000000100000000000000010000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000000 +val1e 1 1 1 [19] [19] 0000000100000000000000010000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000000 +val1e 1 1 1 [19] [19] 0000000100000000000000010000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000000 + + +-- !query +SELECT t1c, (SELECT t1c) FROM t1 +-- !query schema +struct +-- !query output +12 12 +12 12 +16 16 +16 16 +16 16 +16 16 +8 8 +8 8 +NULL NULL +NULL NULL +NULL NULL +NULL NULL + + +-- !query +SELECT t1c, (SELECT t1c WHERE t1c = 8) FROM t1 +-- !query schema +struct +-- !query output +12 NULL +12 NULL +16 NULL +16 NULL +16 NULL +16 NULL +8 8 +8 8 +NULL NULL +NULL NULL +NULL NULL +NULL NULL + + +-- !query +SELECT t1c, t1d, (SELECT c + d FROM (SELECT t1c AS c, t1d AS d)) FROM t1 +-- !query schema +struct +-- !query output +12 10 22 +12 21 33 +16 19 35 +16 19 35 +16 19 35 +16 22 38 +8 10 18 +8 10 18 +NULL 12 NULL +NULL 19 NULL +NULL 19 NULL +NULL 25 NULL + + +-- !query +SELECT t1c, (SELECT SUM(c) FROM (SELECT t1c AS c)) FROM t1 +-- !query schema +struct +-- !query output +12 12 +12 12 +16 16 +16 16 +16 16 +16 16 +8 8 +8 8 +NULL NULL +NULL NULL +NULL NULL +NULL NULL + + +-- !query +SELECT t1a, (SELECT SUM(t2b) FROM t2 JOIN (SELECT t1a AS a) ON t2a = a) FROM t1 +-- !query schema +struct +-- !query output +val1a NULL +val1a NULL +val1a NULL +val1a NULL +val1b 36 +val1c 24 +val1d NULL +val1d NULL +val1d NULL +val1e 8 +val1e 8 +val1e 8 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS VALUES (0, 1), (1, 2) t1(c1, c2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t2 AS VALUES (0, 2), (0, 3) t2(c1, c2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT c1, (WITH t AS (SELECT 1 AS a) SELECT a + c1 FROM t) FROM t1 +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +SELECT c1, (WITH t AS (SELECT * FROM t2 WHERE c1 = t1.c1) SELECT SUM(c2) FROM t) FROM t1 +-- !query schema +struct +-- !query output +0 5 +1 NULL + + +-- !query +SELECT c1, ( + WITH t3 AS (SELECT c1 + 1 AS c1, c2 + 1 AS c2 FROM t2), + t4 AS (SELECT * FROM t3 WHERE t1.c1 = c1) + SELECT SUM(c2) FROM t4 +) FROM t1 +-- !query schema +struct +-- !query output +0 NULL +1 7 + + +-- !query +SELECT c1, ( + WITH t AS (SELECT * FROM t2) + SELECT SUM(c2) FROM (SELECT c1, c2 FROM t UNION SELECT c2, c1 FROM t) r(c1, c2) + WHERE c1 = t1.c1 +) FROM t1 +-- !query schema +struct +-- !query output +0 5 +1 NULL + + +-- !query +WITH v AS (SELECT * FROM t2) +SELECT * FROM t1 WHERE c1 > ( + WITH t AS (SELECT * FROM t2) + SELECT COUNT(*) FROM v WHERE c1 = t1.c1 AND c1 > (SELECT SUM(c2) FROM t WHERE c1 = v.c1) +) +-- !query schema +struct +-- !query output +1 2 + + +-- !query +WITH t AS (SELECT 1 AS a) +SELECT c1, (SELECT a FROM t WHERE a = c1) FROM t1 +-- !query schema +struct +-- !query output +0 NULL +1 1 + + +-- !query +WITH +v1 AS (SELECT c1, c2, rand(0) c3 FROM t1), +v2 AS (SELECT c1, c2, rand(0) c4 FROM v1 WHERE c3 IN (SELECT c3 FROM v1)) +SELECT c1, ( + WITH v3 AS (SELECT c1, c2, rand(0) c5 FROM t2) + SELECT COUNT(*) FROM ( + SELECT * FROM v2 WHERE c1 > 0 + UNION SELECT * FROM v2 WHERE c2 > 0 + UNION SELECT * FROM v3 WHERE c2 > 0 + ) WHERE c1 = v1.c1 +) FROM v1 +-- !query schema +struct +-- !query output +0 3 +1 1 + + +-- !query +SELECT (SELECT a FROM (SELECT 1 AS a UNION ALL SELECT 2 AS a) t) AS b +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkException +{ + "errorClass" : "SCALAR_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "(SELECT a FROM (SELECT 1 AS a UNION ALL SELECT 2 AS a) t)" + } ] +} + + +-- !query +CREATE OR REPLACE TEMP VIEW t1(c1, c2) AS (VALUES (0, 1), (1, 2)) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMP VIEW t2(c1, c2) AS (VALUES (0, 2), (0, 3)) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMP VIEW students(id, name, major, year) AS (VALUES + (0, 'A', 'CS', 2022), + (1, 'B', 'CS', 2022), + (2, 'C', 'Math', 2022)) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMP VIEW exams(sid, course, curriculum, grade, date) AS (VALUES + (0, 'C1', 'CS', 4, 2020), + (0, 'C2', 'CS', 3, 2021), + (1, 'C1', 'CS', 2, 2020), + (1, 'C2', 'CS', 1, 2021)) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT students.name, exams.course +FROM students, exams +WHERE students.id = exams.sid + AND (students.major = 'CS' OR students.major = 'Games Eng') + AND exams.grade >= ( + SELECT avg(exams.grade) + 1 + FROM exams + WHERE students.id = exams.sid + OR (exams.curriculum = students.major AND students.year > exams.date)) +-- !query schema +struct +-- !query output +A C1 + + +-- !query +SELECT (SELECT min(c2) FROM t2 WHERE t1.c1 > t2.c1) FROM t1 +-- !query schema +struct +-- !query output +2 +NULL + + +-- !query +SELECT (SELECT min(c2) FROM t2 WHERE t1.c1 >= t2.c1 AND t1.c2 < t2.c2) FROM t1 +-- !query schema +struct +-- !query output +2 +3 + + +-- !query +SELECT (SELECT count(*) FROM t2 WHERE t1.c1 > t2.c1) FROM t1 +-- !query schema +struct +-- !query output +0 +2 + + +-- !query +SELECT c, ( + SELECT count(*) + FROM (VALUES ('ab'), ('abc'), ('bc')) t2(c) + WHERE t1.c = substring(t2.c, 1, 1) +) FROM (VALUES ('a'), ('b')) t1(c) +-- !query schema +struct +-- !query output +a 2 +b 1 + + +-- !query +SELECT c, ( + SELECT count(*) + FROM (VALUES (0, 6), (1, 5), (2, 4), (3, 3)) t1(a, b) + WHERE a + b = c +) FROM (VALUES (6)) t2(c) +-- !query schema +struct +-- !query output +6 4 + + +-- !query +SELECT *, (SELECT count(1) is null FROM t2 WHERE t1.c1 = t2.c1) FROM t1 +-- !query schema +struct +-- !query output +0 1 false +1 2 false + + +-- !query +select (select f from (select false as f, max(c2) from t1 where t1.c1 = t1.c1)) from t2 +-- !query schema +struct +-- !query output +false +false + + +-- !query +set spark.sql.optimizer.optimizeOneRowRelationSubquery.alwaysInline=false +-- !query schema +struct +-- !query output +spark.sql.optimizer.optimizeOneRowRelationSubquery.alwaysInline false + + +-- !query +WITH T AS (SELECT 1 AS a) +SELECT (SELECT sum(1) FROM T WHERE a = col OR upper(col)= 'Y') +FROM (SELECT null as col) as foo +-- !query schema +struct +-- !query output +NULL + + +-- !query +set spark.sql.optimizer.optimizeOneRowRelationSubquery.alwaysInline=true +-- !query schema +struct +-- !query output +spark.sql.optimizer.optimizeOneRowRelationSubquery.alwaysInline true + + +-- !query +select * from ( + select t1.id c1, ( + select t2.id c from range (1, 2) t2 + where t1.id = t2.id ) c2 + from range (1, 3) t1 ) t +where t.c2 is not null +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT c1, c2, (SELECT count(*) cnt FROM t1 t2 WHERE t1.c1 = t2.c1 HAVING cnt = 0) FROM t1 +-- !query schema +struct +-- !query output +0 1 NULL +1 2 NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out new file mode 100644 index 000000000000..33a57a73be08 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/scalar-subquery/scalar-subquery-set-op.sql.out @@ -0,0 +1,1075 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMP VIEW t0(t0a, t0b) AS VALUES (1, 1), (2, 0) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMP VIEW t1(t1a, t1b, t1c) AS VALUES (1, 1, 3) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMP VIEW t2(t2a, t2b, t2c) AS VALUES (1, 1, 5), (2, 2, 7) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 8 +2 NULL + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +FROM t0 +-- !query schema +struct +-- !query output +1 8 +2 7 + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + UNION ALL + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 5 +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 8 +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 32 +2 NULL + + +-- !query +SELECT t0a, (SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 2 +2 0 + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + UNION ALL + SELECT t2a - t0a as d + FROM t2) +) +FROM t0 +-- !query schema +struct +-- !query output +1 1 +2 -2 + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + UNION ALL + SELECT sum(t2a) + t0a as d + FROM t2) +) +FROM t0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"sum(t0a) AS d\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 67, + "fragment" : "SELECT sum(t0a) as d\n FROM t1" + } ] +} + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 8 +2 NULL + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +FROM t0 +-- !query schema +struct +-- !query output +1 8 +2 7 + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + UNION DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 5 +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 8 +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 32 +2 NULL + + +-- !query +SELECT t0a, (SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + UNION DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 2 +2 0 + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + UNION DISTINCT + SELECT t2a - t0a as d + FROM t2) +) +FROM t0 +-- !query schema +struct +-- !query output +1 1 +2 -1 + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + UNION DISTINCT + SELECT sum(t2a) + t0a as d + FROM t2) +) +FROM t0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"sum(t0a) AS d\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 67, + "fragment" : "SELECT sum(t0a) as d\n FROM t1" + } ] +} + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + INTERSECT ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 NULL +2 NULL + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + INTERSECT ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + INTERSECT ALL + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +FROM t0 +-- !query schema +struct +-- !query output +1 NULL +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + INTERSECT ALL + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 NULL +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + INTERSECT ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 NULL +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + INTERSECT ALL + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 NULL +2 NULL + + +-- !query +SELECT t0a, (SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + INTERSECT ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 0 +2 0 + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + INTERSECT ALL + SELECT t2a - t0a as d + FROM t2) +) +FROM t0 +-- !query schema +struct +-- !query output +1 0 +2 -1 + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + INTERSECT ALL + SELECT sum(t2a) + t0a as d + FROM t2) +) +FROM t0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"sum(t0a) AS d\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 67, + "fragment" : "SELECT sum(t0a) as d\n FROM t1" + } ] +} + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + INTERSECT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 NULL +2 NULL + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + INTERSECT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + INTERSECT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +FROM t0 +-- !query schema +struct +-- !query output +1 NULL +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + INTERSECT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 NULL +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + INTERSECT DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 NULL +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + INTERSECT DISTINCT + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 NULL +2 NULL + + +-- !query +SELECT t0a, (SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + INTERSECT DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 0 +2 0 + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + INTERSECT DISTINCT + SELECT t2a - t0a as d + FROM t2) +) +FROM t0 +-- !query schema +struct +-- !query output +1 0 +2 -1 + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + INTERSECT DISTINCT + SELECT sum(t2a) + t0a as d + FROM t2) +) +FROM t0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"sum(t0a) AS d\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 67, + "fragment" : "SELECT sum(t0a) as d\n FROM t1" + } ] +} + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + EXCEPT ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 3 +2 NULL + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + EXCEPT ALL + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + EXCEPT ALL + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +FROM t0 +-- !query schema +struct +-- !query output +1 3 +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + EXCEPT ALL + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 NULL +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + EXCEPT ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 3 +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + EXCEPT ALL + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 11 +2 NULL + + +-- !query +SELECT t0a, (SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + EXCEPT ALL + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 1 +2 0 + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + EXCEPT ALL + SELECT t2a - t0a as d + FROM t2) +) +FROM t0 +-- !query schema +struct +-- !query output +1 NULL +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + EXCEPT ALL + SELECT sum(t2a) + t0a as d + FROM t2) +) +FROM t0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"sum(t0a) AS d\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 67, + "fragment" : "SELECT sum(t0a) as d\n FROM t1" + } ] +} + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + EXCEPT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 3 +2 NULL + + +-- !query +SELECT * FROM t0 WHERE t0a < +(SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + EXCEPT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b = t0b) +) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a = t0a + EXCEPT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2a = t0a) +) +FROM t0 +-- !query schema +struct +-- !query output +1 3 +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(c) FROM + (SELECT t1c as c + FROM t1 + WHERE t1a > t0a + EXCEPT DISTINCT + SELECT t2c as c + FROM t2 + WHERE t2b <= t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 NULL +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + EXCEPT DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 3 +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM + (SELECT t1c as t1a, t1a as t1b, t0a as t1c + FROM t1 + WHERE t1a = t0a + EXCEPT DISTINCT + SELECT t0a as t2b, t2c as t1a, t0b as t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 11 +2 NULL + + +-- !query +SELECT t0a, (SELECT count(t1c) FROM + (SELECT t1c + FROM t1 + WHERE t1a = t0a + EXCEPT DISTINCT + SELECT t2c + FROM t2 + WHERE t2b = t0b) +) +FROM t0 +-- !query schema +struct +-- !query output +1 1 +2 0 + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT t1a - t0a as d + FROM t1 + EXCEPT DISTINCT + SELECT t2a - t0a as d + FROM t2) +) +FROM t0 +-- !query schema +struct +-- !query output +1 NULL +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(d) FROM + (SELECT sum(t0a) as d + FROM t1 + EXCEPT DISTINCT + SELECT sum(t2a) + t0a as d + FROM t2) +) +FROM t0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"sum(t0a) AS d\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 67, + "fragment" : "SELECT sum(t0a) as d\n FROM t1" + } ] +} + + +-- !query +SELECT t0a, (SELECT sum(t1b) FROM + (SELECT t1b + FROM t1 join t2 ON (t1a = t0a and t1b = t2b) + UNION ALL + SELECT t2b + FROM t1 join t2 ON (t2a = t0a and t1a = t2a)) +) +FROM t0 +-- !query schema +struct +-- !query output +1 2 +2 NULL + + +-- !query +SELECT t0a, (SELECT sum(t1b) FROM + (SELECT t1b + FROM t1 left join t2 ON (t1a = t0a and t1b = t2b) + UNION ALL + SELECT t2b + FROM t1 join t2 ON (t2a = t0a + 1 and t1a = t2a)) +) +FROM t0 +-- !query schema +struct +-- !query output +1 1 +2 1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/subquery-in-from.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/subquery-in-from.sql.out new file mode 100644 index 000000000000..dfb29478467e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/subquery-in-from.sql.out @@ -0,0 +1,47 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT * FROM (SELECT * FROM testData) AS t WHERE key = 1 +-- !query schema +struct +-- !query output +1 1 + + +-- !query +FROM (SELECT * FROM testData WHERE key = 1) AS t SELECT * +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT * FROM (SELECT * FROM testData) t WHERE key = 1 +-- !query schema +struct +-- !query output +1 1 + + +-- !query +FROM (SELECT * FROM testData WHERE key = 1) t SELECT * +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT * FROM (SELECT * FROM testData) WHERE key = 1 +-- !query schema +struct +-- !query output +1 1 + + +-- !query +FROM (SELECT * FROM testData WHERE key = 1) SELECT * +-- !query schema +struct +-- !query output +1 1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/subquery-nested-data.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/subquery-nested-data.sql.out new file mode 100644 index 000000000000..e0fd040ca4bc --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/subquery-nested-data.sql.out @@ -0,0 +1,314 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +drop table if exists x +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table if exists y +-- !query schema +struct<> +-- !query output + + + +-- !query +create table x(xm map, x2 int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into x values (map(1, 2), 3), (map(1, 4), 5), (map(2, 3), 4), (map(5, 6), 7) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table y(ym map, y2 int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into y values (map(1, 2), 10), (map(1, 3), 20), (map(2, 3), 20), (map(8, 3), 20) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from x where (select sum(y2) from y where xm[1] = ym[1]) > 2 +-- !query schema +struct,x2:int> +-- !query output +{1:2} 3 + + +-- !query +select * from x where exists (select * from y where xm[1] = ym[1]) +-- !query schema +struct,x2:int> +-- !query output +{1:2} 3 + + +-- !query +select * from x where exists (select * from y where xm[1] = ym[1] limit 1) +-- !query schema +struct,x2:int> +-- !query output +{1:2} 3 + + +-- !query +select * from x join lateral (select * from y where xm[1] = ym[1]) +-- !query schema +struct,x2:int,ym:map,y2:int> +-- !query output +{1:2} 3 {1:2} 10 + + +-- !query +select * from x join lateral (select * from y where xm[1] = ym[1] union all select * from y where xm[1] = ym[1] + 1) +-- !query schema +struct,x2:int,ym:map,y2:int> +-- !query output +{1:2} 3 {1:2} 10 +{1:4} 5 {1:3} 20 + + +-- !query +select * from x join lateral (select * from y where xm[1] = ym[1] limit 1) +-- !query schema +struct,x2:int,ym:map,y2:int> +-- !query output +{1:2} 3 {1:2} 10 + + +-- !query +select * from x join lateral (select count(*) from y where xm[1] = ym[1] group by y2) +-- !query schema +struct,x2:int,count(1):bigint> +-- !query output +{1:2} 3 1 + + +-- !query +select * from x where xm[1] in (select ym[1] from y) +-- !query schema +struct,x2:int> +-- !query output +{1:2} 3 + + +-- !query +select * from x where xm[1] in (select sum(ym[1]) from y group by y2) +-- !query schema +struct,x2:int> +-- !query output +{1:2} 3 + + +-- !query +select * from x where (select sum(y2) from y where xm[1] = ym[1] and xm[1] >= 1) > 2 +-- !query schema +struct,x2:int> +-- !query output +{1:2} 3 + + +-- !query +select * from x where (select sum(y2) from y where xm[1] = ym[1] and xm[2] >= ym[2]) > 2 +-- !query schema +struct,x2:int> +-- !query output + + + +-- !query +select * from x where (select sum(y2) from y where xm[1] = ym[1]) > 2 and (select count(y2) from y where xm[1] = ym[1]) < 3 +-- !query schema +struct,x2:int> +-- !query output +{1:2} 3 + + +-- !query +select * from x join lateral (select xm[1] - ym[1] from y) +-- !query schema +struct,x2:int,(outer(spark_catalog.default.x.xm)[1] - ym[1]):int> +-- !query output +{1:2} 3 -1 +{1:2} 3 0 +{1:2} 3 NULL +{1:2} 3 NULL +{1:4} 5 1 +{1:4} 5 2 +{1:4} 5 NULL +{1:4} 5 NULL +{2:3} 4 NULL +{2:3} 4 NULL +{2:3} 4 NULL +{2:3} 4 NULL +{5:6} 7 NULL +{5:6} 7 NULL +{5:6} 7 NULL +{5:6} 7 NULL + + +-- !query +select * from x join lateral (select xm[1], xm[1] as s1, xm[1] - ym[1] as s2 from y) +-- !query schema +struct,x2:int,outer(spark_catalog.default.x.xm)[1]:int,s1:int,s2:int> +-- !query output +{1:2} 3 2 2 -1 +{1:2} 3 2 2 0 +{1:2} 3 2 2 NULL +{1:2} 3 2 2 NULL +{1:4} 5 4 4 1 +{1:4} 5 4 4 2 +{1:4} 5 4 4 NULL +{1:4} 5 4 4 NULL +{2:3} 4 NULL NULL NULL +{2:3} 4 NULL NULL NULL +{2:3} 4 NULL NULL NULL +{2:3} 4 NULL NULL NULL +{5:6} 7 NULL NULL NULL +{5:6} 7 NULL NULL NULL +{5:6} 7 NULL NULL NULL +{5:6} 7 NULL NULL NULL + + +-- !query +select * from x join lateral (select xm[1], sum(ym[1]), xm[1] - sum(ym[1]) from y group by xm[1]) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE", + "sqlState" : "0A000", + "messageParameters" : { + "sqlExprs" : "\"xm[1]\",\"xm[1] AS `outer(spark_catalog.default.x.xm)[1]`\",\"(xm[1] - sum(ym[1])) AS `(outer(spark_catalog.default.x.xm)[1] - sum(ym[1]))`\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 83, + "stopIndex" : 96, + "fragment" : "group by xm[1]" + } ] +} + + +-- !query +select * from x where (select sum(y2) from y where xm[x2] = ym[1]) > 2 +-- !query schema +struct,x2:int> +-- !query output + + + +-- !query +select * from x where (select sum(y2) from y where xm[x2+1] = ym[1]) > 2 +-- !query schema +struct,x2:int> +-- !query output + + + +-- !query +select * from x where (select sum(y2) from y where xm[x2+1] = ym[1] and xm[1+x2] = ym[2]) > 2 +-- !query schema +struct,x2:int> +-- !query output + + + +-- !query +select * from x where (select sum(y2) from y where xm[y2] = ym[1]) > 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_REFERENCE_DATA_TYPE", + "sqlState" : "0A000", + "messageParameters" : { + "dataType" : "map", + "expr" : "spark_catalog.default.x.xm" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 44, + "fragment" : "y" + } ] +} + + +-- !query +set spark.sql.optimizer.pullOutNestedDataOuterRefExpressions.enabled = false +-- !query schema +struct +-- !query output +spark.sql.optimizer.pullOutNestedDataOuterRefExpressions.enabled false + + +-- !query +select * from x where (select sum(y2) from y where xm[1] = ym[1]) > 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.UNSUPPORTED_CORRELATED_REFERENCE_DATA_TYPE", + "sqlState" : "0A000", + "messageParameters" : { + "dataType" : "map", + "expr" : "spark_catalog.default.x.xm" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 44, + "fragment" : "y" + } ] +} + + +-- !query +reset spark.sql.optimizer.rewriteNestedDataCorrelation.enabled +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table x +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table y +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/subquery-offset.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/subquery-offset.sql.out new file mode 100644 index 000000000000..d547b4272ded --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/subquery/subquery-offset.sql.out @@ -0,0 +1,222 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +drop table if exists x +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table if exists y +-- !query schema +struct<> +-- !query output + + + +-- !query +create table x(x1 int, x2 int) using json +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into x values (1, 1), (2, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create table y(y1 int, y2 int) using json +-- !query schema +struct<> +-- !query output + + + +-- !query +insert into y values (1, 1), (1, 2), (2, 4) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from x where exists (select * from y where x1 = y1 limit 1 offset 2) +-- !query schema +struct +-- !query output + + + +-- !query +select * from x join lateral (select * from y where x1 = y1 limit 1 offset 2) +-- !query schema +struct +-- !query output + + + +-- !query +select * from x where x1 in (select y1 from y limit 1 offset 2) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select * from x where (select sum(y2) from y where x1 = y1 limit 1 offset 2) > 2 +-- !query schema +struct +-- !query output + + + +-- !query +select * from x where exists (select * from y where x1 = y1 offset 2) +-- !query schema +struct +-- !query output + + + +-- !query +select * from x join lateral (select * from y where x1 = y1 offset 2) +-- !query schema +struct +-- !query output + + + +-- !query +select * from x where x1 in (select y1 from y offset 2) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select * from x where (select sum(y2) from y where x1 = y1 offset 2) > 2 +-- !query schema +struct +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (100, "emp 1", date "2005-01-01", 100.00D, 10), + (200, "emp 2", date "2003-01-01", 200.00D, 10), + (300, "emp 3", date "2002-01-01", 300.00D, 20), + (400, "emp 4", date "2005-01-01", 400.00D, 30), + (500, "emp 5", date "2001-01-01", 400.00D, NULL), + (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100), + (700, "emp 7", date "2010-01-01", 400.00D, 100), + (800, "emp 8", date "2016-01-01", 150.00D, 70) +AS EMP(id, emp_name, hiredate, salary, dept_id) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES + (10, "dept 1", "CA"), + (20, "dept 2", "NY"), + (30, "dept 3", "TX"), + (40, "dept 4 - unassigned", "OR"), + (50, "dept 5 - unassigned", "NJ"), + (70, "dept 7", "FL") +AS DEPT(dept_id, dept_name, state) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT emp_name +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY state + ORDER BY state + LIMIT 2 + OFFSET 1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT emp_name +FROM emp +JOIN LATERAL (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY state + ORDER BY state + LIMIT 2 + OFFSET 1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT emp_name +FROM emp +WHERE EXISTS (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY state + ORDER BY state + OFFSET 1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT emp_name +FROM emp +JOIN LATERAL (SELECT max(dept.dept_id) a + FROM dept + WHERE dept.dept_id = emp.dept_id + GROUP BY state + ORDER BY state + OFFSET 1) +-- !query schema +struct +-- !query output + + + +-- !query +drop table x +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table y +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/table-aliases.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/table-aliases.sql.out new file mode 100644 index 000000000000..ce9f0ecbd1b5 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/table-aliases.sql.out @@ -0,0 +1,207 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES (1, 1), (1, 2), (2, 1) AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM testData AS t(col1, col2) WHERE col1 = 1 +-- !query schema +struct +-- !query output +1 1 +1 2 + + +-- !query +SELECT * FROM testData AS t(col1, col2) WHERE col1 = 2 +-- !query schema +struct +-- !query output +2 1 + + +-- !query +SELECT * FROM testData AS t(col1, col2, col3) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "ASSIGNMENT_ARITY_MISMATCH", + "sqlState" : "42802", + "messageParameters" : { + "numExpr" : "3", + "numTarget" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 45, + "fragment" : "testData AS t(col1, col2, col3)" + } ] +} + + +-- !query +SELECT * FROM testData AS t(col1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "ASSIGNMENT_ARITY_MISMATCH", + "sqlState" : "42802", + "messageParameters" : { + "numExpr" : "1", + "numTarget" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 33, + "fragment" : "testData AS t(col1)" + } ] +} + + +-- !query +SELECT a AS col1, b AS col2 FROM testData AS t(c, d) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`", + "proposal" : "`c`, `d`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 8, + "fragment" : "a" + } ] +} + + +-- !query +SELECT * FROM (SELECT 1 AS a, 1 AS b) t(col1, col2) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT t.* FROM (SELECT 1 AS a, 1 AS b) t(col1, col2) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT col1, col2 FROM (SELECT 1 AS a, 1 AS b) t(col1, col2) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT t.col1, t.col2 FROM (SELECT 1 AS a, 1 AS b) t(col1, col2) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW src1 AS SELECT * FROM VALUES (1, "a"), (2, "b"), (3, "c") AS src1(id, v1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW src2 AS SELECT * FROM VALUES (2, 1.0), (3, 3.2), (1, 8.5) AS src2(id, v2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d) +-- !query schema +struct +-- !query output +1 a 1 8.5 +2 b 2 1.0 +3 c 3 3.2 + + +-- !query +SELECT dst.* FROM (src1 s1 INNER JOIN src2 s2 ON s1.id = s2.id) dst(a, b, c, d) +-- !query schema +struct +-- !query output +1 a 1 8.5 +2 b 2 1.0 +3 c 3 3.2 + + +-- !query +SELECT src1.* FROM src1 a ORDER BY id LIMIT 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CANNOT_RESOLVE_STAR_EXPAND", + "sqlState" : "42704", + "messageParameters" : { + "columns" : "`id`, `v1`", + "targetString" : "`src1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 13, + "fragment" : "src1.*" + } ] +} + + +-- !query +SELECT src1.id FROM (SELECT * FROM src1 ORDER BY id LIMIT 1) a +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`src1`.`id`", + "proposal" : "`a`.`id`, `a`.`v1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 14, + "fragment" : "src1.id" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/table-valued-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/table-valued-functions.sql.out new file mode 100644 index 000000000000..768f0e8c010e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/table-valued-functions.sql.out @@ -0,0 +1,1071 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select * from dummy(3) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVABLE_TABLE_VALUED_FUNCTION", + "sqlState" : "42883", + "messageParameters" : { + "name" : "`dummy`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 22, + "fragment" : "dummy(3)" + } ] +} + + +-- !query +select * from range(6 + cos(3)) +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 + + +-- !query +select * from range(5, 10) +-- !query schema +struct +-- !query output +5 +6 +7 +8 +9 + + +-- !query +select * from range(0, 10, 2) +-- !query schema +struct +-- !query output +0 +2 +4 +6 +8 + + +-- !query +select * from range(0, 10, 1, 200) +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +select * from range(1, 1, 1, 1, 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "5", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[1, 2, 3, 4]", + "functionName" : "`range`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 34, + "fragment" : "range(1, 1, 1, 1, 1)" + } ] +} + + +-- !query +select * from range(1, null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`range`", + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "second", + "requiredType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 28, + "fragment" : "range(1, null)" + } ] +} + + +-- !query +select * from range(array(1, 2, 3)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`range`", + "inputSql" : "\"array(1, 2, 3)\"", + "inputType" : "\"ARRAY\"", + "paramIndex" : "second", + "requiredType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 35, + "fragment" : "range(array(1, 2, 3))" + } ] +} + + +-- !query +select * from range(0, 5, 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "FAILED_FUNCTION_CALL", + "sqlState" : "38000", + "messageParameters" : { + "funcName" : "`range`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 28, + "fragment" : "range(0, 5, 0)" + } ] +} + + +-- !query +select * from RaNgE(2) +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +select i from range(0, 2) t(i) +-- !query schema +struct +-- !query output +0 +1 + + +-- !query +select * from range(0, (select 1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NON_FOLDABLE_ARGUMENT", + "sqlState" : "42K08", + "messageParameters" : { + "funcName" : "`range`", + "paramName" : "`end`", + "paramType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 34, + "fragment" : "range(0, (select 1))" + } ] +} + + +-- !query +select * from values (0, 1) t(c1, c2), lateral range(0, c2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NON_FOLDABLE_ARGUMENT", + "sqlState" : "42K08", + "messageParameters" : { + "funcName" : "`range`", + "paramName" : "`end`", + "paramType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 48, + "stopIndex" : 59, + "fragment" : "range(0, c2)" + } ] +} + + +-- !query +select * from explode(array(1, 2)) +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +select * from explode(map('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +a 1 +b 2 + + +-- !query +select * from explode(array()) +-- !query schema +struct +-- !query output + + + +-- !query +select * from explode(map()) +-- !query schema +struct +-- !query output + + + +-- !query +select * from explode(array(1, 2)) t(c1) +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +select * from explode(map('a', 1, 'b', 2)) t(k, v) +-- !query schema +struct +-- !query output +a 1 +b 2 + + +-- !query +select * from explode(array(rand(0))) +-- !query schema +struct +-- !query output +0.7604953758285915 + + +-- !query +select * from explode(null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "first", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"explode(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 27, + "fragment" : "explode(null)" + } ] +} + + +-- !query +select * from explode(null) t(c1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "first", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"explode(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 33, + "fragment" : "explode(null) t(c1)" + } ] +} + + +-- !query +select * from explode(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"explode(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 24, + "fragment" : "explode(1)" + } ] +} + + +-- !query +select * from explode(1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`explode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 27, + "fragment" : "explode(1, 2)" + } ] +} + + +-- !query +select * from explode(explode(array(1))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + "sqlState" : "42K0E", + "messageParameters" : { + "expression" : "\"explode(explode(array(1)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 40, + "fragment" : "explode(explode(array(1)))" + } ] +} + + +-- !query +select * from explode(array(1, 2)) t(c1, c2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "sqlState" : "42826", + "messageParameters" : { + "aliasesNum" : "2", + "funcName" : "`explode`", + "outColsNum" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 44, + "fragment" : "explode(array(1, 2)) t(c1, c2)" + } ] +} + + +-- !query +select * from explode_outer(array(1, 2)) +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +select * from explode_outer(map('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +a 1 +b 2 + + +-- !query +select * from explode_outer(array()) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select * from explode_outer(map()) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +select * from range(2) join explode(array(1, 2)) +-- !query schema +struct +-- !query output +0 1 +0 2 +1 1 +1 2 + + +-- !query +select * from range(2) join explode_outer(array()) +-- !query schema +struct +-- !query output +0 NULL +1 NULL + + +-- !query +select * from inline(array(struct(1, 'a'), struct(2, 'b'))) +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +select * from inline(array(struct(1, 'a'), struct(2, 'b'))) t(x, y) +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +select * from inline(array_remove(array(struct(1, 'a')), struct(1, 'a'))) +-- !query schema +struct +-- !query output + + + +-- !query +select * from inline(null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"NULL\"", + "inputType" : "\"VOID\"", + "paramIndex" : "first", + "requiredType" : "\"ARRAY\"", + "sqlExpr" : "\"inline(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 26, + "fragment" : "inline(null)" + } ] +} + + +-- !query +select * from inline(array(struct(1, 2), struct(2, 3))) t(a, b, c) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "sqlState" : "42826", + "messageParameters" : { + "aliasesNum" : "3", + "funcName" : "`inline`", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 66, + "fragment" : "inline(array(struct(1, 2), struct(2, 3))) t(a, b, c)" + } ] +} + + +-- !query +select * from inline_outer(array(struct(1, 'a'), struct(2, 'b'))) +-- !query schema +struct +-- !query output +1 a +2 b + + +-- !query +select * from inline_outer(array_remove(array(struct(1, 'a')), struct(1, 'a'))) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +select * from posexplode(array()) +-- !query schema +struct +-- !query output + + + +-- !query +select * from posexplode(array(1, 2)) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +select * from posexplode(array(1, 2)) t(pos, x) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +select * from posexplode(map()) +-- !query schema +struct +-- !query output + + + +-- !query +select * from posexplode(map('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +0 a 1 +1 b 2 + + +-- !query +select * from posexplode(map('a', 1, 'b', 2)) t(pos, k, v) +-- !query schema +struct +-- !query output +0 a 1 +1 b 2 + + +-- !query +select * from posexplode(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "(\"ARRAY\" or \"MAP\")", + "sqlExpr" : "\"posexplode(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 27, + "fragment" : "posexplode(1)" + } ] +} + + +-- !query +select * from posexplode(1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`posexplode`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 30, + "fragment" : "posexplode(1, 2)" + } ] +} + + +-- !query +select * from posexplode(explode(array(1))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + "sqlState" : "42K0E", + "messageParameters" : { + "expression" : "\"posexplode(explode(array(1)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 43, + "fragment" : "posexplode(explode(array(1)))" + } ] +} + + +-- !query +select * from posexplode(array(1, 2)) t(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "sqlState" : "42826", + "messageParameters" : { + "aliasesNum" : "1", + "funcName" : "`posexplode`", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 42, + "fragment" : "posexplode(array(1, 2)) t(x)" + } ] +} + + +-- !query +select * from posexplode_outer(array()) +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +select * from posexplode_outer(array(1, 2)) +-- !query schema +struct +-- !query output +0 1 +1 2 + + +-- !query +select * from posexplode_outer(map()) +-- !query schema +struct +-- !query output +NULL NULL NULL + + +-- !query +select * from posexplode_outer(map('a', 1, 'b', 2)) +-- !query schema +struct +-- !query output +0 a 1 +1 b 2 + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') +-- !query schema +struct +-- !query output +1 2 + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'c') +-- !query schema +struct +-- !query output +1 NULL + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'a') +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x, y) +-- !query schema +struct +-- !query output +1 2 + + +-- !query +select * from json_tuple('{"a": bad, "b": string}', 'a', 'b') +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +select * from json_tuple() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 1", + "functionName" : "`json_tuple`" + } +} + + +-- !query +select * from json_tuple('{"a": 1}') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 1", + "functionName" : "`json_tuple`" + } +} + + +-- !query +select * from json_tuple('{"a": 1}', 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_STRING_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "funcName" : "`json_tuple`", + "sqlExpr" : "\"json_tuple({\"a\": 1}, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 39, + "fragment" : "json_tuple('{\"a\": 1}', 1)" + } ] +} + + +-- !query +select * from json_tuple('{"a": 1}', null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_STRING_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "funcName" : "`json_tuple`", + "sqlExpr" : "\"json_tuple({\"a\": 1}, NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 42, + "fragment" : "json_tuple('{\"a\": 1}', null)" + } ] +} + + +-- !query +select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "sqlState" : "42826", + "messageParameters" : { + "aliasesNum" : "1", + "funcName" : "`json_tuple`", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 62, + "fragment" : "json_tuple('{\"a\": 1, \"b\": 2}', 'a', 'b') AS t(x)" + } ] +} + + +-- !query +select * from stack(1, 1, 2, 3) +-- !query schema +struct +-- !query output +1 2 3 + + +-- !query +select * from stack(2, 1, 2, 3) +-- !query schema +struct +-- !query output +1 2 +3 NULL + + +-- !query +select * from stack(3, 1, 2, 3) t(x) +-- !query schema +struct +-- !query output +1 +2 +3 + + +-- !query +select * from stack(4, 1, 2, 3) t(x) +-- !query schema +struct +-- !query output +1 +2 +3 +NULL + + +-- !query +select * from stack(2, 1, 1.1, 'a', 2, 2.2, 'b') t(a, b, c) +-- !query schema +struct +-- !query output +1 1.1 a +2 2.2 b + + +-- !query +select * from stack(2, 1, 1.1, null, 2, null, 'b') t(a, b, c) +-- !query schema +struct +-- !query output +1 1.1 NULL +2 NULL b + + +-- !query +select * from stack() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 1", + "functionName" : "`stack`" + } +} + + +-- !query +select * from stack(2, 1, 2, 3) t(a, b, c) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NUM_TABLE_VALUE_ALIASES_MISMATCH", + "sqlState" : "42826", + "messageParameters" : { + "aliasesNum" : "3", + "funcName" : "`stack`", + "outColsNum" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 42, + "fragment" : "stack(2, 1, 2, 3) t(a, b, c)" + } ] +} + + +-- !query +select * from stack(2, 1, '1.1', 'a', 2, 2.2, 'b') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.STACK_COLUMN_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "columnIndex" : "1", + "leftParamIndex" : "2", + "leftType" : "\"STRING\"", + "rightParamIndex" : "5", + "rightType" : "\"DECIMAL(2,1)\"", + "sqlExpr" : "\"stack(2, 1, 1.1, a, 2, 2.2, b)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 50, + "fragment" : "stack(2, 1, '1.1', 'a', 2, 2.2, 'b')" + } ] +} + + +-- !query +select * from stack(2, explode(array(1, 2, 3))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GENERATOR.NESTED_IN_EXPRESSIONS", + "sqlState" : "42K0E", + "messageParameters" : { + "expression" : "\"stack(2, explode(array(1, 2, 3)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 47, + "fragment" : "stack(2, explode(array(1, 2, 3)))" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/tablesample-negative.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/tablesample-negative.sql.out new file mode 100644 index 000000000000..fd852992397d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/tablesample-negative.sql.out @@ -0,0 +1,73 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE DATABASE mydb1 +-- !query schema +struct<> +-- !query output + + + +-- !query +USE mydb1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t1 USING parquet AS SELECT 1 AS i1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT mydb1.t1 FROM t1 TABLESAMPLE (-1 PERCENT) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Sampling fraction (-0.01) must be on interval [0, 1]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 48, + "fragment" : "TABLESAMPLE (-1 PERCENT)" + } ] +} + + +-- !query +SELECT mydb1.t1 FROM t1 TABLESAMPLE (101 PERCENT) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Sampling fraction (1.01) must be on interval [0, 1]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 49, + "fragment" : "TABLESAMPLE (101 PERCENT)" + } ] +} + + +-- !query +DROP DATABASE mydb1 CASCADE +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestamp-ltz.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestamp-ltz.sql.out new file mode 100644 index 000000000000..963505615225 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestamp-ltz.sql.out @@ -0,0 +1,70 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select timestamp_ltz'2016-12-31 00:12:00', timestamp_ltz'2016-12-31' +-- !query schema +struct +-- !query output +2016-12-31 00:12:00 2016-12-31 00:00:00 + + +-- !query +select to_timestamp_ltz(null), to_timestamp_ltz('2016-12-31 00:12:00'), to_timestamp_ltz('2016-12-31', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 + + +-- !query +select to_timestamp_ltz(to_date(null)), to_timestamp_ltz(to_date('2016-12-31')) +-- !query schema +struct +-- !query output +NULL 2016-12-31 00:00:00 + + +-- !query +select to_timestamp_ltz(to_timestamp_ntz(null)), to_timestamp_ltz(to_timestamp_ntz('2016-12-31 00:12:00')) +-- !query schema +struct +-- !query output +NULL 2016-12-31 00:12:00 + + +-- !query +SELECT make_timestamp_ltz(2021, 07, 11, 6, 30, 45.678) +-- !query schema +struct +-- !query output +2021-07-11 06:30:45.678 + + +-- !query +SELECT make_timestamp_ltz(2021, 07, 11, 6, 30, 45.678, 'CET') +-- !query schema +struct +-- !query output +2021-07-10 21:30:45.678 + + +-- !query +SELECT make_timestamp_ltz(2021, 07, 11, 6, 30, 60.007) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "INVALID_FRACTION_OF_SECOND", + "sqlState" : "22023", + "messageParameters" : { + "secAndMicros" : "60.007" + } +} + + +-- !query +SELECT convert_timezone('Europe/Brussels', timestamp_ltz'2022-03-23 00:00:00 America/Los_Angeles') +-- !query schema +struct +-- !query output +2022-03-23 08:00:00 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestamp-ntz.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestamp-ntz.sql.out new file mode 100644 index 000000000000..9e37bf4e9caa --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestamp-ntz.sql.out @@ -0,0 +1,231 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select timestamp_ntz'2016-12-31 00:12:00', timestamp_ntz'2016-12-31' +-- !query schema +struct +-- !query output +2016-12-31 00:12:00 2016-12-31 00:00:00 + + +-- !query +select to_timestamp_ntz(null), to_timestamp_ntz('2016-12-31 00:12:00'), to_timestamp_ntz('2016-12-31', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 + + +-- !query +select to_timestamp_ntz(to_date(null)), to_timestamp_ntz(to_date('2016-12-31')) +-- !query schema +struct +-- !query output +NULL 2016-12-31 00:00:00 + + +-- !query +select to_timestamp_ntz(to_timestamp_ltz(null)), to_timestamp_ntz(to_timestamp_ltz('2016-12-31 00:12:00')) +-- !query schema +struct +-- !query output +NULL 2016-12-31 00:12:00 + + +-- !query +SELECT make_timestamp_ntz(2021, 07, 11, 6, 30, 45.678) +-- !query schema +struct +-- !query output +2021-07-11 06:30:45.678 + + +-- !query +SELECT make_timestamp_ntz(2021, 07, 11, 6, 30, 45.678, 'CET') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "7", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "6", + "functionName" : "`make_timestamp_ntz`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "make_timestamp_ntz(2021, 07, 11, 6, 30, 45.678, 'CET')" + } ] +} + + +-- !query +SELECT make_timestamp_ntz(2021, 07, 11, 6, 30, 60.007) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "INVALID_FRACTION_OF_SECOND", + "sqlState" : "22023", + "messageParameters" : { + "secAndMicros" : "60.007" + } +} + + +-- !query +SELECT convert_timezone('Europe/Moscow', 'America/Los_Angeles', timestamp_ntz'2022-01-01 00:00:00') +-- !query schema +struct +-- !query output +2021-12-31 13:00:00 + + +-- !query +SELECT convert_timezone('Europe/Brussels', timestamp_ntz'2022-03-23 00:00:00') +-- !query schema +struct +-- !query output +2022-03-23 08:00:00 + + +-- !query +select timestampdiff(QUARTER, timestamp_ntz'2022-01-01 01:02:03', timestamp_ntz'2022-05-02 05:06:07') +-- !query schema +struct +-- !query output +1 + + +-- !query +select timestampdiff(HOUR, timestamp_ntz'2022-02-14 01:02:03', timestamp_ltz'2022-02-14 02:03:04') +-- !query schema +struct +-- !query output +1 + + +-- !query +select timestampdiff(YEAR, date'2022-02-15', timestamp_ntz'2023-02-15 10:11:12') +-- !query schema +struct +-- !query output +1 + + +-- !query +select timestampdiff(MILLISECOND, timestamp_ntz'2022-02-14 23:59:59.123', date'2022-02-15') +-- !query schema +struct +-- !query output +877 + + +-- !query +select timestamp_ntz'2022-01-01 00:00:00' = date'2022-01-01' +-- !query schema +struct<(TIMESTAMP_NTZ '2022-01-01 00:00:00' = DATE '2022-01-01'):boolean> +-- !query output +true + + +-- !query +select timestamp_ntz'2022-01-01 00:00:00' > date'2022-01-01' +-- !query schema +struct<(TIMESTAMP_NTZ '2022-01-01 00:00:00' > DATE '2022-01-01'):boolean> +-- !query output +false + + +-- !query +select timestamp_ntz'2022-01-01 00:00:00' < date'2022-01-01' +-- !query schema +struct<(TIMESTAMP_NTZ '2022-01-01 00:00:00' < DATE '2022-01-01'):boolean> +-- !query output +false + + +-- !query +select timestamp_ntz'2022-01-01 00:00:00' = timestamp_ltz'2022-01-01 00:00:00' +-- !query schema +struct<(TIMESTAMP_NTZ '2022-01-01 00:00:00' = TIMESTAMP '2022-01-01 00:00:00'):boolean> +-- !query output +true + + +-- !query +select timestamp_ntz'2022-01-01 00:00:00' > timestamp_ltz'2022-01-01 00:00:00' +-- !query schema +struct<(TIMESTAMP_NTZ '2022-01-01 00:00:00' > TIMESTAMP '2022-01-01 00:00:00'):boolean> +-- !query output +false + + +-- !query +select timestamp_ntz'2022-01-01 00:00:00' < timestamp_ltz'2022-01-01 00:00:00' +-- !query schema +struct<(TIMESTAMP_NTZ '2022-01-01 00:00:00' < TIMESTAMP '2022-01-01 00:00:00'):boolean> +-- !query output +false + + +-- !query +CREATE TABLE a (a timestamp_ntz, b int) using parquet PARTITIONED BY(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO a PARTITION(a=timestamp_ntz'2018-11-17 13:33:33') VALUES (1) +-- !query schema +struct<> +-- !query output + + + +-- !query +DESC FORMATTED a +-- !query schema +struct +-- !query output +b int +a timestamp_ntz +# Partition Information +# col_name data_type comment +a timestamp_ntz + +# Detailed Table Information +Catalog spark_catalog +Database default +Table a +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type MANAGED +Provider parquet +Location [not included in comparison]/{warehouse_dir}/a +Partition Provider Catalog + + +-- !query +SELECT * FROM a +-- !query schema +struct +-- !query output +1 2018-11-17 13:33:33 + + +-- !query +DROP TABLE a +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestamp.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestamp.sql.out new file mode 100644 index 000000000000..06a8fbad8a29 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestamp.sql.out @@ -0,0 +1,1222 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select timestamp '2019-01-01\t' +-- !query schema +struct +-- !query output +2019-01-01 00:00:00 + + +-- !query +select timestamp '2019-01-01中文' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2019-01-01中文'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "timestamp '2019-01-01中文'" + } ] +} + + +-- !query +select timestamp'4294967297' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'4294967297'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "timestamp'4294967297'" + } ] +} + + +-- !query +select timestamp'2021-01-01T12:30:4294967297.123456' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2021-01-01T12:30:4294967297.123456'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'" + } ] +} + + +-- !query +select current_timestamp = current_timestamp +-- !query schema +struct<(current_timestamp() = current_timestamp()):boolean> +-- !query output +true + + +-- !query +select current_timestamp() = current_timestamp() +-- !query schema +struct<(current_timestamp() = current_timestamp()):boolean> +-- !query output +true + + +-- !query +select localtimestamp() = localtimestamp() +-- !query schema +struct<(localtimestamp() = localtimestamp()):boolean> +-- !query output +true + + +-- !query +SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678) +-- !query schema +struct +-- !query output +2021-07-11 06:30:45.678 + + +-- !query +SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678, 'CET') +-- !query schema +struct +-- !query output +2021-07-10 21:30:45.678 + + +-- !query +SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "INVALID_FRACTION_OF_SECOND", + "sqlState" : "22023", + "messageParameters" : { + "secAndMicros" : "60.007" + } +} + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 1) +-- !query schema +struct +-- !query output +0001-01-01 01:01:01 + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 60) +-- !query schema +struct +-- !query output +0001-01-01 01:02:00 + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 61) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for SecondOfMinute (valid values 0 - 59): 61" + } +} + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 59.999999) +-- !query schema +struct +-- !query output +0001-01-01 01:01:59.999999 + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for SecondOfMinute (valid values 0 - 59): 99" + } +} + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for SecondOfMinute (valid values 0 - 59): 999" + } +} + + +-- !query +select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00 1931-01-07 00:30:00 NULL + + +-- !query +select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23)) +-- !query schema +struct +-- !query output +1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 + + +-- !query +select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00.123 1931-01-07 00:29:59.877 NULL + + +-- !query +select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00.123123 1931-01-07 00:29:59.876877 NULL + + +-- !query +select TIMESTAMP_SECONDS(1230219000123123) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_SECONDS(-1230219000123123) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_MILLIS(92233720368547758) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_MILLIS(-92233720368547758) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_SECONDS(0.1234567) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Rounding necessary + + +-- !query +select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567)) +-- !query schema +struct +-- !query output +1969-12-31 16:00:00.123456 1969-12-31 16:00:00.123456 + + +-- !query +create temporary view ttf1 as select * from values + (1, 2), + (2, 3) + as ttf1(`current_date`, `current_timestamp`) +-- !query schema +struct<> +-- !query output + + + +-- !query +select typeof(current_date), typeof(current_timestamp) from ttf1 +-- !query schema +struct +-- !query output +int int +int int + + +-- !query +create temporary view ttf2 as select * from values + (1, 2), + (2, 3) + as ttf2(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2 +-- !query schema +struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean,a:int,b:int> +-- !query output +true true 1 2 +true true 2 3 + + +-- !query +select a, b from ttf2 order by a, current_date +-- !query schema +struct +-- !query output +1 2 +2 3 + + +-- !query +select UNIX_SECONDS(timestamp'2020-12-01 14:30:08Z'), UNIX_SECONDS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_SECONDS(null) +-- !query schema +struct +-- !query output +1606833008 1606833008 NULL + + +-- !query +select UNIX_MILLIS(timestamp'2020-12-01 14:30:08Z'), UNIX_MILLIS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MILLIS(null) +-- !query schema +struct +-- !query output +1606833008000 1606833008999 NULL + + +-- !query +select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MICROS(null) +-- !query schema +struct +-- !query output +1606833008000000 1606833008999999 NULL + + +-- !query +select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 + + +-- !query +select to_timestamp(1) +-- !query schema +struct +-- !query output +1969-12-31 16:00:01 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '2019-10-06 10:11:12.' could not be parsed at index 20" + } +} + + +-- !query +select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 03:11:12.123 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 08:11:12.12345 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.123456 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '2019-10-06 10:11:12.1234567PST' could not be parsed, unparsed text found at index 26" + } +} + + +-- !query +select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.123456 + + +-- !query +select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '223456 2019-10-06 10:11:12.123456PST' could not be parsed at index 27" + } +} + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.123 + + +-- !query +select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12 + + +-- !query +select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:00 + + +-- !query +select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12345 + + +-- !query +select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 7" + } +} + + +-- !query +select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 9" + } +} + + +-- !query +select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +0019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'") +-- !query schema +struct +-- !query output +2019-10-06 00:00:00 + + +-- !query +select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd") +-- !query schema +struct +-- !query output +2019-10-06 00:00:00 + + +-- !query +select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12 + + +-- !query +select to_timestamp("2019-10-06T10:11:12'", "yyyy-MM-dd'T'HH:mm:ss''") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("16", "dd") +-- !query schema +struct +-- !query output +1970-01-16 00:00:00 + + +-- !query +select to_timestamp("02-29", "MM-dd") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Invalid date 'February 29' as '1970' is not a leap year" + } +} + + +-- !query +select to_timestamp("2019 40", "yyyy mm") +-- !query schema +struct +-- !query output +2019-01-01 00:40:00 + + +-- !query +select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss") +-- !query schema +struct +-- !query output +2019-01-01 10:10:10 + + +-- !query +select timestamp'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query schema +struct<(TIMESTAMP '2011-11-11 11:11:11' - TIMESTAMP '2011-11-11 11:11:10'):interval day to second> +-- !query output +0 00:00:01.000000000 + + +-- !query +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' +-- !query schema +struct<(DATE '2020-01-01' - TIMESTAMP '2019-10-06 10:11:12.345678'):interval day to second> +-- !query output +86 13:48:47.654322000 + + +-- !query +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' +-- !query schema +struct<(TIMESTAMP '2019-10-06 10:11:12.345678' - DATE '2020-01-01'):interval day to second> +-- !query output +-86 13:48:47.654322000 + + +-- !query +select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' +-- !query schema +struct<(TIMESTAMP '2011-11-11 11:11:11' - 2011-11-11 11:11:10):interval day to second> +-- !query output +0 00:00:01.000000000 + + +-- !query +select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query schema +struct<(2011-11-11 11:11:11 - TIMESTAMP '2011-11-11 11:11:10'):interval day to second> +-- !query output +0 00:00:01.000000000 + + +-- !query +select timestamp'2011-11-11 11:11:11' - null +-- !query schema +struct<(TIMESTAMP '2011-11-11 11:11:11' - NULL):interval day to second> +-- !query output +NULL + + +-- !query +select null - timestamp'2011-11-11 11:11:11' +-- !query schema +struct<(NULL - TIMESTAMP '2011-11-11 11:11:11'):interval day to second> +-- !query output +NULL + + +-- !query +create temporary view ts_view as select '2011-11-11 11:11:11' str +-- !query schema +struct<> +-- !query output + + + +-- !query +select str - timestamp'2011-11-11 11:11:11' from ts_view +-- !query schema +struct<(str - TIMESTAMP '2011-11-11 11:11:11'):interval day to second> +-- !query output +0 00:00:00.000000000 + + +-- !query +select timestamp'2011-11-11 11:11:11' - str from ts_view +-- !query schema +struct<(TIMESTAMP '2011-11-11 11:11:11' - str):interval day to second> +-- !query output +0 00:00:00.000000000 + + +-- !query +select timestamp'2011-11-11 11:11:11' + '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "timestamp'2011-11-11 11:11:11' + '1'" + } ] +} + + +-- !query +select '1' + timestamp'2011-11-11 11:11:11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "'1' + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + null +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"VOID\"", + "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "timestamp'2011-11-11 11:11:11' + null" + } ] +} + + +-- !query +select null + timestamp'2011-11-11 11:11:11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "null + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + interval '2' day, + timestamp'2011-11-11 11:11:11' - interval '2-2' year to month, + timestamp'2011-11-11 11:11:11' + interval '-2' second, + timestamp'2011-11-11 11:11:11' - interval '12:12:12.123456789' hour to second, + - interval 2 years + timestamp'2011-11-11 11:11:11', + interval '1 12' day to hour + timestamp'2011-11-11 11:11:11' +-- !query schema +struct +-- !query output +2011-11-13 11:11:11 2009-09-11 11:11:11 2011-11-11 11:11:09 2011-11-10 22:58:58.876544 2009-11-11 11:11:11 2011-11-12 23:11:11 + + +-- !query +select date '2012-01-01' - interval 3 hours, + date '2012-01-01' + interval '12:12:12' hour to second, + interval '2' minute + date '2012-01-01' +-- !query schema +struct +-- !query output +2011-12-31 21:00:00 2012-01-01 12:12:12 2012-01-01 00:02:00 + + +-- !query +select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'yyyy-MM-dd GGGGG'" + } +} + + +-- !query +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd MM yyyy EEEEEE'" + } +} + + +-- !query +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd MM yyyy EEEEE'" + } +} + + +-- !query +select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd MM yyyy EEEEE'" + } +} + + +-- !query +select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd/MMMMM/yyyy'" + } +} + + +-- !query +select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd/MMMMM/yyyy'" + } +} + + +-- !query +select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03') +-- !query schema +struct +-- !query output +2022-01-14 01:02:03 + + +-- !query +select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03') +-- !query schema +struct +-- !query output +2022-02-14 02:00:03 + + +-- !query +select timestampadd(YEAR, 1, date'2022-02-15') +-- !query schema +struct +-- !query output +2023-02-15 00:00:00 + + +-- !query +select timestampadd(SECOND, -1, date'2022-02-15') +-- !query schema +struct +-- !query output +2022-02-14 23:59:59 + + +-- !query +select timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampadd`", + "invalidValue" : "'MONTH'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03')" + } ] +} + + +-- !query +select timestampadd('SECOND', -1, date'2022-02-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampadd`", + "invalidValue" : "'SECOND'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "timestampadd('SECOND', -1, date'2022-02-15')" + } ] +} + + +-- !query +select timestampdiff(MONTH, timestamp'2022-02-14 01:02:03', timestamp'2022-01-14 01:02:03') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select timestampdiff(MINUTE, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03') +-- !query schema +struct +-- !query output +58 + + +-- !query +select timestampdiff(YEAR, date'2022-02-15', date'2023-02-15') +-- !query schema +struct +-- !query output +1 + + +-- !query +select timestampdiff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampdiff`", + "invalidValue" : "'MINUTE'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 94, + "fragment" : "timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03')" + } ] +} + + +-- !query +select timestampdiff('YEAR', date'2022-02-15', date'2023-02-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampdiff`", + "invalidValue" : "'YEAR'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "timestampdiff('YEAR', date'2022-02-15', date'2023-02-15')" + } ] +} + + +-- !query +select timediff(QUARTER, timestamp'2023-08-10 01:02:03', timestamp'2022-01-14 01:02:03') +-- !query schema +struct +-- !query output +-6 + + +-- !query +select timediff(HOUR, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 12:00:03') +-- !query schema +struct +-- !query output +10 + + +-- !query +select timediff(DAY, date'2022-02-15', date'2023-02-15') +-- !query schema +struct +-- !query output +365 + + +-- !query +select timediff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timediff`", + "invalidValue" : "'MINUTE'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 89, + "fragment" : "timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03')" + } ] +} + + +-- !query +select timediff('YEAR', date'2020-02-15', date'2023-02-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timediff`", + "invalidValue" : "'YEAR'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "timediff('YEAR', date'2020-02-15', date'2023-02-15')" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestampNTZ/datetime-special-ansi.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestampNTZ/datetime-special-ansi.sql.out new file mode 100644 index 000000000000..c8cdec4eeb98 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestampNTZ/datetime-special-ansi.sql.out @@ -0,0 +1,31 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select date'999999-03-18', date'-0001-1-28', date'0015' +-- !query schema +struct +-- !query output ++999999-03-18 -0001-01-28 0015-01-01 + + +-- !query +select make_date(999999, 3, 18), make_date(-1, 1, 28) +-- !query schema +struct +-- !query output ++999999-03-18 -0001-01-28 + + +-- !query +select timestamp'-1969-12-31 16:00:00', timestamp'-0015-03-18 16:00:00', timestamp'-000001', timestamp'99999-03-18T12:03:17' +-- !query schema +struct +-- !query output +-1969-12-31 16:00:00 -0015-03-18 16:00:00 -0001-01-01 00:00:00 +99999-03-18 12:03:17 + + +-- !query +select make_timestamp(-1969, 12, 31, 16, 0, 0.0), make_timestamp(-15, 3, 18, 16, 0, 0.0), make_timestamp(99999, 3, 18, 12, 3, 17.0) +-- !query schema +struct +-- !query output +-1969-12-31 16:00:00 -0015-03-18 16:00:00 +99999-03-18 12:03:17 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestampNTZ/datetime-special.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestampNTZ/datetime-special.sql.out new file mode 100644 index 000000000000..c8cdec4eeb98 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestampNTZ/datetime-special.sql.out @@ -0,0 +1,31 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select date'999999-03-18', date'-0001-1-28', date'0015' +-- !query schema +struct +-- !query output ++999999-03-18 -0001-01-28 0015-01-01 + + +-- !query +select make_date(999999, 3, 18), make_date(-1, 1, 28) +-- !query schema +struct +-- !query output ++999999-03-18 -0001-01-28 + + +-- !query +select timestamp'-1969-12-31 16:00:00', timestamp'-0015-03-18 16:00:00', timestamp'-000001', timestamp'99999-03-18T12:03:17' +-- !query schema +struct +-- !query output +-1969-12-31 16:00:00 -0015-03-18 16:00:00 -0001-01-01 00:00:00 +99999-03-18 12:03:17 + + +-- !query +select make_timestamp(-1969, 12, 31, 16, 0, 0.0), make_timestamp(-15, 3, 18, 16, 0, 0.0), make_timestamp(99999, 3, 18, 12, 3, 17.0) +-- !query schema +struct +-- !query output +-1969-12-31 16:00:00 -0015-03-18 16:00:00 +99999-03-18 12:03:17 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out new file mode 100644 index 000000000000..fcb15661601a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestampNTZ/timestamp-ansi.sql.out @@ -0,0 +1,1215 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select timestamp '2019-01-01\t' +-- !query schema +struct +-- !query output +2019-01-01 00:00:00 + + +-- !query +select timestamp '2019-01-01中文' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2019-01-01中文'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "timestamp '2019-01-01中文'" + } ] +} + + +-- !query +select timestamp'4294967297' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'4294967297'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "timestamp'4294967297'" + } ] +} + + +-- !query +select timestamp'2021-01-01T12:30:4294967297.123456' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2021-01-01T12:30:4294967297.123456'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'" + } ] +} + + +-- !query +select current_timestamp = current_timestamp +-- !query schema +struct<(current_timestamp() = current_timestamp()):boolean> +-- !query output +true + + +-- !query +select current_timestamp() = current_timestamp() +-- !query schema +struct<(current_timestamp() = current_timestamp()):boolean> +-- !query output +true + + +-- !query +select localtimestamp() = localtimestamp() +-- !query schema +struct<(localtimestamp() = localtimestamp()):boolean> +-- !query output +true + + +-- !query +SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678) +-- !query schema +struct +-- !query output +2021-07-11 06:30:45.678 + + +-- !query +SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678, 'CET') +-- !query schema +struct +-- !query output +2021-07-11 06:30:45.678 + + +-- !query +SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "INVALID_FRACTION_OF_SECOND", + "sqlState" : "22023", + "messageParameters" : { + "secAndMicros" : "60.007" + } +} + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 1) +-- !query schema +struct +-- !query output +0001-01-01 01:01:01 + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 60) +-- !query schema +struct +-- !query output +0001-01-01 01:02:00 + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 61) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for SecondOfMinute (valid values 0 - 59): 61" + } +} + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 59.999999) +-- !query schema +struct +-- !query output +0001-01-01 01:01:59.999999 + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for SecondOfMinute (valid values 0 - 59): 99" + } +} + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "DATETIME_FIELD_OUT_OF_BOUNDS", + "sqlState" : "22023", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "rangeMessage" : "Invalid value for SecondOfMinute (valid values 0 - 59): 999" + } +} + + +-- !query +select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00 1931-01-07 00:30:00 NULL + + +-- !query +select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23)) +-- !query schema +struct +-- !query output +1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 + + +-- !query +select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00.123 1931-01-07 00:29:59.877 NULL + + +-- !query +select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00.123123 1931-01-07 00:29:59.876877 NULL + + +-- !query +select TIMESTAMP_SECONDS(1230219000123123) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_SECONDS(-1230219000123123) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_MILLIS(92233720368547758) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_MILLIS(-92233720368547758) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_SECONDS(0.1234567) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Rounding necessary + + +-- !query +select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567)) +-- !query schema +struct +-- !query output +1969-12-31 16:00:00.123456 1969-12-31 16:00:00.123456 + + +-- !query +create temporary view ttf1 as select * from values + (1, 2), + (2, 3) + as ttf1(`current_date`, `current_timestamp`) +-- !query schema +struct<> +-- !query output + + + +-- !query +select typeof(current_date), typeof(current_timestamp) from ttf1 +-- !query schema +struct +-- !query output +int int +int int + + +-- !query +create temporary view ttf2 as select * from values + (1, 2), + (2, 3) + as ttf2(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2 +-- !query schema +struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean,a:int,b:int> +-- !query output +true true 1 2 +true true 2 3 + + +-- !query +select a, b from ttf2 order by a, current_date +-- !query schema +struct +-- !query output +1 2 +2 3 + + +-- !query +select UNIX_SECONDS(timestamp'2020-12-01 14:30:08Z'), UNIX_SECONDS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_SECONDS(null) +-- !query schema +struct +-- !query output +1606833008 1606833008 NULL + + +-- !query +select UNIX_MILLIS(timestamp'2020-12-01 14:30:08Z'), UNIX_MILLIS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MILLIS(null) +-- !query schema +struct +-- !query output +1606833008000 1606833008999 NULL + + +-- !query +select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MICROS(null) +-- !query schema +struct +-- !query output +1606833008000000 1606833008999999 NULL + + +-- !query +select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 + + +-- !query +select to_timestamp(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP_NTZ\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "to_timestamp(1)" + } ] +} + + +-- !query +select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '2019-10-06 10:11:12.' could not be parsed at index 20" + } +} + + +-- !query +select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.123 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12345 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.123456 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '2019-10-06 10:11:12.1234567PST' could not be parsed, unparsed text found at index 26" + } +} + + +-- !query +select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.123456 + + +-- !query +select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '223456 2019-10-06 10:11:12.123456PST' could not be parsed at index 27" + } +} + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.123 + + +-- !query +select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12 + + +-- !query +select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:00 + + +-- !query +select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12345 + + +-- !query +select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 7" + } +} + + +-- !query +select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 9" + } +} + + +-- !query +select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +0019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'") +-- !query schema +struct +-- !query output +2019-10-06 00:00:00 + + +-- !query +select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd") +-- !query schema +struct +-- !query output +2019-10-06 00:00:00 + + +-- !query +select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12 + + +-- !query +select to_timestamp("2019-10-06T10:11:12'", "yyyy-MM-dd'T'HH:mm:ss''") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("16", "dd") +-- !query schema +struct +-- !query output +1970-01-16 00:00:00 + + +-- !query +select to_timestamp("02-29", "MM-dd") +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Invalid date 'February 29' as '1970' is not a leap year" + } +} + + +-- !query +select to_timestamp("2019 40", "yyyy mm") +-- !query schema +struct +-- !query output +2019-01-01 00:40:00 + + +-- !query +select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss") +-- !query schema +struct +-- !query output +2019-01-01 10:10:10 + + +-- !query +select timestamp'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query schema +struct<(TIMESTAMP_NTZ '2011-11-11 11:11:11' - TIMESTAMP_NTZ '2011-11-11 11:11:10'):interval day to second> +-- !query output +0 00:00:01.000000000 + + +-- !query +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' +-- !query schema +struct<(DATE '2020-01-01' - TIMESTAMP_NTZ '2019-10-06 10:11:12.345678'):interval day to second> +-- !query output +86 13:48:47.654322000 + + +-- !query +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' +-- !query schema +struct<(TIMESTAMP_NTZ '2019-10-06 10:11:12.345678' - DATE '2020-01-01'):interval day to second> +-- !query output +-86 13:48:47.654322000 + + +-- !query +select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' +-- !query schema +struct<(TIMESTAMP_NTZ '2011-11-11 11:11:11' - 2011-11-11 11:11:10):interval day to second> +-- !query output +0 00:00:01.000000000 + + +-- !query +select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query schema +struct<(2011-11-11 11:11:11 - TIMESTAMP_NTZ '2011-11-11 11:11:10'):interval day to second> +-- !query output +0 00:00:01.000000000 + + +-- !query +select timestamp'2011-11-11 11:11:11' - null +-- !query schema +struct<(TIMESTAMP_NTZ '2011-11-11 11:11:11' - NULL):interval day to second> +-- !query output +NULL + + +-- !query +select null - timestamp'2011-11-11 11:11:11' +-- !query schema +struct<(NULL - TIMESTAMP_NTZ '2011-11-11 11:11:11'):interval day to second> +-- !query output +NULL + + +-- !query +create temporary view ts_view as select '2011-11-11 11:11:11' str +-- !query schema +struct<> +-- !query output + + + +-- !query +select str - timestamp'2011-11-11 11:11:11' from ts_view +-- !query schema +struct<(str - TIMESTAMP_NTZ '2011-11-11 11:11:11'):interval day to second> +-- !query output +0 00:00:00.000000000 + + +-- !query +select timestamp'2011-11-11 11:11:11' - str from ts_view +-- !query schema +struct<(TIMESTAMP_NTZ '2011-11-11 11:11:11' - str):interval day to second> +-- !query output +0 00:00:00.000000000 + + +-- !query +select timestamp'2011-11-11 11:11:11' + '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP_NTZ\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "timestamp'2011-11-11 11:11:11' + '1'" + } ] +} + + +-- !query +select '1' + timestamp'2011-11-11 11:11:11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP_NTZ\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(1 + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "'1' + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + null +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP_NTZ\"", + "right" : "\"VOID\"", + "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "timestamp'2011-11-11 11:11:11' + null" + } ] +} + + +-- !query +select null + timestamp'2011-11-11 11:11:11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"TIMESTAMP_NTZ\"", + "sqlExpr" : "\"(NULL + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "null + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + interval '2' day, + timestamp'2011-11-11 11:11:11' - interval '2-2' year to month, + timestamp'2011-11-11 11:11:11' + interval '-2' second, + timestamp'2011-11-11 11:11:11' - interval '12:12:12.123456789' hour to second, + - interval 2 years + timestamp'2011-11-11 11:11:11', + interval '1 12' day to hour + timestamp'2011-11-11 11:11:11' +-- !query schema +struct +-- !query output +2011-11-13 11:11:11 2009-09-11 11:11:11 2011-11-11 11:11:09 2011-11-10 22:58:58.876544 2009-11-11 11:11:11 2011-11-12 23:11:11 + + +-- !query +select date '2012-01-01' - interval 3 hours, + date '2012-01-01' + interval '12:12:12' hour to second, + interval '2' minute + date '2012-01-01' +-- !query schema +struct +-- !query output +2011-12-31 21:00:00 2012-01-01 12:12:12 2012-01-01 00:02:00 + + +-- !query +select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2130", + "messageParameters" : { + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'yyyy-MM-dd GGGGG'" + } +} + + +-- !query +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2130", + "messageParameters" : { + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd MM yyyy EEEEEE'" + } +} + + +-- !query +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2130", + "messageParameters" : { + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd MM yyyy EEEEE'" + } +} + + +-- !query +select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd MM yyyy EEEEE'" + } +} + + +-- !query +select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct> +-- !query output +{"t":null} + + +-- !query +select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct> +-- !query output +{"t":null} + + +-- !query +select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03') +-- !query schema +struct +-- !query output +2022-01-14 01:02:03 + + +-- !query +select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03') +-- !query schema +struct +-- !query output +2022-02-14 02:00:03 + + +-- !query +select timestampadd(YEAR, 1, date'2022-02-15') +-- !query schema +struct +-- !query output +2023-02-15 00:00:00 + + +-- !query +select timestampadd(SECOND, -1, date'2022-02-15') +-- !query schema +struct +-- !query output +2022-02-14 23:59:59 + + +-- !query +select timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampadd`", + "invalidValue" : "'MONTH'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03')" + } ] +} + + +-- !query +select timestampadd('SECOND', -1, date'2022-02-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampadd`", + "invalidValue" : "'SECOND'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "timestampadd('SECOND', -1, date'2022-02-15')" + } ] +} + + +-- !query +select timestampdiff(MONTH, timestamp'2022-02-14 01:02:03', timestamp'2022-01-14 01:02:03') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select timestampdiff(MINUTE, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03') +-- !query schema +struct +-- !query output +58 + + +-- !query +select timestampdiff(YEAR, date'2022-02-15', date'2023-02-15') +-- !query schema +struct +-- !query output +1 + + +-- !query +select timestampdiff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampdiff`", + "invalidValue" : "'MINUTE'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 94, + "fragment" : "timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03')" + } ] +} + + +-- !query +select timestampdiff('YEAR', date'2022-02-15', date'2023-02-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampdiff`", + "invalidValue" : "'YEAR'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "timestampdiff('YEAR', date'2022-02-15', date'2023-02-15')" + } ] +} + + +-- !query +select timediff(QUARTER, timestamp'2023-08-10 01:02:03', timestamp'2022-01-14 01:02:03') +-- !query schema +struct +-- !query output +-6 + + +-- !query +select timediff(HOUR, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 12:00:03') +-- !query schema +struct +-- !query output +10 + + +-- !query +select timediff(DAY, date'2022-02-15', date'2023-02-15') +-- !query schema +struct +-- !query output +365 + + +-- !query +select timediff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timediff`", + "invalidValue" : "'MINUTE'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 89, + "fragment" : "timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03')" + } ] +} + + +-- !query +select timediff('YEAR', date'2020-02-15', date'2023-02-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timediff`", + "invalidValue" : "'YEAR'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "timediff('YEAR', date'2020-02-15', date'2023-02-15')" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestampNTZ/timestamp.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestampNTZ/timestamp.sql.out new file mode 100644 index 000000000000..102935c90f9c --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timestampNTZ/timestamp.sql.out @@ -0,0 +1,1191 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select timestamp '2019-01-01\t' +-- !query schema +struct +-- !query output +2019-01-01 00:00:00 + + +-- !query +select timestamp '2019-01-01中文' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2019-01-01中文'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "timestamp '2019-01-01中文'" + } ] +} + + +-- !query +select timestamp'4294967297' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'4294967297'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "timestamp'4294967297'" + } ] +} + + +-- !query +select timestamp'2021-01-01T12:30:4294967297.123456' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_TYPED_LITERAL", + "sqlState" : "42604", + "messageParameters" : { + "value" : "'2021-01-01T12:30:4294967297.123456'", + "valueType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'" + } ] +} + + +-- !query +select current_timestamp = current_timestamp +-- !query schema +struct<(current_timestamp() = current_timestamp()):boolean> +-- !query output +true + + +-- !query +select current_timestamp() = current_timestamp() +-- !query schema +struct<(current_timestamp() = current_timestamp()):boolean> +-- !query output +true + + +-- !query +select localtimestamp() = localtimestamp() +-- !query schema +struct<(localtimestamp() = localtimestamp()):boolean> +-- !query output +true + + +-- !query +SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678) +-- !query schema +struct +-- !query output +2021-07-11 06:30:45.678 + + +-- !query +SELECT make_timestamp(2021, 07, 11, 6, 30, 45.678, 'CET') +-- !query schema +struct +-- !query output +2021-07-11 06:30:45.678 + + +-- !query +SELECT make_timestamp(2021, 07, 11, 6, 30, 60.007) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 1) +-- !query schema +struct +-- !query output +0001-01-01 01:01:01 + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 60) +-- !query schema +struct +-- !query output +0001-01-01 01:02:00 + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 61) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 59.999999) +-- !query schema +struct +-- !query output +0001-01-01 01:01:59.999999 + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 99.999999) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT make_timestamp(1, 1, 1, 1, 1, 999.999999) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select TIMESTAMP_SECONDS(1230219000),TIMESTAMP_SECONDS(-1230219000),TIMESTAMP_SECONDS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00 1931-01-07 00:30:00 NULL + + +-- !query +select TIMESTAMP_SECONDS(1.23), TIMESTAMP_SECONDS(1.23d), TIMESTAMP_SECONDS(FLOAT(1.23)) +-- !query schema +struct +-- !query output +1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 1969-12-31 16:00:01.23 + + +-- !query +select TIMESTAMP_MILLIS(1230219000123),TIMESTAMP_MILLIS(-1230219000123),TIMESTAMP_MILLIS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00.123 1931-01-07 00:29:59.877 NULL + + +-- !query +select TIMESTAMP_MICROS(1230219000123123),TIMESTAMP_MICROS(-1230219000123123),TIMESTAMP_MICROS(null) +-- !query schema +struct +-- !query output +2008-12-25 07:30:00.123123 1931-01-07 00:29:59.876877 NULL + + +-- !query +select TIMESTAMP_SECONDS(1230219000123123) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_SECONDS(-1230219000123123) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_MILLIS(92233720368547758) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_MILLIS(-92233720368547758) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +long overflow + + +-- !query +select TIMESTAMP_SECONDS(0.1234567) +-- !query schema +struct<> +-- !query output +java.lang.ArithmeticException +Rounding necessary + + +-- !query +select TIMESTAMP_SECONDS(0.1234567d), TIMESTAMP_SECONDS(FLOAT(0.1234567)) +-- !query schema +struct +-- !query output +1969-12-31 16:00:00.123456 1969-12-31 16:00:00.123456 + + +-- !query +create temporary view ttf1 as select * from values + (1, 2), + (2, 3) + as ttf1(`current_date`, `current_timestamp`) +-- !query schema +struct<> +-- !query output + + + +-- !query +select typeof(current_date), typeof(current_timestamp) from ttf1 +-- !query schema +struct +-- !query output +int int +int int + + +-- !query +create temporary view ttf2 as select * from values + (1, 2), + (2, 3) + as ttf2(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2 +-- !query schema +struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean,a:int,b:int> +-- !query output +true true 1 2 +true true 2 3 + + +-- !query +select a, b from ttf2 order by a, current_date +-- !query schema +struct +-- !query output +1 2 +2 3 + + +-- !query +select UNIX_SECONDS(timestamp'2020-12-01 14:30:08Z'), UNIX_SECONDS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_SECONDS(null) +-- !query schema +struct +-- !query output +1606833008 1606833008 NULL + + +-- !query +select UNIX_MILLIS(timestamp'2020-12-01 14:30:08Z'), UNIX_MILLIS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MILLIS(null) +-- !query schema +struct +-- !query output +1606833008000 1606833008999 NULL + + +-- !query +select UNIX_MICROS(timestamp'2020-12-01 14:30:08Z'), UNIX_MICROS(timestamp'2020-12-01 14:30:08.999999Z'), UNIX_MICROS(null) +-- !query schema +struct +-- !query output +1606833008000000 1606833008999999 NULL + + +-- !query +select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 + + +-- !query +select to_timestamp(1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.0', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123UTC', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.123 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12345CST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12345 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123456PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.123456 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234567PST', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('123456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.123456 + + +-- !query +select to_timestamp('223456 2019-10-06 10:11:12.123456PST', 'SSSSSS yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp('2019-10-06 10:11:12.1234', 'yyyy-MM-dd HH:mm:ss.[SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.123', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.123 + + +-- !query +select to_timestamp('2019-10-06 10:11:12', 'yyyy-MM-dd HH:mm:ss[.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp('2019-10-06 10:11:12.12', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12 + + +-- !query +select to_timestamp('2019-10-06 10:11', 'yyyy-MM-dd HH:mm[:ss.SSSSSS]') +-- !query schema +struct +-- !query output +2019-10-06 10:11:00 + + +-- !query +select to_timestamp("2019-10-06S10:11:12.12345", "yyyy-MM-dd'S'HH:mm:ss.SSSSSS") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12345 + + +-- !query +select to_timestamp("12.12342019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyyyy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("12.1232019-10-06S10:11", "ss.SSSSyy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("12.1234019-10-06S10:11", "ss.SSSSy-MM-dd'S'HH:mm") +-- !query schema +struct +-- !query output +0019-10-06 10:11:12.1234 + + +-- !query +select to_timestamp("2019-10-06S", "yyyy-MM-dd'S'") +-- !query schema +struct +-- !query output +2019-10-06 00:00:00 + + +-- !query +select to_timestamp("S2019-10-06", "'S'yyyy-MM-dd") +-- !query schema +struct +-- !query output +2019-10-06 00:00:00 + + +-- !query +select to_timestamp("2019-10-06T10:11:12'12", "yyyy-MM-dd'T'HH:mm:ss''SSSS") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12.12 + + +-- !query +select to_timestamp("2019-10-06T10:11:12'", "yyyy-MM-dd'T'HH:mm:ss''") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("'2019-10-06T10:11:12", "''yyyy-MM-dd'T'HH:mm:ss") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("P2019-10-06T10:11:12", "'P'yyyy-MM-dd'T'HH:mm:ss") +-- !query schema +struct +-- !query output +2019-10-06 10:11:12 + + +-- !query +select to_timestamp("16", "dd") +-- !query schema +struct +-- !query output +1970-01-16 00:00:00 + + +-- !query +select to_timestamp("02-29", "MM-dd") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select to_timestamp("2019 40", "yyyy mm") +-- !query schema +struct +-- !query output +2019-01-01 00:40:00 + + +-- !query +select to_timestamp("2019 10:10:10", "yyyy hh:mm:ss") +-- !query schema +struct +-- !query output +2019-01-01 10:10:10 + + +-- !query +select timestamp'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query schema +struct<(TIMESTAMP_NTZ '2011-11-11 11:11:11' - TIMESTAMP_NTZ '2011-11-11 11:11:10'):interval day to second> +-- !query output +0 00:00:01.000000000 + + +-- !query +select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678' +-- !query schema +struct<(DATE '2020-01-01' - TIMESTAMP_NTZ '2019-10-06 10:11:12.345678'):interval day to second> +-- !query output +86 13:48:47.654322000 + + +-- !query +select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01' +-- !query schema +struct<(TIMESTAMP_NTZ '2019-10-06 10:11:12.345678' - DATE '2020-01-01'):interval day to second> +-- !query output +-86 13:48:47.654322000 + + +-- !query +select timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2011-11-11 11:11:10\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' - 2011-11-11 11:11:10)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'" + } ] +} + + +-- !query +select '2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2011-11-11 11:11:11\"", + "inputType" : "\"STRING\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(2011-11-11 11:11:11 - TIMESTAMP_NTZ '2011-11-11 11:11:10')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' - null +-- !query schema +struct<(TIMESTAMP_NTZ '2011-11-11 11:11:11' - NULL):interval day to second> +-- !query output +NULL + + +-- !query +select null - timestamp'2011-11-11 11:11:11' +-- !query schema +struct<(NULL - TIMESTAMP_NTZ '2011-11-11 11:11:11'):interval day to second> +-- !query output +NULL + + +-- !query +create temporary view ts_view as select '2011-11-11 11:11:11' str +-- !query schema +struct<> +-- !query output + + + +-- !query +select str - timestamp'2011-11-11 11:11:11' from ts_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(str - TIMESTAMP_NTZ '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "str - timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' - str from ts_view +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"str\"", + "inputType" : "\"STRING\"", + "paramIndex" : "second", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' - str)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "timestamp'2011-11-11 11:11:11' - str" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + '1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP_NTZ\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "timestamp'2011-11-11 11:11:11' + '1'" + } ] +} + + +-- !query +select '1' + timestamp'2011-11-11 11:11:11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"TIMESTAMP_NTZ\"", + "sqlExpr" : "\"(1 + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "'1' + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + null +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP_NTZ\"", + "right" : "\"VOID\"", + "sqlExpr" : "\"(TIMESTAMP_NTZ '2011-11-11 11:11:11' + NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "timestamp'2011-11-11 11:11:11' + null" + } ] +} + + +-- !query +select null + timestamp'2011-11-11 11:11:11' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"VOID\"", + "right" : "\"TIMESTAMP_NTZ\"", + "sqlExpr" : "\"(NULL + TIMESTAMP_NTZ '2011-11-11 11:11:11')\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "null + timestamp'2011-11-11 11:11:11'" + } ] +} + + +-- !query +select timestamp'2011-11-11 11:11:11' + interval '2' day, + timestamp'2011-11-11 11:11:11' - interval '2-2' year to month, + timestamp'2011-11-11 11:11:11' + interval '-2' second, + timestamp'2011-11-11 11:11:11' - interval '12:12:12.123456789' hour to second, + - interval 2 years + timestamp'2011-11-11 11:11:11', + interval '1 12' day to hour + timestamp'2011-11-11 11:11:11' +-- !query schema +struct +-- !query output +2011-11-13 11:11:11 2009-09-11 11:11:11 2011-11-11 11:11:09 2011-11-10 22:58:58.876544 2009-11-11 11:11:11 2011-11-12 23:11:11 + + +-- !query +select date '2012-01-01' - interval 3 hours, + date '2012-01-01' + interval '12:12:12' hour to second, + interval '2' minute + date '2012-01-01' +-- !query schema +struct +-- !query output +2011-12-31 21:00:00 2012-01-01 12:12:12 2012-01-01 00:02:00 + + +-- !query +select to_timestamp('2019-10-06 A', 'yyyy-MM-dd GGGGG') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2130", + "messageParameters" : { + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'yyyy-MM-dd GGGGG'" + } +} + + +-- !query +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2130", + "messageParameters" : { + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd MM yyyy EEEEEE'" + } +} + + +-- !query +select to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_2130", + "messageParameters" : { + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd MM yyyy EEEEE'" + } +} + + +-- !query +select unix_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd MM yyyy EEEEE'" + } +} + + +-- !query +select from_json('{"t":"26/October/2015"}', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct> +-- !query output +{"t":null} + + +-- !query +select from_csv('26/October/2015', 't Timestamp', map('timestampFormat', 'dd/MMMMM/yyyy')) +-- !query schema +struct> +-- !query output +{"t":null} + + +-- !query +select timestampadd(MONTH, -1, timestamp'2022-02-14 01:02:03') +-- !query schema +struct +-- !query output +2022-01-14 01:02:03 + + +-- !query +select timestampadd(MINUTE, 58, timestamp'2022-02-14 01:02:03') +-- !query schema +struct +-- !query output +2022-02-14 02:00:03 + + +-- !query +select timestampadd(YEAR, 1, date'2022-02-15') +-- !query schema +struct +-- !query output +2023-02-15 00:00:00 + + +-- !query +select timestampadd(SECOND, -1, date'2022-02-15') +-- !query schema +struct +-- !query output +2022-02-14 23:59:59 + + +-- !query +select timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampadd`", + "invalidValue" : "'MONTH'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "timestampadd('MONTH', -1, timestamp'2022-02-14 01:02:03')" + } ] +} + + +-- !query +select timestampadd('SECOND', -1, date'2022-02-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampadd`", + "invalidValue" : "'SECOND'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "timestampadd('SECOND', -1, date'2022-02-15')" + } ] +} + + +-- !query +select timestampdiff(MONTH, timestamp'2022-02-14 01:02:03', timestamp'2022-01-14 01:02:03') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select timestampdiff(MINUTE, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03') +-- !query schema +struct +-- !query output +58 + + +-- !query +select timestampdiff(YEAR, date'2022-02-15', date'2023-02-15') +-- !query schema +struct +-- !query output +1 + + +-- !query +select timestampdiff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampdiff`", + "invalidValue" : "'MINUTE'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 94, + "fragment" : "timestampdiff('MINUTE', timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 02:00:03')" + } ] +} + + +-- !query +select timestampdiff('YEAR', date'2022-02-15', date'2023-02-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timestampdiff`", + "invalidValue" : "'YEAR'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "timestampdiff('YEAR', date'2022-02-15', date'2023-02-15')" + } ] +} + + +-- !query +select timediff(QUARTER, timestamp'2023-08-10 01:02:03', timestamp'2022-01-14 01:02:03') +-- !query schema +struct +-- !query output +-6 + + +-- !query +select timediff(HOUR, timestamp'2022-02-14 01:02:03', timestamp'2022-02-14 12:00:03') +-- !query schema +struct +-- !query output +10 + + +-- !query +select timediff(DAY, date'2022-02-15', date'2023-02-15') +-- !query schema +struct +-- !query output +365 + + +-- !query +select timediff(SECOND, date'2022-02-15', timestamp'2022-02-14 23:59:59') +-- !query schema +struct +-- !query output +-1 + + +-- !query +select timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timediff`", + "invalidValue" : "'MINUTE'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 89, + "fragment" : "timediff('MINUTE', timestamp'2023-02-14 01:02:03', timestamp'2023-02-14 02:00:03')" + } ] +} + + +-- !query +select timediff('YEAR', date'2020-02-15', date'2023-02-15') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_PARAMETER_VALUE.DATETIME_UNIT", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "`timediff`", + "invalidValue" : "'YEAR'", + "parameter" : "`unit`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "timediff('YEAR', date'2020-02-15', date'2023-02-15')" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timezone.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timezone.sql.out new file mode 100644 index 000000000000..5f0fdef50e3d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/timezone.sql.out @@ -0,0 +1,179 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SET TIME ZONE 'Asia/Hong_Kong' +-- !query schema +struct +-- !query output +spark.sql.session.timeZone Asia/Hong_Kong + + +-- !query +SET TIME ZONE 'GMT+1' +-- !query schema +struct +-- !query output +spark.sql.session.timeZone GMT+1 + + +-- !query +SET TIME ZONE INTERVAL 10 HOURS +-- !query schema +struct +-- !query output +spark.sql.session.timeZone +10:00 + + +-- !query +SET TIME ZONE INTERVAL '15:40:32' HOUR TO SECOND +-- !query schema +struct +-- !query output +spark.sql.session.timeZone +15:40:32 + + +-- !query +SET TIME ZONE LOCAL +-- !query schema +struct +-- !query output +spark.sql.session.timeZone America/Los_Angeles + + +-- !query +SET TIME ZONE +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0045", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 13, + "fragment" : "SET TIME ZONE" + } ] +} + + +-- !query +SET TIME ZONE 'invalid/zone' +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "INVALID_CONF_VALUE.TIME_ZONE", + "sqlState" : "22022", + "messageParameters" : { + "confName" : "spark.sql.session.timeZone", + "confValue" : "invalid/zone" + } +} + + +-- !query +SET TIME ZONE INTERVAL 3 DAYS +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 29, + "fragment" : "SET TIME ZONE INTERVAL 3 DAYS" + } ] +} + + +-- !query +SET TIME ZONE INTERVAL 24 HOURS +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "24" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 31, + "fragment" : "SET TIME ZONE INTERVAL 24 HOURS" + } ] +} + + +-- !query +SET TIME ZONE INTERVAL '19:40:32' HOUR TO SECOND +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "19" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 48, + "fragment" : "SET TIME ZONE INTERVAL '19:40:32' HOUR TO SECOND" + } ] +} + + +-- !query +SET TIME ZONE INTERVAL 10 HOURS 'GMT+1' +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0045", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 39, + "fragment" : "SET TIME ZONE INTERVAL 10 HOURS 'GMT+1'" + } ] +} + + +-- !query +SET TIME ZONE INTERVAL 10 HOURS 1 MILLISECOND +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "36000" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 45, + "fragment" : "SET TIME ZONE INTERVAL 10 HOURS 1 MILLISECOND" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/to_from_avro.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/to_from_avro.sql.out new file mode 100644 index 000000000000..a94175b1df39 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/to_from_avro.sql.out @@ -0,0 +1,144 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create table t as + select named_struct('u', named_struct('member0', member0, 'member1', member1)) as s + from values (1, null), (null, 'a') tab(member0, member1) +-- !query schema +struct<> +-- !query output + + + +-- !query +declare avro_schema string +-- !query schema +struct<> +-- !query output + + + +-- !query +set variable avro_schema = + '{ "type": "record", "name": "struct", "fields": [{ "name": "u", "type": ["int","string"] }] }' +-- !query schema +struct<> +-- !query output + + + +-- !query +select from_avro(s, 42, map()) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + "sqlState" : "42K09", + "messageParameters" : { + "hint" : "", + "msg" : "The second argument of the FROM_AVRO SQL function must be a constant string containing the JSON representation of the schema to use for converting the value from AVRO format", + "sqlExpr" : "\"from_avro(s, 42, map())\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "from_avro(s, 42, map())" + } ] +} + + +-- !query +select from_avro(s, avro_schema, 42) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + "sqlState" : "42K09", + "messageParameters" : { + "hint" : "", + "msg" : "The third argument of the FROM_AVRO SQL function must be a constant map of strings to strings containing the options to use for converting the value from AVRO format", + "sqlExpr" : "\"from_avro(s, variablereference(system.session.avro_schema='{ \"type\": \"record\", \"name\": \"struct\", \"fields\": [{ \"name\": \"u\", \"type\": [\"int\",\"string\"] }] }'), 42)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "from_avro(s, avro_schema, 42)" + } ] +} + + +-- !query +select to_avro(s, 42) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + "sqlState" : "42K09", + "messageParameters" : { + "hint" : "", + "msg" : "The second argument of the TO_AVRO SQL function must be a constant string containing the JSON representation of the schema to use for converting the value to AVRO format", + "sqlExpr" : "\"to_avro(s, 42)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "to_avro(s, 42)" + } ] +} + + +-- !query +select to_avro(s, avro_schema) as result from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AVRO_NOT_LOADED_SQL_FUNCTIONS_UNUSABLE", + "sqlState" : "22KD3", + "messageParameters" : { + "functionName" : "TO_AVRO" + } +} + + +-- !query +select from_avro(result, avro_schema, map()).u from (select null as result) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AVRO_NOT_LOADED_SQL_FUNCTIONS_UNUSABLE", + "sqlState" : "22KD3", + "messageParameters" : { + "functionName" : "FROM_AVRO" + } +} + + +-- !query +drop temporary variable avro_schema +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table t +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/transform.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/transform.sql.out new file mode 100644 index 000000000000..3c704dfcc461 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/transform.sql.out @@ -0,0 +1,849 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW t AS SELECT * FROM VALUES +('1', true, unhex('537061726B2053514C'), tinyint(1), 1, smallint(100), bigint(1), float(1.0), 1.0, Decimal(1.0), timestamp('1997-01-02'), date('2000-04-01')), +('2', false, unhex('537061726B2053514C'), tinyint(2), 2, smallint(200), bigint(2), float(2.0), 2.0, Decimal(2.0), timestamp('1997-01-02 03:04:05'), date('2000-04-02')), +('3', true, unhex('537061726B2053514C'), tinyint(3), 3, smallint(300), bigint(3), float(3.0), 3.0, Decimal(3.0), timestamp('1997-02-10 17:32:01-08'), date('2000-04-03')) +AS t(a, b, c, d, e, f, g, h, i, j, k, l) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW script_trans AS SELECT * FROM VALUES +(1, 2, 3), +(4, 5, 6), +(7, 8, 9) +AS script_trans(a, b, c) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW complex_trans AS SELECT * FROM VALUES +(1, 1), +(1, 1), +(2, 2), +(2, 2), +(3, 3), +(2, 2), +(3, 3), +(1, 1), +(3, 3) +as complex_trans(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT TRANSFORM(a) +USING 'cat' AS (a) +FROM t +-- !query schema +struct +-- !query output +1 +2 +3 + + +-- !query +SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( + SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) + USING 'cat' AS ( + a string, + b boolean, + c binary, + d tinyint, + e int, + f smallint, + g long, + h float, + i double, + j decimal(38, 18), + k timestamp, + l date) + FROM t +) tmp +-- !query schema +struct +-- !query output +1 true Spark SQL 1 1 100 1 1.0 1.0 1.000000000000000000 1997-01-02 00:00:00 2000-04-01 +2 false Spark SQL 2 2 200 2 2.0 2.0 2.000000000000000000 1997-01-02 03:04:05 2000-04-02 +3 true Spark SQL 3 3 300 3 3.0 3.0 3.000000000000000000 1997-02-10 17:32:01 2000-04-03 + + +-- !query +SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( + SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) + USING 'cat' AS ( + a string, + b string, + c string, + d string, + e string, + f string, + g string, + h string, + i string, + j string, + k string, + l string) + FROM t +) tmp +-- !query schema +struct +-- !query output +1 true Spark SQL 1 1 100 1 1.0 1.0 1 1997-01-02 00:00:00 2000-04-01 +2 false Spark SQL 2 2 200 2 2.0 2.0 2 1997-01-02 03:04:05 2000-04-02 +3 true Spark SQL 3 3 300 3 3.0 3.0 3 1997-02-10 17:32:01 2000-04-03 + + +-- !query +SELECT TRANSFORM(a) +USING 'cat' +FROM t +-- !query schema +struct +-- !query output +1 NULL +2 NULL +3 NULL + + +-- !query +SELECT TRANSFORM(a, b) +USING 'cat' +FROM t +-- !query schema +struct +-- !query output +1 true +2 false +3 true + + +-- !query +SELECT TRANSFORM(a, b, c) +USING 'cat' +FROM t +-- !query schema +struct +-- !query output +1 true +2 false +3 true + + +-- !query +SELECT TRANSFORM(a, b, c, d, e, f, g, h, i) +USING 'cat' AS (a int, b short, c long, d byte, e float, f double, g decimal(38, 18), h date, i timestamp) +FROM VALUES +('a','','1231a','a','213.21a','213.21a','0a.21d','2000-04-01123','1997-0102 00:00:') tmp(a, b, c, d, e, f, g, h, i) +-- !query schema +struct +-- !query output +NULL NULL NULL NULL NULL NULL NULL NULL NULL + + +-- !query +SELECT TRANSFORM(b, max(a), sum(f)) +USING 'cat' AS (a, b) +FROM t +GROUP BY b +-- !query schema +struct +-- !query output +false 2 +true 3 + + +-- !query +MAP a, b USING 'cat' AS (a, b) FROM t +-- !query schema +struct +-- !query output +1 true +2 false +3 true + + +-- !query +REDUCE a, b USING 'cat' AS (a, b) FROM t +-- !query schema +struct +-- !query output +1 true +2 false +3 true + + +-- !query +SELECT TRANSFORM(a, b, c, null) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' +USING 'cat' AS (a, b, c, d) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' +FROM t +-- !query schema +struct +-- !query output +1 true Spark SQL NULL +2 false Spark SQL NULL +3 true Spark SQL NULL + + +-- !query +SELECT TRANSFORM(a, b, c, null) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' +USING 'cat' AS (d) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' +FROM t +-- !query schema +struct +-- !query output +1 +2 +3 + + +-- !query +SELECT TRANSFORM(a, b, c, null) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' +USING 'cat' AS (a, b, c, d) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' +FROM t +-- !query schema +struct +-- !query output +1 true Spark SQL \N +2 false Spark SQL \N +3 true Spark SQL \N + + +-- !query +SELECT TRANSFORM(a, b, c, null) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' +USING 'cat' AS (a, b, c, d) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' +FROM t +-- !query schema +struct +-- !query output +1 true Spark SQL NULL +2 false Spark SQL NULL +3 true Spark SQL NULL + + +-- !query +SELECT TRANSFORM(a, b, c, null) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'XXXX' +USING 'cat' AS (a, b, c, d) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' +FROM t +-- !query schema +struct +-- !query output +1 true Spark SQL XXXX +2 false Spark SQL XXXX +3 true Spark SQL XXXX + + +-- !query +SELECT TRANSFORM(a, b, c, null) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS '\n' +USING 'cat' AS (a, b, c, d) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' +FROM t +-- !query schema +struct +-- !query output + NULL NULL NULL + NULL NULL NULL + NULL NULL NULL +1 true Spark SQL +2 false Spark SQL +3 true Spark SQL + + +-- !query +SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( + SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' + USING 'cat' AS ( + a string, + b boolean, + c binary, + d tinyint, + e int, + f smallint, + g long, + h float, + i double, + j decimal(38, 18), + k timestamp, + l date) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' + FROM t +) tmp +-- !query schema +struct +-- !query output +1 true Spark SQL 1 1 100 1 1.0 1.0 1.000000000000000000 1997-01-02 00:00:00 2000-04-01 +2 false Spark SQL 2 2 200 2 2.0 2.0 2.000000000000000000 1997-01-02 03:04:05 2000-04-02 +3 true Spark SQL 3 3 300 3 3.0 3.0 3.000000000000000000 1997-02-10 17:32:01 2000-04-03 + + +-- !query +SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( + SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' + USING 'cat' AS ( + a string, + b long, + c binary, + d tinyint, + e int, + f smallint, + g long, + h float, + i double, + j decimal(38, 18), + k int, + l long) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' + FROM t +) tmp +-- !query schema +struct +-- !query output +1 NULL Spark SQL 1 1 100 1 1.0 1.0 1.000000000000000000 NULL NULL +2 NULL Spark SQL 2 2 200 2 2.0 2.0 2.000000000000000000 NULL NULL +3 NULL Spark SQL 3 3 300 3 3.0 3.0 3.000000000000000000 NULL NULL + + +-- !query +SELECT a, b, decode(c, 'UTF-8'), d, e, f, g, h, i, j, k, l FROM ( + SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '@' + NULL DEFINED AS 'NULL' + USING 'cat' AS ( + a string, + b string, + c string, + d string, + e string, + f string, + g string, + h string, + i string, + j string, + k string, + l string) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '@' + NULL DEFINED AS 'NULL' + FROM t +) tmp +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "LINES TERMINATED BY only supports newline '\\n' right now: @" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 69, + "stopIndex" : 560, + "fragment" : "SELECT TRANSFORM(a, b, c, d, e, f, g, h, i, j, k, l)\n ROW FORMAT DELIMITED\n FIELDS TERMINATED BY ','\n LINES TERMINATED BY '@'\n NULL DEFINED AS 'NULL'\n USING 'cat' AS (\n a string,\n b string,\n c string,\n d string,\n e string,\n f string,\n g string,\n h string,\n i string,\n j string,\n k string,\n l string)\n ROW FORMAT DELIMITED\n FIELDS TERMINATED BY ','\n LINES TERMINATED BY '@'\n NULL DEFINED AS 'NULL'\n FROM t" + } ] +} + + +-- !query +SELECT TRANSFORM(b, a, CAST(c AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +-- !query schema +struct +-- !query output +2 1 3 +5 4 6 + + +-- !query +SELECT TRANSFORM(1, 2, 3) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +-- !query schema +struct +-- !query output +1 2 3 +1 2 3 + + +-- !query +SELECT TRANSFORM(1, 2) + USING 'cat' AS (a INT, b INT) +FROM script_trans +LIMIT 1 +-- !query schema +struct +-- !query output +1 2 + + +-- !query +SELECT TRANSFORM( + b, a, + CASE + WHEN c > 100 THEN 1 + WHEN c < 100 THEN 2 + ELSE 3 END) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +-- !query schema +struct +-- !query output +2 1 2 +5 4 2 + + +-- !query +SELECT TRANSFORM(b, a, c + 1) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +-- !query schema +struct +-- !query output +2 1 4 +5 4 7 + + +-- !query +SELECT TRANSFORM(*) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +-- !query schema +struct +-- !query output +1 2 3 +4 5 6 + + +-- !query +SELECT TRANSFORM(b, MAX(a), CAST(SUM(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +GROUP BY b +-- !query schema +struct +-- !query output +2 1 3 +5 4 6 + + +-- !query +SELECT TRANSFORM(b, MAX(a) FILTER (WHERE a > 3), CAST(SUM(c) AS STRING)) + USING 'cat' AS (a,b,c) +FROM script_trans +WHERE a <= 4 +GROUP BY b +-- !query schema +struct +-- !query output +2 NULL 3 +5 4 6 + + +-- !query +SELECT TRANSFORM(b, MAX(a), CAST(sum(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 2 +GROUP BY b +-- !query schema +struct +-- !query output +2 1 3 + + +-- !query +SELECT TRANSFORM(b, MAX(a), CAST(SUM(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +GROUP BY b +HAVING MAX(a) > 0 +-- !query schema +struct +-- !query output +2 1 3 +5 4 6 + + +-- !query +SELECT TRANSFORM(b, MAX(a), CAST(SUM(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +GROUP BY b +HAVING MAX(a) > 1 +-- !query schema +struct +-- !query output +5 4 6 + + +-- !query +SELECT TRANSFORM(b, MAX(a) OVER w, CAST(SUM(c) OVER w AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +WINDOW w AS (PARTITION BY b ORDER BY a) +-- !query schema +struct +-- !query output +2 1 3 +5 4 6 + + +-- !query +SELECT TRANSFORM(b, MAX(a), CAST(SUM(c) AS STRING), myCol, myCol2) + USING 'cat' AS (a STRING, b STRING, c STRING, d ARRAY, e STRING) +FROM script_trans +LATERAL VIEW explode(array(array(1,2,3))) myTable AS myCol +LATERAL VIEW explode(myTable.myCol) myTable2 AS myCol2 +WHERE a <= 4 +GROUP BY b, myCol, myCol2 +HAVING max(a) > 1 +-- !query schema +struct,e:string> +-- !query output +5 4 6 [1,2,3] 1 +5 4 6 [1,2,3] 2 +5 4 6 [1,2,3] 3 + + +-- !query +FROM( + FROM script_trans + SELECT TRANSFORM(a, b) + USING 'cat' AS (`a` INT, b STRING) +) t +SELECT a + 1 +-- !query schema +struct<(a + 1):int> +-- !query output +2 +5 +8 + + +-- !query +FROM( + SELECT TRANSFORM(a, SUM(b)) + USING 'cat' AS (`a` INT, b STRING) + FROM script_trans + GROUP BY a +) t +SELECT (b + 1) AS result +ORDER BY result +-- !query schema +struct +-- !query output +3 +6 +9 + + +-- !query +MAP k / 10 USING 'cat' AS (one) FROM (SELECT 10 AS k) +-- !query schema +struct +-- !query output +1.0 + + +-- !query +FROM (SELECT 1 AS key, 100 AS value) src +MAP src.*, src.key, CAST(src.key / 10 AS INT), CAST(src.key % 10 AS INT), src.value + USING 'cat' AS (k, v, tkey, ten, one, tvalue) +-- !query schema +struct +-- !query output +1 100 1 0 1 100 + + +-- !query +SELECT TRANSFORM(1) + USING 'cat' AS (a) +FROM script_trans +HAVING true +-- !query schema +struct +-- !query output +1 + + +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=true +-- !query schema +struct +-- !query output +spark.sql.legacy.parser.havingWithoutGroupByAsWhere true + + +-- !query +SELECT TRANSFORM(1) + USING 'cat' AS (a) +FROM script_trans +HAVING true +-- !query schema +struct +-- !query output +1 +1 +1 + + +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=false +-- !query schema +struct +-- !query output +spark.sql.legacy.parser.havingWithoutGroupByAsWhere false + + +-- !query +WITH temp AS ( + SELECT TRANSFORM(a) USING 'cat' AS (b string) FROM t +) +SELECT t1.b FROM temp t1 JOIN temp t2 ON t1.b = t2.b +-- !query schema +struct +-- !query output +1 +2 +3 + + +-- !query +SELECT TRANSFORM(DISTINCT b, a, c) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TRANSFORM_DISTINCT_ALL", + "sqlState" : "0A000", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 92, + "fragment" : "SELECT TRANSFORM(DISTINCT b, a, c)\n USING 'cat' AS (a, b, c)\nFROM script_trans\nWHERE a <= 4" + } ] +} + + +-- !query +SELECT TRANSFORM(ALL b, a, c) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 4 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "UNSUPPORTED_FEATURE.TRANSFORM_DISTINCT_ALL", + "sqlState" : "0A000", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 87, + "fragment" : "SELECT TRANSFORM(ALL b, a, c)\n USING 'cat' AS (a, b, c)\nFROM script_trans\nWHERE a <= 4" + } ] +} + + +-- !query +SELECT TRANSFORM(b AS b_1, MAX(a), CAST(sum(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 2 +GROUP BY b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'AS'", + "hint" : "" + } +} + + +-- !query +SELECT TRANSFORM(b b_1, MAX(a), CAST(sum(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 2 +GROUP BY b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'b_1'", + "hint" : "" + } +} + + +-- !query +SELECT TRANSFORM(b, MAX(a) AS max_a, CAST(sum(c) AS STRING)) + USING 'cat' AS (a, b, c) +FROM script_trans +WHERE a <= 2 +GROUP BY b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'AS'", + "hint" : "" + } +} + + +-- !query +FROM ( + SELECT TRANSFORM(a, b) + USING 'cat' AS (a, b) + FROM complex_trans + CLUSTER BY a +) map_output +SELECT TRANSFORM(a, b) + USING 'cat' AS (a, b) +-- !query schema +struct +-- !query output +1 1 +1 1 +1 1 +2 2 +2 2 +2 2 +3 3 +3 3 +3 3 + + +-- !query +FROM ( + SELECT TRANSFORM(a, b) + USING 'cat' AS (a, b) + FROM complex_trans + ORDER BY a +) map_output +SELECT TRANSFORM(a, b) + USING 'cat' AS (a, b) +-- !query schema +struct +-- !query output +1 1 +1 1 +1 1 +2 2 +2 2 +2 2 +3 3 +3 3 +3 3 + + +-- !query +SELECT TRANSFORM (a, b) + USING 'cat' AS (a CHAR(10), b VARCHAR(10)) +FROM VALUES('apache', 'spark') t(a, b) +-- !query schema +struct +-- !query output +apache spark diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try-string-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try-string-functions.sql.out new file mode 100644 index 000000000000..4488bb649654 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try-string-functions.sql.out @@ -0,0 +1,283 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select try_to_binary('', 'base64') +-- !query schema +struct +-- !query output + + + +-- !query +select try_to_binary(' ', 'base64') +-- !query schema +struct +-- !query output + + + +-- !query +select try_to_binary(' ab cd ', 'base64') +-- !query schema +struct +-- !query output +i� + + +-- !query +select try_to_binary(' ab c=', 'base64') +-- !query schema +struct +-- !query output +i� + + +-- !query +select try_to_binary(' ab cdef= = ', 'base64') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_binary( + concat(' b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0', + 'd2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl'), 'base64') +-- !query schema +struct +-- !query output +one two three four five six seven eight nine ten eleven twelve thirteen fourteen fivteen sixteen seventeen eightee + + +-- !query +select try_to_binary('a', 'base64') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_binary('a?', 'base64') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_binary('abcde', 'base64') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_binary('abcd=', 'base64') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_binary('a===', 'base64') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_binary('ab==f', 'base64') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_binary( + '∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β)', 'utf-8') +-- !query schema +struct +-- !query output +∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β) + + +-- !query +select try_to_binary('大千世界', 'utf8') +-- !query schema +struct +-- !query output +大千世界 + + +-- !query +select try_to_binary('', 'utf-8') +-- !query schema +struct +-- !query output + + + +-- !query +select try_to_binary(' ', 'utf8') +-- !query schema +struct +-- !query output + + + +-- !query +select try_to_binary('737472696E67') +-- !query schema +struct +-- !query output +string + + +-- !query +select try_to_binary('737472696E67', 'hex') +-- !query schema +struct +-- !query output +string + + +-- !query +select try_to_binary('') +-- !query schema +struct +-- !query output + + + +-- !query +select try_to_binary('1', 'hex') +-- !query schema +struct +-- !query output + + + +-- !query +select try_to_binary('FF') +-- !query schema +struct +-- !query output +� + + +-- !query +select try_to_binary('123') +-- !query schema +struct +-- !query output +# + + +-- !query +select try_to_binary('12345') +-- !query schema +struct +-- !query output +#E + + +-- !query +select try_to_binary('GG') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_binary('01 AF', 'hex') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_binary('abc', concat('utf', '-8')) +-- !query schema +struct +-- !query output +abc + + +-- !query +select try_to_binary(' ab cdef= = ', substr('base64whynot', 0, 6)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_binary(' ab cdef= = ', replace('HEX0', '0')) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_binary('abc', 'Hex') +-- !query schema +struct +-- !query output + +� + + +-- !query +select try_to_binary('abc', null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_binary(null, 'utf-8') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_binary(null, null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_binary(null, cast(null as string)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_binary('abc', 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_binary('abc', 'invalidFormat') +-- !query schema +struct +-- !query output +NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_aggregates.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_aggregates.sql.out new file mode 100644 index 000000000000..94048ac8897b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_aggregates.sql.out @@ -0,0 +1,415 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_sum(col) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +30 + + +-- !query +SELECT try_sum(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct +-- !query output +30.0 + + +-- !query +SELECT try_sum(col) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +25 + + +-- !query +SELECT try_sum(col) FROM VALUES (NULL), (NULL) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (98765432109876543210987654321098765432BD), (98765432109876543210987654321098765432BD) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +0-2 + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct +-- !query output +0 00:00:02.000000000 + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_sum(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 23, + "fragment" : "col + 1L" + } ] +} + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_avg(col) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +10.0 + + +-- !query +SELECT try_avg(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct +-- !query output +10.00000 + + +-- !query +SELECT try_avg(col) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +12.5 + + +-- !query +SELECT try_avg(col) FROM VALUES (NULL), (NULL) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct +-- !query output +4.6116860184273879E18 + + +-- !query +SELECT try_avg(col) FROM VALUES (98765432109876543210987654321098765432BD), (98765432109876543210987654321098765432BD) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +0-1 + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct +-- !query output +0 00:00:01.000000000 + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_avg(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 23, + "fragment" : "col + 1L" + } ] +} + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_aggregates.sql.out.java21 b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_aggregates.sql.out.java21 new file mode 100644 index 000000000000..9d3c97baecab --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_aggregates.sql.out.java21 @@ -0,0 +1,415 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_sum(col) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +30 + + +-- !query +SELECT try_sum(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct +-- !query output +30.0 + + +-- !query +SELECT try_sum(col) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +25 + + +-- !query +SELECT try_sum(col) FROM VALUES (NULL), (NULL) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (98765432109876543210987654321098765432BD), (98765432109876543210987654321098765432BD) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +0-2 + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct +-- !query output +0 00:00:02.000000000 + + +-- !query +SELECT try_sum(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_sum(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 23, + "fragment" : "col + 1L" + } ] +} + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_sum(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_avg(col) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +10.0 + + +-- !query +SELECT try_avg(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct +-- !query output +10.00000 + + +-- !query +SELECT try_avg(col) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct +-- !query output +12.5 + + +-- !query +SELECT try_avg(col) FROM VALUES (NULL), (NULL) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct +-- !query output +4.611686018427388E18 + + +-- !query +SELECT try_avg(col) FROM VALUES (98765432109876543210987654321098765432BD), (98765432109876543210987654321098765432BD) AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +0-1 + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct +-- !query output +0 00:00:01.000000000 + + +-- !query +SELECT try_avg(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (5), (10), (15) AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_avg(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 23, + "fragment" : "col + 1L" + } ] +} + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} + + +-- !query +SELECT try_avg(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "INTERVAL_DIVIDED_BY_ZERO", + "sqlState" : "22012", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 22, + "fragment" : "col / 0" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_arithmetic.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_arithmetic.sql.out new file mode 100644 index 000000000000..5128c5c400df --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_arithmetic.sql.out @@ -0,0 +1,705 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_add(1, 1) +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT try_add(2147483647, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(2147483647, decimal(1)) +-- !query schema +struct +-- !query output +2147483648 + + +-- !query +SELECT try_add(2147483647, "1") +-- !query schema +struct +-- !query output +2147483648 + + +-- !query +SELECT try_add(-2147483648, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(9223372036854775807L, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(-9223372036854775808L, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(1, 1.0 / 0.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 27, + "fragment" : "1.0 / 0.0" + } ] +} + + +-- !query +SELECT try_add(date'2021-01-01', 1) +-- !query schema +struct +-- !query output +2021-01-02 + + +-- !query +SELECT try_add(1, date'2021-01-01') +-- !query schema +struct +-- !query output +2021-01-02 + + +-- !query +SELECT try_add(date'2021-01-01', interval 2 year) +-- !query schema +struct +-- !query output +2023-01-01 + + +-- !query +SELECT try_add(date'2021-01-01', interval 2 second) +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(interval 2 year, date'2021-01-01') +-- !query schema +struct +-- !query output +2023-01-01 + + +-- !query +SELECT try_add(interval 2 second, date'2021-01-01') +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(timestamp_ltz'2021-01-01 00:00:00', interval 2 year) +-- !query schema +struct +-- !query output +2023-01-01 00:00:00 + + +-- !query +SELECT try_add(timestamp_ntz'2021-01-01 00:00:00', interval 2 second) +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(interval 2 year, timestamp_ltz'2021-01-01 00:00:00') +-- !query schema +struct +-- !query output +2023-01-01 00:00:00 + + +-- !query +SELECT try_add(interval 2 second, timestamp_ntz'2021-01-01 00:00:00') +-- !query schema +struct +-- !query output +2021-01-01 00:00:02 + + +-- !query +SELECT try_add(interval 2 year, interval 2 year) +-- !query schema +struct +-- !query output +4-0 + + +-- !query +SELECT try_add(interval 2 second, interval 2 second) +-- !query schema +struct +-- !query output +0 00:00:04.000000000 + + +-- !query +SELECT try_add(interval 2 year, interval 2 second) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"INTERVAL '2' YEAR\"", + "inputType" : "\"INTERVAL YEAR\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"INTERVAL '2' YEAR + INTERVAL '02' SECOND\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "try_add(interval 2 year, interval 2 second)" + } ] +} + + +-- !query +SELECT try_add(interval 2147483647 month, interval 2 month) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_add(interval 106751991 day, interval 3 day) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, 0.5) +-- !query schema +struct +-- !query output +2.000000 + + +-- !query +SELECT try_divide(1, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(0, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, (2147483647 + 1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 36, + "fragment" : "2147483647 + 1" + } ] +} + + +-- !query +SELECT try_divide(1L, (9223372036854775807L + 1L)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 48, + "fragment" : "9223372036854775807L + 1L" + } ] +} + + +-- !query +SELECT try_divide(1, 1.0 / 0.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 30, + "fragment" : "1.0 / 0.0" + } ] +} + + +-- !query +SELECT try_divide(1, decimal(0)) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(1, "0") +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 2 year, 2) +-- !query schema +struct +-- !query output +1-0 + + +-- !query +SELECT try_divide(interval 2 second, 2) +-- !query schema +struct +-- !query output +0 00:00:01.000000000 + + +-- !query +SELECT try_divide(interval 2 year, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 2 second, 0) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 2147483647 month, 0.5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_divide(interval 106751991 day, 0.5) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(1, 1) +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT try_subtract(2147483647, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(2147483647, decimal(-1)) +-- !query schema +struct +-- !query output +2147483648 + + +-- !query +SELECT try_subtract(2147483647, "-1") +-- !query schema +struct +-- !query output +2147483648 + + +-- !query +SELECT try_subtract(-2147483648, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(9223372036854775807L, -1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(-9223372036854775808L, 1) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(1, (2147483647 + 1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 38, + "fragment" : "2147483647 + 1" + } ] +} + + +-- !query +SELECT try_subtract(1L, (9223372036854775807L + 1L)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 50, + "fragment" : "9223372036854775807L + 1L" + } ] +} + + +-- !query +SELECT try_subtract(1, 1.0 / 0.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 32, + "fragment" : "1.0 / 0.0" + } ] +} + + +-- !query +SELECT try_subtract(interval 2 year, interval 3 year) +-- !query schema +struct +-- !query output +-1-0 + + +-- !query +SELECT try_subtract(interval 3 second, interval 2 second) +-- !query schema +struct +-- !query output +0 00:00:01.000000000 + + +-- !query +SELECT try_subtract(interval 2147483647 month, interval -2 month) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_subtract(interval 106751991 day, interval -3 day) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(2, 3) +-- !query schema +struct +-- !query output +6 + + +-- !query +SELECT try_multiply(2147483647, -2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(2147483647, decimal(-2)) +-- !query schema +struct +-- !query output +-4294967294 + + +-- !query +SELECT try_multiply(2147483647, "-2") +-- !query schema +struct +-- !query output +-4294967294 + + +-- !query +SELECT try_multiply(-2147483648, 2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(9223372036854775807L, 2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(-9223372036854775808L, -2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(1, (2147483647 + 1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "integer overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "try_multiply(1, (2147483647 + 1))" + } ] +} + + +-- !query +SELECT try_multiply(1L, (9223372036854775807L + 1L)) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "ARITHMETIC_OVERFLOW", + "sqlState" : "22003", + "messageParameters" : { + "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.", + "config" : "\"spark.sql.ansi.enabled\"", + "message" : "long overflow" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 50, + "fragment" : "9223372036854775807L + 1L" + } ] +} + + +-- !query +SELECT try_multiply(1, 1.0 / 0.0) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 24, + "stopIndex" : 32, + "fragment" : "1.0 / 0.0" + } ] +} + + +-- !query +SELECT try_multiply(interval 2 year, 2) +-- !query schema +struct +-- !query output +4-0 + + +-- !query +SELECT try_multiply(interval 2 second, 2) +-- !query schema +struct +-- !query output +0 00:00:04.000000000 + + +-- !query +SELECT try_multiply(interval 2 year, 0) +-- !query schema +struct +-- !query output +0-0 + + +-- !query +SELECT try_multiply(interval 2 second, 0) +-- !query schema +struct +-- !query output +0 00:00:00.000000000 + + +-- !query +SELECT try_multiply(interval 2147483647 month, 2) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_multiply(interval 106751991 day, 2) +-- !query schema +struct +-- !query output +NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_cast.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_cast.sql.out new file mode 100644 index 000000000000..aa927cbd1174 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_cast.sql.out @@ -0,0 +1,199 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT TRY_CAST('abc' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('abc' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST(NULL AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST(NULL AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('123.a' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('123.a' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('-2147483648' AS int) +-- !query schema +struct +-- !query output +-2147483648 + + +-- !query +SELECT TRY_CAST('-2147483649' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('2147483647' AS int) +-- !query schema +struct +-- !query output +2147483647 + + +-- !query +SELECT TRY_CAST('2147483648' AS int) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('-9223372036854775808' AS long) +-- !query schema +struct +-- !query output +-9223372036854775808 + + +-- !query +SELECT TRY_CAST('-9223372036854775809' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('9223372036854775807' AS long) +-- !query schema +struct +-- !query output +9223372036854775807 + + +-- !query +SELECT TRY_CAST('9223372036854775808' AS long) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST('interval 3 month 1 hour' AS interval) +-- !query schema +struct +-- !query output +3 months 1 hours + + +-- !query +SELECT TRY_CAST('abc' AS interval) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select TRY_CAST('true' as boolean) +-- !query schema +struct +-- !query output +true + + +-- !query +select TRY_CAST('false' as boolean) +-- !query schema +struct +-- !query output +false + + +-- !query +select TRY_CAST('abc' as boolean) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST("2021-01-01" AS date) +-- !query schema +struct +-- !query output +2021-01-01 + + +-- !query +SELECT TRY_CAST("2021-101-01" AS date) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT TRY_CAST("2021-01-01 00:00:00" AS timestamp) +-- !query schema +struct +-- !query output +2021-01-01 00:00:00 + + +-- !query +SELECT TRY_CAST("2021-101-01 00:00:00" AS timestamp) +-- !query schema +struct +-- !query output +NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_datetime_functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_datetime_functions.sql.out new file mode 100644 index 000000000000..75a6f15bd363 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_datetime_functions.sql.out @@ -0,0 +1,56 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select try_to_timestamp(null), try_to_timestamp('2016-12-31 00:12:00'), try_to_timestamp('2016-12-31', 'yyyy-MM-dd') +-- !query schema +struct +-- !query output +NULL 2016-12-31 00:12:00 2016-12-31 00:00:00 + + +-- !query +select try_to_timestamp(1) +-- !query schema +struct +-- !query output +1969-12-31 16:00:01 + + +-- !query +select try_to_timestamp('2016-12-31 abc') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_timestamp("02-29", "MM-dd") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkUpgradeException +{ + "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION", + "sqlState" : "42K0B", + "messageParameters" : { + "config" : "\"spark.sql.legacy.timeParserPolicy\"", + "docroot" : "https://spark.apache.org/docs/latest", + "pattern" : "'dd MM yyyy EEEEEE'" + } +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_element_at.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_element_at.sql.out new file mode 100644 index 000000000000..0437f9d6dd9e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_element_at.sql.out @@ -0,0 +1,67 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_element_at(array(1, 2, 3), 0) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "INVALID_INDEX_OF_ZERO", + "sqlState" : "22003" +} + + +-- !query +SELECT try_element_at(array(1, 2, 3), 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT try_element_at(array(1, 2, 3), 3) +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT try_element_at(array(1, 2, 3), 4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_element_at(array(1, 2, 3), -1) +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT try_element_at(array(1, 2, 3), -4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_element_at(map('a','b'), 'a') +-- !query schema +struct +-- !query output +b + + +-- !query +SELECT try_element_at(map('a','b'), 'abc') +-- !query schema +struct +-- !query output +NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_reflect.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_reflect.sql.out new file mode 100644 index 000000000000..8b2c4f0ed737 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/try_reflect.sql.out @@ -0,0 +1,170 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT try_reflect("java.util.UUID", "fromString", "a5cf6c42-0c85-418f-af6c-3e4e5b1328f2") +-- !query schema +struct +-- !query output +a5cf6c42-0c85-418f-af6c-3e4e5b1328f2 + + +-- !query +SELECT try_reflect("java.lang.String", "valueOf", 1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT try_reflect("java.lang.Math", "max", 2, 3) +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT try_reflect("java.lang.Math", "min", 2, 3) +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT try_reflect("java.lang.Integer", "valueOf", "10", 16) +-- !query schema +struct +-- !query output +16 + + +-- !query +SELECT try_reflect("java.util.UUID", "fromString", "b") +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_reflect("java.net.URLDecoder", "decode", "%") +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT try_reflect("java.wrongclass.Math", "max", 2, 3) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_CLASS_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "className" : "java.wrongclass.Math", + "sqlExpr" : "\"reflect(java.wrongclass.Math, max, 2, 3)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "try_reflect(\"java.wrongclass.Math\", \"max\", 2, 3)" + } ] +} + + +-- !query +SELECT try_reflect("java.lang.Math", "wrongmethod", 2, 3) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_STATIC_METHOD", + "sqlState" : "42K09", + "messageParameters" : { + "className" : "java.lang.Math", + "methodName" : "wrongmethod", + "sqlExpr" : "\"reflect(java.lang.Math, wrongmethod, 2, 3)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "try_reflect(\"java.lang.Math\", \"wrongmethod\", 2, 3)" + } ] +} + + +-- !query +SELECT try_reflect("java.lang.Math") +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "> 1", + "functionName" : "`reflect`" + } +} + + +-- !query +SELECT try_reflect("java.lang.Math", "round", 2.5) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"2.5\"", + "inputType" : "\"DECIMAL(2,1)\"", + "paramIndex" : "third", + "requiredType" : "(\"BOOLEAN\" or \"TINYINT\" or \"SMALLINT\" or \"INT\" or \"BIGINT\" or \"FLOAT\" or \"DOUBLE\" or \"STRING\")", + "sqlExpr" : "\"reflect(java.lang.Math, round, 2.5)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "try_reflect(\"java.lang.Math\", \"round\", 2.5)" + } ] +} + + +-- !query +SELECT try_reflect("java.lang.Object", "toString") +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_STATIC_METHOD", + "sqlState" : "42K09", + "messageParameters" : { + "className" : "java.lang.Object", + "methodName" : "toString", + "sqlExpr" : "\"reflect(java.lang.Object, toString)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "try_reflect(\"java.lang.Object\", \"toString\")" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/arrayJoin.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/arrayJoin.sql.out new file mode 100644 index 000000000000..6e33fab7a32f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/arrayJoin.sql.out @@ -0,0 +1,87 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT array_join(array(true, false), ', ') +-- !query schema +struct +-- !query output +true, false + + +-- !query +SELECT array_join(array(2Y, 1Y), ', ') +-- !query schema +struct +-- !query output +2, 1 + + +-- !query +SELECT array_join(array(2S, 1S), ', ') +-- !query schema +struct +-- !query output +2, 1 + + +-- !query +SELECT array_join(array(2, 1), ', ') +-- !query schema +struct +-- !query output +2, 1 + + +-- !query +SELECT array_join(array(2L, 1L), ', ') +-- !query schema +struct +-- !query output +2, 1 + + +-- !query +SELECT array_join(array(9223372036854775809, 9223372036854775808), ', ') +-- !query schema +struct +-- !query output +9223372036854775809, 9223372036854775808 + + +-- !query +SELECT array_join(array(2.0D, 1.0D), ', ') +-- !query schema +struct +-- !query output +2.0, 1.0 + + +-- !query +SELECT array_join(array(float(2.0), float(1.0)), ', ') +-- !query schema +struct +-- !query output +2.0, 1.0 + + +-- !query +SELECT array_join(array(date '2016-03-14', date '2016-03-13'), ', ') +-- !query schema +struct +-- !query output +2016-03-14, 2016-03-13 + + +-- !query +SELECT array_join(array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), ', ') +-- !query schema +struct +-- !query output +2016-11-15 20:54:00, 2016-11-12 20:54:00 + + +-- !query +SELECT array_join(array('a', 'b'), ', ') +-- !query schema +struct +-- !query output +a, b diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/binaryComparison.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/binaryComparison.sql.out new file mode 100644 index 000000000000..f3263241a556 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/binaryComparison.sql.out @@ -0,0 +1,2551 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT cast(1 as binary) = '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT cast(1 as binary) > '2' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT cast(1 as binary) >= '2' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT cast(1 as binary) < '2' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT cast(1 as binary) <= '2' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT cast(1 as binary) <> '2' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT cast(1 as binary) = cast(null as string) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT cast(1 as binary) > cast(null as string) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT cast(1 as binary) >= cast(null as string) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT cast(1 as binary) < cast(null as string) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT cast(1 as binary) <= cast(null as string) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT cast(1 as binary) <> cast(null as string) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT '1' = cast(1 as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 14, + "stopIndex" : 30, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT '2' > cast(1 as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 14, + "stopIndex" : 30, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT '2' >= cast(1 as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 31, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT '2' < cast(1 as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 14, + "stopIndex" : 30, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT '2' <= cast(1 as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 31, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT '2' <> cast(1 as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 31, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT cast(null as string) = cast(1 as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 31, + "stopIndex" : 47, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT cast(null as string) > cast(1 as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 31, + "stopIndex" : 47, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT cast(null as string) >= cast(1 as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 48, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT cast(null as string) < cast(1 as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 31, + "stopIndex" : 47, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT cast(null as string) <= cast(1 as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 48, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT cast(null as string) <> cast(1 as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"", + "configVal" : "'false'", + "sqlExpr" : "\"CAST(1 AS BINARY)\"", + "srcType" : "\"INT\"", + "targetType" : "\"BINARY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 32, + "stopIndex" : 48, + "fragment" : "cast(1 as binary)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) = '1' FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) = 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) > '2' FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) > 2):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as tinyint) >= '2' FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) >= 2):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as tinyint) < '2' FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) < 2):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) <= '2' FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) <= 2):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) <> '2' FROM t +-- !query schema +struct<(NOT (CAST(1 AS TINYINT) = 2)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) = cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) = CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as tinyint) > cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) > CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as tinyint) >= cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) >= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as tinyint) < cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) < CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as tinyint) <= cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) <= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as tinyint) <> cast(null as string) FROM t +-- !query schema +struct<(NOT (CAST(1 AS TINYINT) = CAST(NULL AS STRING))):boolean> +-- !query output +NULL + + +-- !query +SELECT '1' = cast(1 as tinyint) FROM t +-- !query schema +struct<(1 = CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT '2' > cast(1 as tinyint) FROM t +-- !query schema +struct<(2 > CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT '2' >= cast(1 as tinyint) FROM t +-- !query schema +struct<(2 >= CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT '2' < cast(1 as tinyint) FROM t +-- !query schema +struct<(2 < CAST(1 AS TINYINT)):boolean> +-- !query output +false + + +-- !query +SELECT '2' <= cast(1 as tinyint) FROM t +-- !query schema +struct<(2 <= CAST(1 AS TINYINT)):boolean> +-- !query output +false + + +-- !query +SELECT '2' <> cast(1 as tinyint) FROM t +-- !query schema +struct<(NOT (2 = CAST(1 AS TINYINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(null as string) = cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) = CAST(1 AS TINYINT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) > cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) > CAST(1 AS TINYINT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) >= cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) >= CAST(1 AS TINYINT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) < cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) < CAST(1 AS TINYINT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) <= cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) <= CAST(1 AS TINYINT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) <> cast(1 as tinyint) FROM t +-- !query schema +struct<(NOT (CAST(NULL AS STRING) = CAST(1 AS TINYINT))):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as smallint) = '1' FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) = 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) > '2' FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) > 2):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as smallint) >= '2' FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) >= 2):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as smallint) < '2' FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) < 2):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) <= '2' FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) <= 2):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) <> '2' FROM t +-- !query schema +struct<(NOT (CAST(1 AS SMALLINT) = 2)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) = cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) = CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as smallint) > cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) > CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as smallint) >= cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) >= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as smallint) < cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) < CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as smallint) <= cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) <= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as smallint) <> cast(null as string) FROM t +-- !query schema +struct<(NOT (CAST(1 AS SMALLINT) = CAST(NULL AS STRING))):boolean> +-- !query output +NULL + + +-- !query +SELECT '1' = cast(1 as smallint) FROM t +-- !query schema +struct<(1 = CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT '2' > cast(1 as smallint) FROM t +-- !query schema +struct<(2 > CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT '2' >= cast(1 as smallint) FROM t +-- !query schema +struct<(2 >= CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT '2' < cast(1 as smallint) FROM t +-- !query schema +struct<(2 < CAST(1 AS SMALLINT)):boolean> +-- !query output +false + + +-- !query +SELECT '2' <= cast(1 as smallint) FROM t +-- !query schema +struct<(2 <= CAST(1 AS SMALLINT)):boolean> +-- !query output +false + + +-- !query +SELECT '2' <> cast(1 as smallint) FROM t +-- !query schema +struct<(NOT (2 = CAST(1 AS SMALLINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(null as string) = cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) = CAST(1 AS SMALLINT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) > cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) > CAST(1 AS SMALLINT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) >= cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) >= CAST(1 AS SMALLINT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) < cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) < CAST(1 AS SMALLINT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) <= cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) <= CAST(1 AS SMALLINT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) <> cast(1 as smallint) FROM t +-- !query schema +struct<(NOT (CAST(NULL AS STRING) = CAST(1 AS SMALLINT))):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as int) = '1' FROM t +-- !query schema +struct<(CAST(1 AS INT) = 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) > '2' FROM t +-- !query schema +struct<(CAST(1 AS INT) > 2):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as int) >= '2' FROM t +-- !query schema +struct<(CAST(1 AS INT) >= 2):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as int) < '2' FROM t +-- !query schema +struct<(CAST(1 AS INT) < 2):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) <= '2' FROM t +-- !query schema +struct<(CAST(1 AS INT) <= 2):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) <> '2' FROM t +-- !query schema +struct<(NOT (CAST(1 AS INT) = 2)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) = cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS INT) = CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as int) > cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS INT) > CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as int) >= cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS INT) >= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as int) < cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS INT) < CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as int) <= cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS INT) <= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as int) <> cast(null as string) FROM t +-- !query schema +struct<(NOT (CAST(1 AS INT) = CAST(NULL AS STRING))):boolean> +-- !query output +NULL + + +-- !query +SELECT '1' = cast(1 as int) FROM t +-- !query schema +struct<(1 = CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT '2' > cast(1 as int) FROM t +-- !query schema +struct<(2 > CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT '2' >= cast(1 as int) FROM t +-- !query schema +struct<(2 >= CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT '2' < cast(1 as int) FROM t +-- !query schema +struct<(2 < CAST(1 AS INT)):boolean> +-- !query output +false + + +-- !query +SELECT '2' <> cast(1 as int) FROM t +-- !query schema +struct<(NOT (2 = CAST(1 AS INT))):boolean> +-- !query output +true + + +-- !query +SELECT '2' <= cast(1 as int) FROM t +-- !query schema +struct<(2 <= CAST(1 AS INT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(null as string) = cast(1 as int) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) = CAST(1 AS INT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) > cast(1 as int) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) > CAST(1 AS INT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) >= cast(1 as int) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) >= CAST(1 AS INT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) < cast(1 as int) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) < CAST(1 AS INT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) <> cast(1 as int) FROM t +-- !query schema +struct<(NOT (CAST(NULL AS STRING) = CAST(1 AS INT))):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) <= cast(1 as int) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) <= CAST(1 AS INT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as bigint) = '1' FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) = 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) > '2' FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) > 2):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as bigint) >= '2' FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) >= 2):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as bigint) < '2' FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) < 2):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) <= '2' FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) <= 2):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) <> '2' FROM t +-- !query schema +struct<(NOT (CAST(1 AS BIGINT) = 2)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) = cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) = CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as bigint) > cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) > CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as bigint) >= cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) >= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as bigint) < cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) < CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as bigint) <= cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) <= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as bigint) <> cast(null as string) FROM t +-- !query schema +struct<(NOT (CAST(1 AS BIGINT) = CAST(NULL AS STRING))):boolean> +-- !query output +NULL + + +-- !query +SELECT '1' = cast(1 as bigint) FROM t +-- !query schema +struct<(1 = CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT '2' > cast(1 as bigint) FROM t +-- !query schema +struct<(2 > CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT '2' >= cast(1 as bigint) FROM t +-- !query schema +struct<(2 >= CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT '2' < cast(1 as bigint) FROM t +-- !query schema +struct<(2 < CAST(1 AS BIGINT)):boolean> +-- !query output +false + + +-- !query +SELECT '2' <= cast(1 as bigint) FROM t +-- !query schema +struct<(2 <= CAST(1 AS BIGINT)):boolean> +-- !query output +false + + +-- !query +SELECT '2' <> cast(1 as bigint) FROM t +-- !query schema +struct<(NOT (2 = CAST(1 AS BIGINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(null as string) = cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) = CAST(1 AS BIGINT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) > cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) > CAST(1 AS BIGINT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) >= cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) >= CAST(1 AS BIGINT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) < cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) < CAST(1 AS BIGINT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) <= cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) <= CAST(1 AS BIGINT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) <> cast(1 as bigint) FROM t +-- !query schema +struct<(NOT (CAST(NULL AS STRING) = CAST(1 AS BIGINT))):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as decimal(10, 0)) = '1' FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) = 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) > '2' FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) > 2):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= '2' FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) >= 2):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) < '2' FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) < 2):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> '2' FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = 2)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= '2' FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <= 2):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) = CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) > CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) < CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast(null as string) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(NULL AS STRING))):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT '1' = cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(1 = CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT '2' > cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(2 > CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT '2' >= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(2 >= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT '2' < cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(2 < CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT '2' <= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(2 <= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT '2' <> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(NOT (2 = CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +true + + +-- !query +SELECT cast(null as string) = cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) = CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) > cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) > CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) >= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) >= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) < cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) < CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) <= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) <= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) <> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(NOT (CAST(NULL AS STRING) = CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as double) = '1' FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) = 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) > '2' FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) > 2):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as double) >= '2' FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) >= 2):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as double) < '2' FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) < 2):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) <= '2' FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) <= 2):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) <> '2' FROM t +-- !query schema +struct<(NOT (CAST(1 AS DOUBLE) = 2)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) = cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) = CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as double) > cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) > CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as double) >= cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) >= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as double) < cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) < CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as double) <= cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) <= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as double) <> cast(null as string) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DOUBLE) = CAST(NULL AS STRING))):boolean> +-- !query output +NULL + + +-- !query +SELECT '1' = cast(1 as double) FROM t +-- !query schema +struct<(1 = CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT '2' > cast(1 as double) FROM t +-- !query schema +struct<(2 > CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT '2' >= cast(1 as double) FROM t +-- !query schema +struct<(2 >= CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT '2' < cast(1 as double) FROM t +-- !query schema +struct<(2 < CAST(1 AS DOUBLE)):boolean> +-- !query output +false + + +-- !query +SELECT '2' <= cast(1 as double) FROM t +-- !query schema +struct<(2 <= CAST(1 AS DOUBLE)):boolean> +-- !query output +false + + +-- !query +SELECT '2' <> cast(1 as double) FROM t +-- !query schema +struct<(NOT (2 = CAST(1 AS DOUBLE))):boolean> +-- !query output +true + + +-- !query +SELECT cast(null as string) = cast(1 as double) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) = CAST(1 AS DOUBLE)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) > cast(1 as double) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) > CAST(1 AS DOUBLE)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) >= cast(1 as double) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) >= CAST(1 AS DOUBLE)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) < cast(1 as double) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) < CAST(1 AS DOUBLE)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) <= cast(1 as double) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) <= CAST(1 AS DOUBLE)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) <> cast(1 as double) FROM t +-- !query schema +struct<(NOT (CAST(NULL AS STRING) = CAST(1 AS DOUBLE))):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as float) = '1' FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) = 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) > '2' FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) > 2):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as float) >= '2' FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) >= 2):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as float) < '2' FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) < 2):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) <= '2' FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) <= 2):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) <> '2' FROM t +-- !query schema +struct<(NOT (CAST(1 AS FLOAT) = 2)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) = cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) = CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as float) > cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) > CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as float) >= cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) >= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as float) < cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) < CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as float) <= cast(null as string) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) <= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(1 as float) <> cast(null as string) FROM t +-- !query schema +struct<(NOT (CAST(1 AS FLOAT) = CAST(NULL AS STRING))):boolean> +-- !query output +NULL + + +-- !query +SELECT '1' = cast(1 as float) FROM t +-- !query schema +struct<(1 = CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT '2' > cast(1 as float) FROM t +-- !query schema +struct<(2 > CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT '2' >= cast(1 as float) FROM t +-- !query schema +struct<(2 >= CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT '2' < cast(1 as float) FROM t +-- !query schema +struct<(2 < CAST(1 AS FLOAT)):boolean> +-- !query output +false + + +-- !query +SELECT '2' <= cast(1 as float) FROM t +-- !query schema +struct<(2 <= CAST(1 AS FLOAT)):boolean> +-- !query output +false + + +-- !query +SELECT '2' <> cast(1 as float) FROM t +-- !query schema +struct<(NOT (2 = CAST(1 AS FLOAT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(null as string) = cast(1 as float) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) = CAST(1 AS FLOAT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) > cast(1 as float) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) > CAST(1 AS FLOAT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) >= cast(1 as float) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) >= CAST(1 AS FLOAT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) < cast(1 as float) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) < CAST(1 AS FLOAT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) <= cast(1 as float) FROM t +-- !query schema +struct<(CAST(NULL AS STRING) <= CAST(1 AS FLOAT)):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) <> cast(1 as float) FROM t +-- !query schema +struct<(NOT (CAST(NULL AS STRING) = CAST(1 AS FLOAT))):boolean> +-- !query output +NULL + + +-- !query +SELECT '1996-09-09' = date('1996-09-09') FROM t +-- !query schema +struct<(1996-09-09 = 1996-09-09):boolean> +-- !query output +true + + +-- !query +SELECT '1996-9-10' > date('1996-09-09') FROM t +-- !query schema +struct<(1996-9-10 > 1996-09-09):boolean> +-- !query output +true + + +-- !query +SELECT '1996-9-10' >= date('1996-09-09') FROM t +-- !query schema +struct<(1996-9-10 >= 1996-09-09):boolean> +-- !query output +true + + +-- !query +SELECT '1996-9-10' < date('1996-09-09') FROM t +-- !query schema +struct<(1996-9-10 < 1996-09-09):boolean> +-- !query output +false + + +-- !query +SELECT '1996-9-10' <= date('1996-09-09') FROM t +-- !query schema +struct<(1996-9-10 <= 1996-09-09):boolean> +-- !query output +false + + +-- !query +SELECT '1996-9-10' <> date('1996-09-09') FROM t +-- !query schema +struct<(NOT (1996-9-10 = 1996-09-09)):boolean> +-- !query output +true + + +-- !query +SELECT cast(null as string) = date('1996-09-09') FROM t +-- !query schema +struct<(CAST(NULL AS STRING) = 1996-09-09):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string)> date('1996-09-09') FROM t +-- !query schema +struct<(CAST(NULL AS STRING) > 1996-09-09):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string)>= date('1996-09-09') FROM t +-- !query schema +struct<(CAST(NULL AS STRING) >= 1996-09-09):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string)< date('1996-09-09') FROM t +-- !query schema +struct<(CAST(NULL AS STRING) < 1996-09-09):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string)<= date('1996-09-09') FROM t +-- !query schema +struct<(CAST(NULL AS STRING) <= 1996-09-09):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string)<> date('1996-09-09') FROM t +-- !query schema +struct<(NOT (CAST(NULL AS STRING) = 1996-09-09)):boolean> +-- !query output +NULL + + +-- !query +SELECT date('1996-09-09') = '1996-09-09' FROM t +-- !query schema +struct<(1996-09-09 = 1996-09-09):boolean> +-- !query output +true + + +-- !query +SELECT date('1996-9-10') > '1996-09-09' FROM t +-- !query schema +struct<(1996-9-10 > 1996-09-09):boolean> +-- !query output +true + + +-- !query +SELECT date('1996-9-10') >= '1996-09-09' FROM t +-- !query schema +struct<(1996-9-10 >= 1996-09-09):boolean> +-- !query output +true + + +-- !query +SELECT date('1996-9-10') < '1996-09-09' FROM t +-- !query schema +struct<(1996-9-10 < 1996-09-09):boolean> +-- !query output +false + + +-- !query +SELECT date('1996-9-10') <= '1996-09-09' FROM t +-- !query schema +struct<(1996-9-10 <= 1996-09-09):boolean> +-- !query output +false + + +-- !query +SELECT date('1996-9-10') <> '1996-09-09' FROM t +-- !query schema +struct<(NOT (1996-9-10 = 1996-09-09)):boolean> +-- !query output +true + + +-- !query +SELECT date('1996-09-09') = cast(null as string) FROM t +-- !query schema +struct<(1996-09-09 = CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT date('1996-9-10') > cast(null as string) FROM t +-- !query schema +struct<(1996-9-10 > CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT date('1996-9-10') >= cast(null as string) FROM t +-- !query schema +struct<(1996-9-10 >= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT date('1996-9-10') < cast(null as string) FROM t +-- !query schema +struct<(1996-9-10 < CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT date('1996-9-10') <= cast(null as string) FROM t +-- !query schema +struct<(1996-9-10 <= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT date('1996-9-10') <> cast(null as string) FROM t +-- !query schema +struct<(NOT (1996-9-10 = CAST(NULL AS STRING))):boolean> +-- !query output +NULL + + +-- !query +SELECT '1996-09-09 12:12:12.4' = timestamp('1996-09-09 12:12:12.4') FROM t +-- !query schema +struct<(1996-09-09 12:12:12.4 = 1996-09-09 12:12:12.4):boolean> +-- !query output +true + + +-- !query +SELECT '1996-09-09 12:12:12.5' > timestamp('1996-09-09 12:12:12.4') FROM t +-- !query schema +struct<(1996-09-09 12:12:12.5 > 1996-09-09 12:12:12.4):boolean> +-- !query output +true + + +-- !query +SELECT '1996-09-09 12:12:12.5' >= timestamp('1996-09-09 12:12:12.4') FROM t +-- !query schema +struct<(1996-09-09 12:12:12.5 >= 1996-09-09 12:12:12.4):boolean> +-- !query output +true + + +-- !query +SELECT '1996-09-09 12:12:12.5' < timestamp('1996-09-09 12:12:12.4') FROM t +-- !query schema +struct<(1996-09-09 12:12:12.5 < 1996-09-09 12:12:12.4):boolean> +-- !query output +false + + +-- !query +SELECT '1996-09-09 12:12:12.5' <= timestamp('1996-09-09 12:12:12.4') FROM t +-- !query schema +struct<(1996-09-09 12:12:12.5 <= 1996-09-09 12:12:12.4):boolean> +-- !query output +false + + +-- !query +SELECT '1996-09-09 12:12:12.5' <> timestamp('1996-09-09 12:12:12.4') FROM t +-- !query schema +struct<(NOT (1996-09-09 12:12:12.5 = 1996-09-09 12:12:12.4)):boolean> +-- !query output +true + + +-- !query +SELECT cast(null as string) = timestamp('1996-09-09 12:12:12.4') FROM t +-- !query schema +struct<(CAST(NULL AS STRING) = 1996-09-09 12:12:12.4):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) > timestamp('1996-09-09 12:12:12.4') FROM t +-- !query schema +struct<(CAST(NULL AS STRING) > 1996-09-09 12:12:12.4):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) >= timestamp('1996-09-09 12:12:12.4') FROM t +-- !query schema +struct<(CAST(NULL AS STRING) >= 1996-09-09 12:12:12.4):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) < timestamp('1996-09-09 12:12:12.4') FROM t +-- !query schema +struct<(CAST(NULL AS STRING) < 1996-09-09 12:12:12.4):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) <= timestamp('1996-09-09 12:12:12.4') FROM t +-- !query schema +struct<(CAST(NULL AS STRING) <= 1996-09-09 12:12:12.4):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) <> timestamp('1996-09-09 12:12:12.4') FROM t +-- !query schema +struct<(NOT (CAST(NULL AS STRING) = 1996-09-09 12:12:12.4)):boolean> +-- !query output +NULL + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.4' )= '1996-09-09 12:12:12.4' FROM t +-- !query schema +struct<(1996-09-09 12:12:12.4 = 1996-09-09 12:12:12.4):boolean> +-- !query output +true + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )> '1996-09-09 12:12:12.4' FROM t +-- !query schema +struct<(1996-09-09 12:12:12.5 > 1996-09-09 12:12:12.4):boolean> +-- !query output +true + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )>= '1996-09-09 12:12:12.4' FROM t +-- !query schema +struct<(1996-09-09 12:12:12.5 >= 1996-09-09 12:12:12.4):boolean> +-- !query output +true + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )< '1996-09-09 12:12:12.4' FROM t +-- !query schema +struct<(1996-09-09 12:12:12.5 < 1996-09-09 12:12:12.4):boolean> +-- !query output +false + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )<= '1996-09-09 12:12:12.4' FROM t +-- !query schema +struct<(1996-09-09 12:12:12.5 <= 1996-09-09 12:12:12.4):boolean> +-- !query output +false + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )<> '1996-09-09 12:12:12.4' FROM t +-- !query schema +struct<(NOT (1996-09-09 12:12:12.5 = 1996-09-09 12:12:12.4)):boolean> +-- !query output +true + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.4' )= cast(null as string) FROM t +-- !query schema +struct<(1996-09-09 12:12:12.4 = CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )> cast(null as string) FROM t +-- !query schema +struct<(1996-09-09 12:12:12.5 > CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )>= cast(null as string) FROM t +-- !query schema +struct<(1996-09-09 12:12:12.5 >= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )< cast(null as string) FROM t +-- !query schema +struct<(1996-09-09 12:12:12.5 < CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )<= cast(null as string) FROM t +-- !query schema +struct<(1996-09-09 12:12:12.5 <= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT timestamp('1996-09-09 12:12:12.5' )<> cast(null as string) FROM t +-- !query schema +struct<(NOT (1996-09-09 12:12:12.5 = CAST(NULL AS STRING))):boolean> +-- !query output +NULL + + +-- !query +SELECT ' ' = X'0020' FROM t +-- !query schema +struct<( = X'0020'):boolean> +-- !query output +false + + +-- !query +SELECT ' ' > X'001F' FROM t +-- !query schema +struct<( > X'001F'):boolean> +-- !query output +true + + +-- !query +SELECT ' ' >= X'001F' FROM t +-- !query schema +struct<( >= X'001F'):boolean> +-- !query output +true + + +-- !query +SELECT ' ' < X'001F' FROM t +-- !query schema +struct<( < X'001F'):boolean> +-- !query output +false + + +-- !query +SELECT ' ' <= X'001F' FROM t +-- !query schema +struct<( <= X'001F'):boolean> +-- !query output +false + + +-- !query +SELECT ' ' <> X'001F' FROM t +-- !query schema +struct<(NOT ( = X'001F')):boolean> +-- !query output +true + + +-- !query +SELECT cast(null as string) = X'0020' FROM t +-- !query schema +struct<(CAST(NULL AS STRING) = X'0020'):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) > X'001F' FROM t +-- !query schema +struct<(CAST(NULL AS STRING) > X'001F'):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) >= X'001F' FROM t +-- !query schema +struct<(CAST(NULL AS STRING) >= X'001F'):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) < X'001F' FROM t +-- !query schema +struct<(CAST(NULL AS STRING) < X'001F'):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) <= X'001F' FROM t +-- !query schema +struct<(CAST(NULL AS STRING) <= X'001F'):boolean> +-- !query output +NULL + + +-- !query +SELECT cast(null as string) <> X'001F' FROM t +-- !query schema +struct<(NOT (CAST(NULL AS STRING) = X'001F')):boolean> +-- !query output +NULL + + +-- !query +SELECT X'0020' = ' ' FROM t +-- !query schema +struct<(X'0020' = ):boolean> +-- !query output +false + + +-- !query +SELECT X'001F' > ' ' FROM t +-- !query schema +struct<(X'001F' > ):boolean> +-- !query output +false + + +-- !query +SELECT X'001F' >= ' ' FROM t +-- !query schema +struct<(X'001F' >= ):boolean> +-- !query output +false + + +-- !query +SELECT X'001F' < ' ' FROM t +-- !query schema +struct<(X'001F' < ):boolean> +-- !query output +true + + +-- !query +SELECT X'001F' <= ' ' FROM t +-- !query schema +struct<(X'001F' <= ):boolean> +-- !query output +true + + +-- !query +SELECT X'001F' <> ' ' FROM t +-- !query schema +struct<(NOT (X'001F' = )):boolean> +-- !query output +true + + +-- !query +SELECT X'0020' = cast(null as string) FROM t +-- !query schema +struct<(X'0020' = CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT X'001F' > cast(null as string) FROM t +-- !query schema +struct<(X'001F' > CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT X'001F' >= cast(null as string) FROM t +-- !query schema +struct<(X'001F' >= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT X'001F' < cast(null as string) FROM t +-- !query schema +struct<(X'001F' < CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT X'001F' <= cast(null as string) FROM t +-- !query schema +struct<(X'001F' <= CAST(NULL AS STRING)):boolean> +-- !query output +NULL + + +-- !query +SELECT X'001F' <> cast(null as string) FROM t +-- !query schema +struct<(NOT (X'001F' = CAST(NULL AS STRING))):boolean> +-- !query output +NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/booleanEquality.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/booleanEquality.sql.out new file mode 100644 index 000000000000..f83284e9bf6f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/booleanEquality.sql.out @@ -0,0 +1,2055 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT true = cast(1 as tinyint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(true = CAST(1 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "true = cast(1 as tinyint)" + } ] +} + + +-- !query +SELECT true = cast(1 as smallint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(true = CAST(1 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "true = cast(1 as smallint)" + } ] +} + + +-- !query +SELECT true = cast(1 as int) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(true = CAST(1 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "true = cast(1 as int)" + } ] +} + + +-- !query +SELECT true = cast(1 as bigint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(true = CAST(1 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "true = cast(1 as bigint)" + } ] +} + + +-- !query +SELECT true = cast(1 as float) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(true = CAST(1 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "true = cast(1 as float)" + } ] +} + + +-- !query +SELECT true = cast(1 as double) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(true = CAST(1 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "true = cast(1 as double)" + } ] +} + + +-- !query +SELECT true = cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(true = CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "true = cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT true = cast(1 as string) FROM t +-- !query schema +struct<(true = CAST(1 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT true = cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(true = CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "true = cast('1' as binary)" + } ] +} + + +-- !query +SELECT true = cast(1 as boolean) FROM t +-- !query schema +struct<(true = CAST(1 AS BOOLEAN)):boolean> +-- !query output +true + + +-- !query +SELECT true = cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(true = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "true = cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT true = cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(true = CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "true = cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT true <=> cast(1 as tinyint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(true <=> CAST(1 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "true <=> cast(1 as tinyint)" + } ] +} + + +-- !query +SELECT true <=> cast(1 as smallint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(true <=> CAST(1 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "true <=> cast(1 as smallint)" + } ] +} + + +-- !query +SELECT true <=> cast(1 as int) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(true <=> CAST(1 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "true <=> cast(1 as int)" + } ] +} + + +-- !query +SELECT true <=> cast(1 as bigint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(true <=> CAST(1 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "true <=> cast(1 as bigint)" + } ] +} + + +-- !query +SELECT true <=> cast(1 as float) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(true <=> CAST(1 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "true <=> cast(1 as float)" + } ] +} + + +-- !query +SELECT true <=> cast(1 as double) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(true <=> CAST(1 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "true <=> cast(1 as double)" + } ] +} + + +-- !query +SELECT true <=> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(true <=> CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "true <=> cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT true <=> cast(1 as string) FROM t +-- !query schema +struct<(true <=> CAST(1 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT true <=> cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(true <=> CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "true <=> cast('1' as binary)" + } ] +} + + +-- !query +SELECT true <=> cast(1 as boolean) FROM t +-- !query schema +struct<(true <=> CAST(1 AS BOOLEAN)):boolean> +-- !query output +true + + +-- !query +SELECT true <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(true <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "true <=> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT true <=> cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(true <=> CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "true <=> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) = true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS TINYINT) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "cast(1 as tinyint) = true" + } ] +} + + +-- !query +SELECT cast(1 as smallint) = true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast(1 as smallint) = true" + } ] +} + + +-- !query +SELECT cast(1 as int) = true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS INT) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 28, + "fragment" : "cast(1 as int) = true" + } ] +} + + +-- !query +SELECT cast(1 as bigint) = true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS BIGINT) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "cast(1 as bigint) = true" + } ] +} + + +-- !query +SELECT cast(1 as float) = true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS FLOAT) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "cast(1 as float) = true" + } ] +} + + +-- !query +SELECT cast(1 as double) = true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "cast(1 as double) = true" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) = true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "cast(1 as decimal(10, 0)) = true" + } ] +} + + +-- !query +SELECT cast(1 as string) = true FROM t +-- !query schema +struct<(CAST(1 AS STRING) = true):boolean> +-- !query output +true + + +-- !query +SELECT cast('1' as binary) = true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast('1' as binary) = true" + } ] +} + + +-- !query +SELECT cast(1 as boolean) = true FROM t +-- !query schema +struct<(CAST(1 AS BOOLEAN) = true):boolean> +-- !query output +true + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = true" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) = true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "cast('2017-12-11 09:30:00' as date) = true" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) <=> true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS TINYINT) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast(1 as tinyint) <=> true" + } ] +} + + +-- !query +SELECT cast(1 as smallint) <=> true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "cast(1 as smallint) <=> true" + } ] +} + + +-- !query +SELECT cast(1 as int) <=> true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS INT) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "cast(1 as int) <=> true" + } ] +} + + +-- !query +SELECT cast(1 as bigint) <=> true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS BIGINT) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast(1 as bigint) <=> true" + } ] +} + + +-- !query +SELECT cast(1 as float) <=> true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS FLOAT) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "cast(1 as float) <=> true" + } ] +} + + +-- !query +SELECT cast(1 as double) <=> true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast(1 as double) <=> true" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "cast(1 as decimal(10, 0)) <=> true" + } ] +} + + +-- !query +SELECT cast(1 as string) <=> true FROM t +-- !query schema +struct<(CAST(1 AS STRING) <=> true):boolean> +-- !query output +true + + +-- !query +SELECT cast('1' as binary) <=> true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "cast('1' as binary) <=> true" + } ] +} + + +-- !query +SELECT cast(1 as boolean) <=> true FROM t +-- !query schema +struct<(CAST(1 AS BOOLEAN) <=> true):boolean> +-- !query output +true + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> true" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <=> true FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> true)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> true" + } ] +} + + +-- !query +SELECT false = cast(0 as tinyint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(false = CAST(0 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "false = cast(0 as tinyint)" + } ] +} + + +-- !query +SELECT false = cast(0 as smallint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(false = CAST(0 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "false = cast(0 as smallint)" + } ] +} + + +-- !query +SELECT false = cast(0 as int) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(false = CAST(0 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "false = cast(0 as int)" + } ] +} + + +-- !query +SELECT false = cast(0 as bigint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(false = CAST(0 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "false = cast(0 as bigint)" + } ] +} + + +-- !query +SELECT false = cast(0 as float) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(false = CAST(0 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "false = cast(0 as float)" + } ] +} + + +-- !query +SELECT false = cast(0 as double) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(false = CAST(0 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "false = cast(0 as double)" + } ] +} + + +-- !query +SELECT false = cast(0 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(false = CAST(0 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "false = cast(0 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT false = cast(0 as string) FROM t +-- !query schema +struct<(false = CAST(0 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT false = cast('0' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(false = CAST(0 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "false = cast('0' as binary)" + } ] +} + + +-- !query +SELECT false = cast(0 as boolean) FROM t +-- !query schema +struct<(false = CAST(0 AS BOOLEAN)):boolean> +-- !query output +true + + +-- !query +SELECT false = cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(false = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "false = cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT false = cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(false = CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "false = cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT false <=> cast(0 as tinyint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(false <=> CAST(0 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "false <=> cast(0 as tinyint)" + } ] +} + + +-- !query +SELECT false <=> cast(0 as smallint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(false <=> CAST(0 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "false <=> cast(0 as smallint)" + } ] +} + + +-- !query +SELECT false <=> cast(0 as int) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(false <=> CAST(0 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "false <=> cast(0 as int)" + } ] +} + + +-- !query +SELECT false <=> cast(0 as bigint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(false <=> CAST(0 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "false <=> cast(0 as bigint)" + } ] +} + + +-- !query +SELECT false <=> cast(0 as float) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(false <=> CAST(0 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "false <=> cast(0 as float)" + } ] +} + + +-- !query +SELECT false <=> cast(0 as double) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(false <=> CAST(0 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "false <=> cast(0 as double)" + } ] +} + + +-- !query +SELECT false <=> cast(0 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(false <=> CAST(0 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "false <=> cast(0 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT false <=> cast(0 as string) FROM t +-- !query schema +struct<(false <=> CAST(0 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT false <=> cast('0' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(false <=> CAST(0 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "false <=> cast('0' as binary)" + } ] +} + + +-- !query +SELECT false <=> cast(0 as boolean) FROM t +-- !query schema +struct<(false <=> CAST(0 AS BOOLEAN)):boolean> +-- !query output +true + + +-- !query +SELECT false <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(false <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "false <=> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT false <=> cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(false <=> CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "false <=> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(0 as tinyint) = false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS TINYINT) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast(0 as tinyint) = false" + } ] +} + + +-- !query +SELECT cast(0 as smallint) = false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS SMALLINT) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast(0 as smallint) = false" + } ] +} + + +-- !query +SELECT cast(0 as int) = false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS INT) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 29, + "fragment" : "cast(0 as int) = false" + } ] +} + + +-- !query +SELECT cast(0 as bigint) = false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS BIGINT) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "cast(0 as bigint) = false" + } ] +} + + +-- !query +SELECT cast(0 as float) = false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS FLOAT) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "cast(0 as float) = false" + } ] +} + + +-- !query +SELECT cast(0 as double) = false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS DOUBLE) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "cast(0 as double) = false" + } ] +} + + +-- !query +SELECT cast(0 as decimal(10, 0)) = false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS DECIMAL(10,0)) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "cast(0 as decimal(10, 0)) = false" + } ] +} + + +-- !query +SELECT cast(0 as string) = false FROM t +-- !query schema +struct<(CAST(0 AS STRING) = false):boolean> +-- !query output +true + + +-- !query +SELECT cast('0' as binary) = false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS BINARY) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast('0' as binary) = false" + } ] +} + + +-- !query +SELECT cast(0 as boolean) = false FROM t +-- !query schema +struct<(CAST(0 AS BOOLEAN) = false):boolean> +-- !query output +true + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = false" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) = false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "cast('2017-12-11 09:30:00' as date) = false" + } ] +} + + +-- !query +SELECT cast(0 as tinyint) <=> false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS TINYINT) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "cast(0 as tinyint) <=> false" + } ] +} + + +-- !query +SELECT cast(0 as smallint) <=> false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS SMALLINT) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "cast(0 as smallint) <=> false" + } ] +} + + +-- !query +SELECT cast(0 as int) <=> false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS INT) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "cast(0 as int) <=> false" + } ] +} + + +-- !query +SELECT cast(0 as bigint) <=> false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS BIGINT) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast(0 as bigint) <=> false" + } ] +} + + +-- !query +SELECT cast(0 as float) <=> false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS FLOAT) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 33, + "fragment" : "cast(0 as float) <=> false" + } ] +} + + +-- !query +SELECT cast(0 as double) <=> false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS DOUBLE) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast(0 as double) <=> false" + } ] +} + + +-- !query +SELECT cast(0 as decimal(10, 0)) <=> false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS DECIMAL(10,0)) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "cast(0 as decimal(10, 0)) <=> false" + } ] +} + + +-- !query +SELECT cast(0 as string) <=> false FROM t +-- !query schema +struct<(CAST(0 AS STRING) <=> false):boolean> +-- !query output +true + + +-- !query +SELECT cast('0' as binary) <=> false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(0 AS BINARY) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "cast('0' as binary) <=> false" + } ] +} + + +-- !query +SELECT cast(0 as boolean) <=> false FROM t +-- !query schema +struct<(CAST(0 AS BOOLEAN) <=> false):boolean> +-- !query output +true + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> false" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <=> false FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> false" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out new file mode 100644 index 000000000000..252d8c884ca7 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out @@ -0,0 +1,2249 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as tinyint) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as smallint) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as int) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as bigint) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as float) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as double) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as string) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2' as binary) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as boolean) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"TIMESTAMP\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"DATE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2017-12-11 09:30:00 AS DATE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 90, + "fragment" : "CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as tinyint) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as smallint) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as int) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as bigint) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as float) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as double) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as string) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2' as binary) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "CASE WHEN true THEN cast(1 as smallint) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as boolean) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"TIMESTAMP\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 98, + "fragment" : "CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"DATE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2017-12-11 09:30:00 AS DATE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 91, + "fragment" : "CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as tinyint) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as smallint) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as int) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as bigint) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as float) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as double) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as string) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2' as binary) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "CASE WHEN true THEN cast(1 as int) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as boolean) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "CASE WHEN true THEN cast(1 as int) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"TIMESTAMP\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 93, + "fragment" : "CASE WHEN true THEN cast(1 as int) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"DATE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2017-12-11 09:30:00 AS DATE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 86, + "fragment" : "CASE WHEN true THEN cast(1 as int) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as tinyint) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as smallint) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as int) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as bigint) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as float) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as double) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as string) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2' as binary) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "CASE WHEN true THEN cast(1 as bigint) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as boolean) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"TIMESTAMP\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 96, + "fragment" : "CASE WHEN true THEN cast(1 as bigint) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"DATE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2017-12-11 09:30:00 AS DATE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 89, + "fragment" : "CASE WHEN true THEN cast(1 as bigint) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as tinyint) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as smallint) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as int) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as bigint) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as float) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as double) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as string) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2' as binary) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "CASE WHEN true THEN cast(1 as float) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as boolean) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "CASE WHEN true THEN cast(1 as float) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"TIMESTAMP\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 95, + "fragment" : "CASE WHEN true THEN cast(1 as float) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"DATE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2017-12-11 09:30:00 AS DATE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 88, + "fragment" : "CASE WHEN true THEN cast(1 as float) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as tinyint) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as smallint) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as int) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as bigint) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as float) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as double) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as string) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2' as binary) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "CASE WHEN true THEN cast(1 as double) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as boolean) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "CASE WHEN true THEN cast(1 as double) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"TIMESTAMP\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 96, + "fragment" : "CASE WHEN true THEN cast(1 as double) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"DATE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2017-12-11 09:30:00 AS DATE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 89, + "fragment" : "CASE WHEN true THEN cast(1 as double) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as tinyint) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as smallint) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as int) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as bigint) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as float) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as double) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as string) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2' as binary) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 81, + "fragment" : "CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as boolean) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"TIMESTAMP\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 104, + "fragment" : "CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"DATE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2017-12-11 09:30:00 AS DATE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as tinyint) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as smallint) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as int) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as bigint) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as float) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as double) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as string) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2' as binary) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as boolean) END FROM t +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 96, + "fragment" : "CASE WHEN true THEN cast(1 as string) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 89, + "fragment" : "CASE WHEN true THEN cast(1 as string) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as tinyint) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"TINYINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS TINYINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as tinyint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as smallint) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"SMALLINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS SMALLINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as smallint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as int) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"INT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS INT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as int) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as bigint) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BIGINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS BIGINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as bigint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as float) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"FLOAT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS FLOAT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as float) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as double) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"DOUBLE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS DOUBLE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as double) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"DECIMAL(10,0)\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS DECIMAL(10,0)) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 81, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as decimal(10, 0)) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as string) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast('2' as binary) END FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as boolean) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"TIMESTAMP\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 98, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"DATE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BINARY) ELSE CAST(2017-12-11 09:30:00 AS DATE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 91, + "fragment" : "CASE WHEN true THEN cast('1' as binary) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as tinyint) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"TINYINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS TINYINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as tinyint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as smallint) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"SMALLINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS SMALLINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as smallint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as int) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"INT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS INT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as int) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as bigint) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BIGINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS BIGINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as bigint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as float) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"FLOAT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS FLOAT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as float) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as double) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"DOUBLE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS DOUBLE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as double) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"DECIMAL(10,0)\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS DECIMAL(10,0)) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as decimal(10, 0)) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as string) END FROM t +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2' as binary) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as boolean) END FROM t +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"TIMESTAMP\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"DATE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2017-12-11 09:30:00 AS DATE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 90, + "fragment" : "CASE WHEN true THEN cast(1 as boolean) ELSE cast('2017-12-11 09:30:00' as date) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as tinyint) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TINYINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS TINYINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as tinyint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as smallint) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"SMALLINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS SMALLINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 98, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as smallint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as int) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"INT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS INT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 93, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as int) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as bigint) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"BIGINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS BIGINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 96, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as bigint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as float) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"FLOAT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS FLOAT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 95, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as float) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as double) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"DOUBLE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS DOUBLE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 96, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as double) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"DECIMAL(10,0)\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS DECIMAL(10,0)) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 104, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as decimal(10, 0)) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as string) END FROM t +-- !query schema +struct +-- !query output +2017-12-12 09:30:00 + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast('2' as binary) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 98, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as boolean) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query schema +struct +-- !query output +2017-12-12 09:30:00 + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query schema +struct +-- !query output +2017-12-12 09:30:00 + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as tinyint) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"TINYINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS TINYINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 90, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as tinyint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as smallint) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"SMALLINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS SMALLINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 91, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as smallint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as int) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"INT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS INT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 86, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as int) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as bigint) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"BIGINT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS BIGINT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 89, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as bigint) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as float) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"FLOAT\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS FLOAT) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 88, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as float) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as double) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DOUBLE\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS DOUBLE) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 89, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as double) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as decimal(10, 0)) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DECIMAL(10,0)\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS DECIMAL(10,0)) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 97, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as decimal(10, 0)) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as string) END FROM t +-- !query schema +struct +-- !query output +2017-12-12 + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast('2' as binary) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"BINARY\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS BINARY) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 91, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast('2' as binary) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as boolean) END FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"BOOLEAN\"]", + "functionName" : "`casewhen`", + "sqlExpr" : "\"CASE WHEN true THEN CAST(2017-12-12 09:30:00 AS DATE) ELSE CAST(2 AS BOOLEAN) END\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 90, + "fragment" : "CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as boolean) END" + } ] +} + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast('2017-12-11 09:30:00.0' as timestamp) END FROM t +-- !query schema +struct +-- !query output +2017-12-12 00:00:00 + + +-- !query +SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast('2017-12-11 09:30:00' as date) END FROM t +-- !query schema +struct +-- !query output +2017-12-12 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/concat.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/concat.sql.out new file mode 100644 index 000000000000..3e28c696a259 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/concat.sql.out @@ -0,0 +1,333 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT (col1 || col2 || col3) col +FROM ( + SELECT + id col1, + string(id + 1) col2, + encode(string(id + 2), 'utf-8') col3 + FROM range(10) +) +-- !query schema +struct +-- !query output +012 +123 +234 +345 +456 +567 +678 +789 +8910 +91011 + + +-- !query +SELECT ((col1 || col2) || (col3 || col4) || col5) col +FROM ( + SELECT + 'prefix_' col1, + id col2, + string(id + 1) col3, + encode(string(id + 2), 'utf-8') col4, + CAST(id AS DOUBLE) col5 + FROM range(10) +) +-- !query schema +struct +-- !query output +prefix_0120.0 +prefix_1231.0 +prefix_2342.0 +prefix_3453.0 +prefix_4564.0 +prefix_5675.0 +prefix_6786.0 +prefix_7897.0 +prefix_89108.0 +prefix_910119.0 + + +-- !query +SELECT ((col1 || col2) || (col3 || col4)) col +FROM ( + SELECT + string(id) col1, + string(id + 1) col2, + encode(string(id + 2), 'utf-8') col3, + encode(string(id + 3), 'utf-8') col4 + FROM range(10) +) +-- !query schema +struct +-- !query output +0123 +1234 +2345 +3456 +4567 +5678 +6789 +78910 +891011 +9101112 + + +-- !query +set spark.sql.function.concatBinaryAsString=true +-- !query schema +struct +-- !query output +spark.sql.function.concatBinaryAsString true + + +-- !query +SELECT (col1 || col2) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2 + FROM range(10) +) +-- !query schema +struct +-- !query output +01 +12 +23 +34 +45 +56 +67 +78 +89 +910 + + +-- !query +SELECT (col1 || col2 || col3 || col4) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2, + encode(string(id + 2), 'utf-8') col3, + encode(string(id + 3), 'utf-8') col4 + FROM range(10) +) +-- !query schema +struct +-- !query output +0123 +1234 +2345 +3456 +4567 +5678 +6789 +78910 +891011 +9101112 + + +-- !query +SELECT ((col1 || col2) || (col3 || col4)) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2, + encode(string(id + 2), 'utf-8') col3, + encode(string(id + 3), 'utf-8') col4 + FROM range(10) +) +-- !query schema +struct +-- !query output +0123 +1234 +2345 +3456 +4567 +5678 +6789 +78910 +891011 +9101112 + + +-- !query +set spark.sql.function.concatBinaryAsString=false +-- !query schema +struct +-- !query output +spark.sql.function.concatBinaryAsString false + + +-- !query +SELECT (col1 || col2) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2 + FROM range(10) +) +-- !query schema +struct +-- !query output +01 +12 +23 +34 +45 +56 +67 +78 +89 +910 + + +-- !query +SELECT (col1 || col2 || col3 || col4) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2, + encode(string(id + 2), 'utf-8') col3, + encode(string(id + 3), 'utf-8') col4 + FROM range(10) +) +-- !query schema +struct +-- !query output +0123 +1234 +2345 +3456 +4567 +5678 +6789 +78910 +891011 +9101112 + + +-- !query +SELECT ((col1 || col2) || (col3 || col4)) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2, + encode(string(id + 2), 'utf-8') col3, + encode(string(id + 3), 'utf-8') col4 + FROM range(10) +) +-- !query schema +struct +-- !query output +0123 +1234 +2345 +3456 +4567 +5678 +6789 +78910 +891011 +9101112 + + +-- !query +CREATE TEMPORARY VIEW various_arrays AS SELECT * FROM VALUES ( + array(true, false), array(true), + array(2Y, 1Y), array(3Y, 4Y), + array(2S, 1S), array(3S, 4S), + array(2, 1), array(3, 4), + array(2L, 1L), array(3L, 4L), + array(9223372036854775809, 9223372036854775808), array(9223372036854775808, 9223372036854775809), + array(2.0D, 1.0D), array(3.0D, 4.0D), + array(float(2.0), float(1.0)), array(float(3.0), float(4.0)), + array(date '2016-03-14', date '2016-03-13'), array(date '2016-03-12', date '2016-03-11'), + array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), + array(timestamp '2016-11-11 20:54:00.000'), + array('a', 'b'), array('c', 'd'), + array(array('a', 'b'), array('c', 'd')), array(array('e'), array('f')), + array(struct('a', 1), struct('b', 2)), array(struct('c', 3), struct('d', 4)), + array(map('a', 1), map('b', 2)), array(map('c', 3), map('d', 4)) +) AS various_arrays( + boolean_array1, boolean_array2, + tinyint_array1, tinyint_array2, + smallint_array1, smallint_array2, + int_array1, int_array2, + bigint_array1, bigint_array2, + decimal_array1, decimal_array2, + double_array1, double_array2, + float_array1, float_array2, + date_array1, data_array2, + timestamp_array1, timestamp_array2, + string_array1, string_array2, + array_array1, array_array2, + struct_array1, struct_array2, + map_array1, map_array2 +) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + (boolean_array1 || boolean_array2) boolean_array, + (tinyint_array1 || tinyint_array2) tinyint_array, + (smallint_array1 || smallint_array2) smallint_array, + (int_array1 || int_array2) int_array, + (bigint_array1 || bigint_array2) bigint_array, + (decimal_array1 || decimal_array2) decimal_array, + (double_array1 || double_array2) double_array, + (float_array1 || float_array2) float_array, + (date_array1 || data_array2) data_array, + (timestamp_array1 || timestamp_array2) timestamp_array, + (string_array1 || string_array2) string_array, + (array_array1 || array_array2) array_array, + (struct_array1 || struct_array2) struct_array, + (map_array1 || map_array2) map_array +FROM various_arrays +-- !query schema +struct,tinyint_array:array,smallint_array:array,int_array:array,bigint_array:array,decimal_array:array,double_array:array,float_array:array,data_array:array,timestamp_array:array,string_array:array,array_array:array>,struct_array:array>,map_array:array>> +-- !query output +[true,false,true] [2,1,3,4] [2,1,3,4] [2,1,3,4] [2,1,3,4] [9223372036854775809,9223372036854775808,9223372036854775808,9223372036854775809] [2.0,1.0,3.0,4.0] [2.0,1.0,3.0,4.0] [2016-03-14,2016-03-13,2016-03-12,2016-03-11] [2016-11-15 20:54:00,2016-11-12 20:54:00,2016-11-11 20:54:00] ["a","b","c","d"] [["a","b"],["c","d"],["e"],["f"]] [{"col1":"a","col2":1},{"col1":"b","col2":2},{"col1":"c","col2":3},{"col1":"d","col2":4}] [{"a":1},{"b":2},{"c":3},{"d":4}] + + +-- !query +SELECT + (tinyint_array1 || smallint_array2) ts_array, + (smallint_array1 || int_array2) si_array, + (int_array1 || bigint_array2) ib_array, + (bigint_array1 || decimal_array2) bd_array, + (decimal_array1 || double_array2) dd_array, + (double_array1 || float_array2) df_array, + (string_array1 || data_array2) std_array, + (timestamp_array1 || string_array2) tst_array, + (string_array1 || int_array2) sti_array +FROM various_arrays +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 295, + "stopIndex" : 322, + "fragment" : "string_array1 || data_array2" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out new file mode 100644 index 000000000000..10cceabca052 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out @@ -0,0 +1,805 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +select cast(1 as tinyint) + interval 2 day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS TINYINT)\"", + "inputType" : "\"TINYINT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS TINYINT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "cast(1 as tinyint) + interval 2 day" + } ] +} + + +-- !query +select cast(1 as smallint) + interval 2 day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS SMALLINT)\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS SMALLINT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "cast(1 as smallint) + interval 2 day" + } ] +} + + +-- !query +select cast(1 as int) + interval 2 day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS INT)\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS INT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "cast(1 as int) + interval 2 day" + } ] +} + + +-- !query +select cast(1 as bigint) + interval 2 day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS BIGINT)\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS BIGINT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "cast(1 as bigint) + interval 2 day" + } ] +} + + +-- !query +select cast(1 as float) + interval 2 day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS FLOAT)\"", + "inputType" : "\"FLOAT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS FLOAT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "cast(1 as float) + interval 2 day" + } ] +} + + +-- !query +select cast(1 as double) + interval 2 day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DOUBLE)\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS DOUBLE) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "cast(1 as double) + interval 2 day" + } ] +} + + +-- !query +select cast(1 as decimal(10, 0)) + interval 2 day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(10,0))\"", + "inputType" : "\"DECIMAL(10,0)\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS DECIMAL(10,0)) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "cast(1 as decimal(10, 0)) + interval 2 day" + } ] +} + + +-- !query +select cast('2017-12-11' as string) + interval 2 day +-- !query schema +struct +-- !query output +2017-12-13 00:00:00 + + +-- !query +select cast('2017-12-11 09:30:00' as string) + interval 2 day +-- !query schema +struct +-- !query output +2017-12-13 09:30:00 + + +-- !query +select cast('1' as binary) + interval 2 day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS BINARY)\"", + "inputType" : "\"BINARY\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS BINARY) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "cast('1' as binary) + interval 2 day" + } ] +} + + +-- !query +select cast(1 as boolean) + interval 2 day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS BOOLEAN)\"", + "inputType" : "\"BOOLEAN\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS BOOLEAN) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "cast(1 as boolean) + interval 2 day" + } ] +} + + +-- !query +select cast('2017-12-11 09:30:00.0' as timestamp) + interval 2 day +-- !query schema +struct +-- !query output +2017-12-13 09:30:00 + + +-- !query +select cast('2017-12-11 09:30:00' as date) + interval 2 day +-- !query schema +struct +-- !query output +2017-12-13 + + +-- !query +select interval 2 day + cast(1 as tinyint) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS TINYINT)\"", + "inputType" : "\"TINYINT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS TINYINT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "interval 2 day + cast(1 as tinyint)" + } ] +} + + +-- !query +select interval 2 day + cast(1 as smallint) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS SMALLINT)\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS SMALLINT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "interval 2 day + cast(1 as smallint)" + } ] +} + + +-- !query +select interval 2 day + cast(1 as int) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS INT)\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS INT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "interval 2 day + cast(1 as int)" + } ] +} + + +-- !query +select interval 2 day + cast(1 as bigint) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS BIGINT)\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS BIGINT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "interval 2 day + cast(1 as bigint)" + } ] +} + + +-- !query +select interval 2 day + cast(1 as float) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS FLOAT)\"", + "inputType" : "\"FLOAT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS FLOAT) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "interval 2 day + cast(1 as float)" + } ] +} + + +-- !query +select interval 2 day + cast(1 as double) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DOUBLE)\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS DOUBLE) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "interval 2 day + cast(1 as double)" + } ] +} + + +-- !query +select interval 2 day + cast(1 as decimal(10, 0)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(10,0))\"", + "inputType" : "\"DECIMAL(10,0)\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS DECIMAL(10,0)) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "interval 2 day + cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +select interval 2 day + cast('2017-12-11' as string) +-- !query schema +struct +-- !query output +2017-12-13 00:00:00 + + +-- !query +select interval 2 day + cast('2017-12-11 09:30:00' as string) +-- !query schema +struct +-- !query output +2017-12-13 09:30:00 + + +-- !query +select interval 2 day + cast('1' as binary) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS BINARY)\"", + "inputType" : "\"BINARY\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS BINARY) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "interval 2 day + cast('1' as binary)" + } ] +} + + +-- !query +select interval 2 day + cast(1 as boolean) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS BOOLEAN)\"", + "inputType" : "\"BOOLEAN\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS BOOLEAN) + INTERVAL '2' DAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "interval 2 day + cast(1 as boolean)" + } ] +} + + +-- !query +select interval 2 day + cast('2017-12-11 09:30:00.0' as timestamp) +-- !query schema +struct +-- !query output +2017-12-13 09:30:00 + + +-- !query +select interval 2 day + cast('2017-12-11 09:30:00' as date) +-- !query schema +struct +-- !query output +2017-12-13 + + +-- !query +select cast(1 as tinyint) - interval 2 day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS TINYINT)\"", + "inputType" : "\"TINYINT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS TINYINT) + (- INTERVAL '2' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "cast(1 as tinyint) - interval 2 day" + } ] +} + + +-- !query +select cast(1 as smallint) - interval 2 day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS SMALLINT)\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS SMALLINT) + (- INTERVAL '2' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "cast(1 as smallint) - interval 2 day" + } ] +} + + +-- !query +select cast(1 as int) - interval 2 day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS INT)\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS INT) + (- INTERVAL '2' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 38, + "fragment" : "cast(1 as int) - interval 2 day" + } ] +} + + +-- !query +select cast(1 as bigint) - interval 2 day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS BIGINT)\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS BIGINT) + (- INTERVAL '2' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "cast(1 as bigint) - interval 2 day" + } ] +} + + +-- !query +select cast(1 as float) - interval 2 day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS FLOAT)\"", + "inputType" : "\"FLOAT\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS FLOAT) + (- INTERVAL '2' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 40, + "fragment" : "cast(1 as float) - interval 2 day" + } ] +} + + +-- !query +select cast(1 as double) - interval 2 day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DOUBLE)\"", + "inputType" : "\"DOUBLE\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS DOUBLE) + (- INTERVAL '2' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 41, + "fragment" : "cast(1 as double) - interval 2 day" + } ] +} + + +-- !query +select cast(1 as decimal(10, 0)) - interval 2 day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(10,0))\"", + "inputType" : "\"DECIMAL(10,0)\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS DECIMAL(10,0)) + (- INTERVAL '2' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "cast(1 as decimal(10, 0)) - interval 2 day" + } ] +} + + +-- !query +select cast('2017-12-11' as string) - interval 2 day +-- !query schema +struct +-- !query output +2017-12-09 00:00:00 + + +-- !query +select cast('2017-12-11 09:30:00' as string) - interval 2 day +-- !query schema +struct +-- !query output +2017-12-09 09:30:00 + + +-- !query +select cast('1' as binary) - interval 2 day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS BINARY)\"", + "inputType" : "\"BINARY\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS BINARY) + (- INTERVAL '2' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "cast('1' as binary) - interval 2 day" + } ] +} + + +-- !query +select cast(1 as boolean) - interval 2 day +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS BOOLEAN)\"", + "inputType" : "\"BOOLEAN\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"CAST(1 AS BOOLEAN) + (- INTERVAL '2' DAY)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "cast(1 as boolean) - interval 2 day" + } ] +} + + +-- !query +select cast('2017-12-11 09:30:00.0' as timestamp) - interval 2 day +-- !query schema +struct +-- !query output +2017-12-09 09:30:00 + + +-- !query +select cast('2017-12-11 09:30:00' as date) - interval 2 day +-- !query schema +struct +-- !query output +2017-12-09 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out new file mode 100644 index 000000000000..54e26851ba57 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out @@ -0,0 +1,15031 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT cast(1 as tinyint) + cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) + CAST(1 AS DECIMAL(3,0))):decimal(4,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as tinyint) + cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) + CAST(1 AS DECIMAL(5,0))):decimal(6,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as tinyint) + cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) + CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as tinyint) + cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) + CAST(1 AS DECIMAL(20,0))):decimal(21,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as smallint) + cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) + CAST(1 AS DECIMAL(3,0))):decimal(6,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as smallint) + cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) + CAST(1 AS DECIMAL(5,0))):decimal(6,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as smallint) + cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) + CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as smallint) + cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) + CAST(1 AS DECIMAL(20,0))):decimal(21,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as int) + cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) + CAST(1 AS DECIMAL(3,0))):decimal(11,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as int) + cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) + CAST(1 AS DECIMAL(5,0))):decimal(11,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as int) + cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) + CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as int) + cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) + CAST(1 AS DECIMAL(20,0))):decimal(21,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as bigint) + cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) + CAST(1 AS DECIMAL(3,0))):decimal(21,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as bigint) + cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) + CAST(1 AS DECIMAL(5,0))):decimal(21,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as bigint) + cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) + CAST(1 AS DECIMAL(10,0))):decimal(21,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as bigint) + cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) + CAST(1 AS DECIMAL(20,0))):decimal(21,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as float) + cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) + CAST(1 AS DECIMAL(3,0))):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as float) + cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) + CAST(1 AS DECIMAL(5,0))):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as float) + cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) + CAST(1 AS DECIMAL(10,0))):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as float) + cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) + CAST(1 AS DECIMAL(20,0))):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as double) + cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) + CAST(1 AS DECIMAL(3,0))):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as double) + cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) + CAST(1 AS DECIMAL(5,0))):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as double) + cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) + CAST(1 AS DECIMAL(10,0))):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as double) + cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) + CAST(1 AS DECIMAL(20,0))):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS DECIMAL(3,0))):decimal(11,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS DECIMAL(5,0))):decimal(11,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS DECIMAL(20,0))):decimal(21,0)> +-- !query output +2 + + +-- !query +SELECT cast('1' as binary) + cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) + cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) + cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) + cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) + cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) + cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) + cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) + CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) + cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) + cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(3,0))\"", + "inputType" : "\"DECIMAL(3,0)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(5,0))\"", + "inputType" : "\"DECIMAL(5,0)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(10,0))\"", + "inputType" : "\"DECIMAL(10,0)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(20,0))\"", + "inputType" : "\"DECIMAL(20,0)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) + cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) + cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS TINYINT)):decimal(4,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(5, 0)) + cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS TINYINT)):decimal(6,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS TINYINT)):decimal(11,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(20, 0)) + cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS TINYINT)):decimal(21,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(3, 0)) + cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS SMALLINT)):decimal(6,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(5, 0)) + cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS SMALLINT)):decimal(6,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS SMALLINT)):decimal(11,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(20, 0)) + cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS SMALLINT)):decimal(21,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(3, 0)) + cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS INT)):decimal(11,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(5, 0)) + cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS INT)):decimal(11,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS INT)):decimal(11,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(20, 0)) + cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS INT)):decimal(21,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(3, 0)) + cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS BIGINT)):decimal(21,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(5, 0)) + cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS BIGINT)):decimal(21,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS BIGINT)):decimal(21,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(20, 0)) + cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS BIGINT)):decimal(21,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(3, 0)) + cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS FLOAT)):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) + cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS FLOAT)):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS FLOAT)):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) + cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS FLOAT)):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) + cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS DOUBLE)):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) + cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS DOUBLE)):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS DOUBLE)):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) + cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS DOUBLE)):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) + cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(5, 0)) + cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(20, 0)) + cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS DECIMAL(10,0))):decimal(21,0)> +-- !query output +2 + + +-- !query +SELECT cast(1 as decimal(3, 0)) + cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS STRING)):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) + cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS STRING)):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS STRING)):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) + cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS STRING)):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) + cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) + cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) + cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) + cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) + cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) + cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) + cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) + cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) + CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(3, 0)) + cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) + cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) + CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(5, 0)) + cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) + CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) + cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) + cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) + CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(20, 0)) + cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(3,0))\"", + "inputType" : "\"DECIMAL(3,0)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) + cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(5,0))\"", + "inputType" : "\"DECIMAL(5,0)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) + cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(10,0))\"", + "inputType" : "\"DECIMAL(10,0)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) + cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(20,0))\"", + "inputType" : "\"DECIMAL(20,0)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) + cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) - cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) - CAST(1 AS DECIMAL(3,0))):decimal(4,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as tinyint) - cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) - CAST(1 AS DECIMAL(5,0))):decimal(6,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as tinyint) - cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) - CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as tinyint) - cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) - CAST(1 AS DECIMAL(20,0))):decimal(21,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as smallint) - cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) - CAST(1 AS DECIMAL(3,0))):decimal(6,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as smallint) - cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) - CAST(1 AS DECIMAL(5,0))):decimal(6,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as smallint) - cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) - CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as smallint) - cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) - CAST(1 AS DECIMAL(20,0))):decimal(21,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as int) - cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) - CAST(1 AS DECIMAL(3,0))):decimal(11,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as int) - cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) - CAST(1 AS DECIMAL(5,0))):decimal(11,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as int) - cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) - CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as int) - cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) - CAST(1 AS DECIMAL(20,0))):decimal(21,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as bigint) - cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) - CAST(1 AS DECIMAL(3,0))):decimal(21,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as bigint) - cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) - CAST(1 AS DECIMAL(5,0))):decimal(21,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as bigint) - cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) - CAST(1 AS DECIMAL(10,0))):decimal(21,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as bigint) - cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) - CAST(1 AS DECIMAL(20,0))):decimal(21,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as float) - cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) - CAST(1 AS DECIMAL(3,0))):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as float) - cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) - CAST(1 AS DECIMAL(5,0))):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as float) - cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) - CAST(1 AS DECIMAL(10,0))):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as float) - cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) - CAST(1 AS DECIMAL(20,0))):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as double) - cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) - CAST(1 AS DECIMAL(3,0))):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as double) - cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) - CAST(1 AS DECIMAL(5,0))):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as double) - cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) - CAST(1 AS DECIMAL(10,0))):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as double) - cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) - CAST(1 AS DECIMAL(20,0))):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS DECIMAL(3,0))):decimal(11,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS DECIMAL(5,0))):decimal(11,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS DECIMAL(20,0))):decimal(21,0)> +-- !query output +0 + + +-- !query +SELECT cast('1' as binary) - cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) - cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) - cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) - cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) - cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) - cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) - cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) - CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) - cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(3,0))\"", + "inputType" : "\"DECIMAL(3,0)\"", + "paramIndex" : "second", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) - CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(5,0))\"", + "inputType" : "\"DECIMAL(5,0)\"", + "paramIndex" : "second", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) - CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(10,0))\"", + "inputType" : "\"DECIMAL(10,0)\"", + "paramIndex" : "second", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) - CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(20,0))\"", + "inputType" : "\"DECIMAL(20,0)\"", + "paramIndex" : "second", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) - CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) - cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(3,0))\"", + "inputType" : "\"DECIMAL(3,0)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(5,0))\"", + "inputType" : "\"DECIMAL(5,0)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(10,0))\"", + "inputType" : "\"DECIMAL(10,0)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(20,0))\"", + "inputType" : "\"DECIMAL(20,0)\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_sub(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) - cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS TINYINT)):decimal(4,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) - cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS TINYINT)):decimal(6,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS TINYINT)):decimal(11,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) - cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS TINYINT)):decimal(21,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) - cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS SMALLINT)):decimal(6,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) - cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS SMALLINT)):decimal(6,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS SMALLINT)):decimal(11,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) - cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS SMALLINT)):decimal(21,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) - cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS INT)):decimal(11,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) - cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS INT)):decimal(11,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS INT)):decimal(11,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) - cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS INT)):decimal(21,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) - cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS BIGINT)):decimal(21,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) - cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS BIGINT)):decimal(21,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS BIGINT)):decimal(21,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) - cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS BIGINT)):decimal(21,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) - cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS FLOAT)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) - cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS FLOAT)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS FLOAT)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) - cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS FLOAT)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) - cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS DOUBLE)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) - cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS DOUBLE)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS DOUBLE)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) - cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS DOUBLE)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) - cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) - cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS DECIMAL(10,0))):decimal(11,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) - cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS DECIMAL(10,0))):decimal(21,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) - cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS STRING)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) - cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS STRING)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS STRING)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) - cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS STRING)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) - cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) - cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) - cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) - cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) - cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) - cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) - cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) - cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) - CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(3, 0)) - cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) - cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) - CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(5, 0)) - cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) - CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) - cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) - cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) - CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(20, 0)) - cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(3,0))\"", + "inputType" : "\"DECIMAL(3,0)\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) - CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(5,0))\"", + "inputType" : "\"DECIMAL(5,0)\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) - CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(10,0))\"", + "inputType" : "\"DECIMAL(10,0)\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) - CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(20,0))\"", + "inputType" : "\"DECIMAL(20,0)\"", + "paramIndex" : "first", + "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) - CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(3,0))\"", + "inputType" : "\"DECIMAL(3,0)\"", + "paramIndex" : "first", + "requiredType" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) - CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) - cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(5,0))\"", + "inputType" : "\"DECIMAL(5,0)\"", + "paramIndex" : "first", + "requiredType" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) - CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) - cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(10,0))\"", + "inputType" : "\"DECIMAL(10,0)\"", + "paramIndex" : "first", + "requiredType" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) - CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) - cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(1 AS DECIMAL(20,0))\"", + "inputType" : "\"DECIMAL(20,0)\"", + "paramIndex" : "first", + "requiredType" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) - CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) - cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) * cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) * CAST(1 AS DECIMAL(3,0))):decimal(7,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as tinyint) * cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) * CAST(1 AS DECIMAL(5,0))):decimal(9,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as tinyint) * cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) * CAST(1 AS DECIMAL(10,0))):decimal(14,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as tinyint) * cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) * CAST(1 AS DECIMAL(20,0))):decimal(24,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as smallint) * cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) * CAST(1 AS DECIMAL(3,0))):decimal(9,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as smallint) * cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) * CAST(1 AS DECIMAL(5,0))):decimal(11,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as smallint) * cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) * CAST(1 AS DECIMAL(10,0))):decimal(16,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as smallint) * cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) * CAST(1 AS DECIMAL(20,0))):decimal(26,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as int) * cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) * CAST(1 AS DECIMAL(3,0))):decimal(14,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as int) * cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) * CAST(1 AS DECIMAL(5,0))):decimal(16,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as int) * cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) * CAST(1 AS DECIMAL(10,0))):decimal(21,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as int) * cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) * CAST(1 AS DECIMAL(20,0))):decimal(31,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as bigint) * cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) * CAST(1 AS DECIMAL(3,0))):decimal(24,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as bigint) * cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) * CAST(1 AS DECIMAL(5,0))):decimal(26,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as bigint) * cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) * CAST(1 AS DECIMAL(10,0))):decimal(31,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as bigint) * cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) * CAST(1 AS DECIMAL(20,0))):decimal(38,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as float) * cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) * CAST(1 AS DECIMAL(3,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as float) * cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) * CAST(1 AS DECIMAL(5,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as float) * cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) * CAST(1 AS DECIMAL(10,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as float) * cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) * CAST(1 AS DECIMAL(20,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as double) * cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) * CAST(1 AS DECIMAL(3,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as double) * cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) * CAST(1 AS DECIMAL(5,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as double) * cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) * CAST(1 AS DECIMAL(10,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as double) * cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) * CAST(1 AS DECIMAL(20,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS DECIMAL(3,0))):decimal(14,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS DECIMAL(5,0))):decimal(16,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS DECIMAL(10,0))):decimal(21,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS DECIMAL(20,0))):decimal(31,0)> +-- !query output +1 + + +-- !query +SELECT cast('1' as binary) * cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) * cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) * cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) * cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) * cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) * cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) * cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) * CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) * cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00.0 AS TIMESTAMP) * CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017*12*11 09:30:00.0' as timestamp) * cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017*12*11 09:30:00 AS DATE) * CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017*12*11 09:30:00' as date) * cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) * cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS TINYINT)):decimal(7,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(5, 0)) * cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS TINYINT)):decimal(9,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS TINYINT)):decimal(14,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(20, 0)) * cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS TINYINT)):decimal(24,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(3, 0)) * cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS SMALLINT)):decimal(9,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(5, 0)) * cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS SMALLINT)):decimal(11,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS SMALLINT)):decimal(16,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(20, 0)) * cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS SMALLINT)):decimal(26,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(3, 0)) * cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS INT)):decimal(14,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(5, 0)) * cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS INT)):decimal(16,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS INT)):decimal(21,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(20, 0)) * cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS INT)):decimal(31,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(3, 0)) * cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS BIGINT)):decimal(24,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(5, 0)) * cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS BIGINT)):decimal(26,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS BIGINT)):decimal(31,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(20, 0)) * cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS BIGINT)):decimal(38,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(3, 0)) * cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS FLOAT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) * cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS FLOAT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS FLOAT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) * cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS FLOAT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) * cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS DOUBLE)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) * cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS DOUBLE)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS DOUBLE)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) * cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS DOUBLE)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) * cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS DECIMAL(10,0))):decimal(14,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(5, 0)) * cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS DECIMAL(10,0))):decimal(16,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS DECIMAL(10,0))):decimal(21,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(20, 0)) * cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS DECIMAL(10,0))):decimal(31,0)> +-- !query output +1 + + +-- !query +SELECT cast(1 as decimal(3, 0)) * cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS STRING)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) * cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS STRING)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS STRING)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) * cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS STRING)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) * cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) * cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) * cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) * cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) * cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) * cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) * cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) * cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(3, 0)) * cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) * cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(5, 0)) * cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) * cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) * cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(20, 0)) * cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(2017*12*11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) * cast('2017*12*11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) * cast('2017*12*11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) * cast('2017*12*11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) * CAST(2017*12*11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) * cast('2017*12*11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) / cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) / CAST(1 AS DECIMAL(3,0))):decimal(9,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as tinyint) / cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) / CAST(1 AS DECIMAL(5,0))):decimal(9,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) / CAST(1 AS DECIMAL(10,0))):decimal(14,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as tinyint) / cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) / CAST(1 AS DECIMAL(20,0))):decimal(24,21)> +-- !query output +1.000000000000000000000 + + +-- !query +SELECT cast(1 as smallint) / cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) / CAST(1 AS DECIMAL(3,0))):decimal(11,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as smallint) / cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) / CAST(1 AS DECIMAL(5,0))):decimal(11,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as smallint) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) / CAST(1 AS DECIMAL(10,0))):decimal(16,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as smallint) / cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) / CAST(1 AS DECIMAL(20,0))):decimal(26,21)> +-- !query output +1.000000000000000000000 + + +-- !query +SELECT cast(1 as int) / cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) / CAST(1 AS DECIMAL(3,0))):decimal(16,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as int) / cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) / CAST(1 AS DECIMAL(5,0))):decimal(16,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as int) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as int) / cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) / CAST(1 AS DECIMAL(20,0))):decimal(31,21)> +-- !query output +1.000000000000000000000 + + +-- !query +SELECT cast(1 as bigint) / cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) / CAST(1 AS DECIMAL(3,0))):decimal(26,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as bigint) / cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) / CAST(1 AS DECIMAL(5,0))):decimal(26,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as bigint) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) / CAST(1 AS DECIMAL(10,0))):decimal(31,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as bigint) / cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) / CAST(1 AS DECIMAL(20,0))):decimal(38,18)> +-- !query output +1.000000000000000000 + + +-- !query +SELECT cast(1 as float) / cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) / CAST(1 AS DECIMAL(3,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as float) / cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) / CAST(1 AS DECIMAL(5,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as float) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) / CAST(1 AS DECIMAL(10,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as float) / cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) / CAST(1 AS DECIMAL(20,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as double) / cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) / CAST(1 AS DECIMAL(3,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as double) / cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) / CAST(1 AS DECIMAL(5,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as double) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) / CAST(1 AS DECIMAL(10,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as double) / cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) / CAST(1 AS DECIMAL(20,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(3,0))):decimal(16,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(5,0))):decimal(16,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(20,0))):decimal(31,21)> +-- !query output +1.000000000000000000000 + + +-- !query +SELECT cast('1' as binary) / cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) / cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) / cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) / cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) / cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017/12/11 09:30:00.0' as timestamp) / cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017/12/11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017/12/11 09:30:00' as date) / cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) / cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS TINYINT)):decimal(9,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as decimal(5, 0)) / cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS TINYINT)):decimal(11,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS TINYINT)):decimal(16,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as decimal(20, 0)) / cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS TINYINT)):decimal(26,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as decimal(3, 0)) / cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS SMALLINT)):decimal(9,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as decimal(5, 0)) / cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS SMALLINT)):decimal(11,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS SMALLINT)):decimal(16,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as decimal(20, 0)) / cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS SMALLINT)):decimal(26,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as decimal(3, 0)) / cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS INT)):decimal(14,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as decimal(5, 0)) / cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS INT)):decimal(16,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS INT)):decimal(21,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as decimal(20, 0)) / cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS INT)):decimal(31,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as decimal(3, 0)) / cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS BIGINT)):decimal(24,21)> +-- !query output +1.000000000000000000000 + + +-- !query +SELECT cast(1 as decimal(5, 0)) / cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS BIGINT)):decimal(26,21)> +-- !query output +1.000000000000000000000 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BIGINT)):decimal(31,21)> +-- !query output +1.000000000000000000000 + + +-- !query +SELECT cast(1 as decimal(20, 0)) / cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS BIGINT)):decimal(38,18)> +-- !query output +1.000000000000000000 + + +-- !query +SELECT cast(1 as decimal(3, 0)) / cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS FLOAT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) / cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS FLOAT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS FLOAT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) / cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS FLOAT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) / cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS DOUBLE)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) / cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS DOUBLE)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DOUBLE)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) / cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS DOUBLE)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS DECIMAL(10,0))):decimal(14,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as decimal(5, 0)) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS DECIMAL(10,0))):decimal(16,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as decimal(20, 0)) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS DECIMAL(10,0))):decimal(31,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as decimal(3, 0)) / cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS STRING)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) / cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS STRING)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS STRING)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) / cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS STRING)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) / cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) / cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) / cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) / cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(3, 0)) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) / cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(5, 0)) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) / cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(20, 0)) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(2017/12/11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) / cast('2017/12/11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) / cast('2017/12/11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) / cast('2017/12/11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) / CAST(2017/12/11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) / cast('2017/12/11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) % cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) % CAST(1 AS DECIMAL(3,0))):decimal(3,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as tinyint) % cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) % CAST(1 AS DECIMAL(5,0))):decimal(3,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as tinyint) % cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) % CAST(1 AS DECIMAL(10,0))):decimal(3,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as tinyint) % cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) % CAST(1 AS DECIMAL(20,0))):decimal(3,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as smallint) % cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) % CAST(1 AS DECIMAL(3,0))):decimal(3,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as smallint) % cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) % CAST(1 AS DECIMAL(5,0))):decimal(5,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as smallint) % cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) % CAST(1 AS DECIMAL(10,0))):decimal(5,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as smallint) % cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) % CAST(1 AS DECIMAL(20,0))):decimal(5,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as int) % cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) % CAST(1 AS DECIMAL(3,0))):decimal(3,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as int) % cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) % CAST(1 AS DECIMAL(5,0))):decimal(5,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as int) % cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) % CAST(1 AS DECIMAL(10,0))):decimal(10,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as int) % cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) % CAST(1 AS DECIMAL(20,0))):decimal(10,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as bigint) % cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) % CAST(1 AS DECIMAL(3,0))):decimal(3,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as bigint) % cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) % CAST(1 AS DECIMAL(5,0))):decimal(5,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as bigint) % cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) % CAST(1 AS DECIMAL(10,0))):decimal(10,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as bigint) % cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) % CAST(1 AS DECIMAL(20,0))):decimal(20,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as float) % cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) % CAST(1 AS DECIMAL(3,0))):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as float) % cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) % CAST(1 AS DECIMAL(5,0))):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as float) % cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) % CAST(1 AS DECIMAL(10,0))):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as float) % cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) % CAST(1 AS DECIMAL(20,0))):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as double) % cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) % CAST(1 AS DECIMAL(3,0))):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as double) % cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) % CAST(1 AS DECIMAL(5,0))):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as double) % cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) % CAST(1 AS DECIMAL(10,0))):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as double) % cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) % CAST(1 AS DECIMAL(20,0))):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS DECIMAL(3,0))):decimal(3,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS DECIMAL(5,0))):decimal(5,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS DECIMAL(10,0))):decimal(10,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS DECIMAL(20,0))):decimal(10,0)> +-- !query output +0 + + +-- !query +SELECT cast('1' as binary) % cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) % cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) % cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) % cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) % cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) % cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) % cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) % CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) % cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) % cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) % cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) % cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS TINYINT)):decimal(3,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) % cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS TINYINT)):decimal(3,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS TINYINT)):decimal(3,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) % cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS TINYINT)):decimal(3,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) % cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS SMALLINT)):decimal(3,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) % cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS SMALLINT)):decimal(5,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS SMALLINT)):decimal(5,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) % cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS SMALLINT)):decimal(5,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) % cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS INT)):decimal(3,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) % cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS INT)):decimal(5,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS INT)):decimal(10,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) % cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS INT)):decimal(10,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) % cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS BIGINT)):decimal(3,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) % cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS BIGINT)):decimal(5,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS BIGINT)):decimal(10,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) % cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS BIGINT)):decimal(20,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) % cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS FLOAT)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) % cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS FLOAT)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS FLOAT)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) % cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS FLOAT)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) % cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS DOUBLE)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) % cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS DOUBLE)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS DOUBLE)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) % cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS DOUBLE)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) % cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS DECIMAL(10,0))):decimal(3,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) % cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS DECIMAL(10,0))):decimal(5,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS DECIMAL(10,0))):decimal(10,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) % cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS DECIMAL(10,0))):decimal(10,0)> +-- !query output +0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) % cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS STRING)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(5, 0)) % cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS STRING)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS STRING)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(20, 0)) % cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS STRING)):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(3, 0)) % cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) % cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) % cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) % cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) % cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) % cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) % cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) % cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(3, 0)) % cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) % cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(5, 0)) % cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) % cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) % cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(20, 0)) % cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) % cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) % cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) % cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) % CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) % cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT pmod(cast(1 as tinyint), cast(1 as decimal(3, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as tinyint), cast(1 as decimal(5, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as tinyint), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as tinyint), cast(1 as decimal(20, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as smallint), cast(1 as decimal(3, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as smallint), cast(1 as decimal(5, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as smallint), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as smallint), cast(1 as decimal(20, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as int), cast(1 as decimal(3, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as int), cast(1 as decimal(5, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as int), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as int), cast(1 as decimal(20, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as bigint), cast(1 as decimal(3, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as bigint), cast(1 as decimal(5, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as bigint), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as bigint), cast(1 as decimal(20, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as float), cast(1 as decimal(3, 0))) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as float), cast(1 as decimal(5, 0))) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as float), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as float), cast(1 as decimal(20, 0))) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as double), cast(1 as decimal(3, 0))) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as double), cast(1 as decimal(5, 0))) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as double), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as double), cast(1 as decimal(20, 0))) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(3, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(5, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(20, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast('1' as binary), cast(1 as decimal(3, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "pmod(cast('1' as binary), cast(1 as decimal(3, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('1' as binary), cast(1 as decimal(5, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "pmod(cast('1' as binary), cast(1 as decimal(5, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('1' as binary), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "pmod(cast('1' as binary), cast(1 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('1' as binary), cast(1 as decimal(20, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"pmod(CAST(1 AS BINARY), CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "pmod(cast('1' as binary), cast(1 as decimal(20, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(3, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 81, + "fragment" : "pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(3, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(5, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 81, + "fragment" : "pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(5, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(20, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "pmod(cast('2017-12-11 09:30:00.0' as timestamp), cast(1 as decimal(20, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(3, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(3, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(5, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 74, + "fragment" : "pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(5, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(20, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "pmod(cast('2017-12-11 09:30:00' as date), cast(1 as decimal(20, 0)))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as tinyint)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as tinyint)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as tinyint)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as tinyint)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as smallint)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as smallint)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as smallint)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as smallint)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as int)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as int)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as int)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as int)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as bigint)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as bigint)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as bigint)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as bigint)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as float)) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as float)) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as float)) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as float)) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as double)) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as double)) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as double)) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as double)) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as string)) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as string)) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as string)) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as string)) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as decimal(3, 0)) , cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "pmod(cast(1 as decimal(3, 0)) , cast('1' as binary))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(5, 0)) , cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "pmod(cast(1 as decimal(5, 0)) , cast('1' as binary))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "pmod(cast(1 as decimal(10, 0)), cast('1' as binary))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(20, 0)), cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "pmod(cast(1 as decimal(20, 0)), cast('1' as binary))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(3, 0)) , cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "pmod(cast(1 as decimal(3, 0)) , cast(1 as boolean))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(5, 0)) , cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "pmod(cast(1 as decimal(5, 0)) , cast(1 as boolean))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "pmod(cast(1 as decimal(10, 0)), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(20, 0)), cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 58, + "fragment" : "pmod(cast(1 as decimal(20, 0)), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 82, + "fragment" : "pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(3,0)), CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "pmod(cast(1 as decimal(3, 0)) , cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(5,0)), CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "pmod(cast(1 as decimal(5, 0)) , cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(10,0)), CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "pmod(cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"pmod(CAST(1 AS DECIMAL(20,0)), CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "pmod(cast(1 as decimal(20, 0)), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) = cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) = CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) = cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) = CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) = cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) = CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) = cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) = CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) = cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) = CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) = cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) = CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) = cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) = CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) = cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) = CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) = cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) = CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) = cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) = CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) = cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) = CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) = cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) = CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) = cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) = CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) = cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) = CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) = cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) = CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) = cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) = CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) = cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) = CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) = cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) = CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) = cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) = CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) = cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) = CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) = cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) = CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) = cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) = CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) = cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) = CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) = cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) = CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast('1' as binary) = cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) = cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) = cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) = cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) = cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) = cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) = cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) = cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) = cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) = cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) = cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) = cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) = cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) = cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) = cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) = cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) = cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) = cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) = cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) = cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) = cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) = cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) = cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) = cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) = cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) = cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) = cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) = cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) = cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) = cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) = cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) = cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) = cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) = cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) = cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) = cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) = cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) = cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) = cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) = cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) = cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(3, 0)) = cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) = cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(5, 0)) = cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) = cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) = cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(20, 0)) = cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) = cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) = cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) = cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) = cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) <=> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) <=> CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) <=> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) <=> CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) <=> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) <=> CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) <=> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) <=> CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) <=> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) <=> CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) <=> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) <=> CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) <=> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) <=> CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) <=> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) <=> CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) <=> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) <=> CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) <=> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) <=> CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) <=> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) <=> CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) <=> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) <=> CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) <=> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) <=> CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) <=> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) <=> CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) <=> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) <=> CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) <=> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) <=> CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) <=> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) <=> CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) <=> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) <=> CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) <=> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) <=> CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) <=> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) <=> CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) <=> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) <=> CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) <=> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) <=> CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) <=> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) <=> CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) <=> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) <=> CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast('1' as binary) <=> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) <=> cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) <=> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) <=> cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) <=> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('1' as binary) <=> cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) <=> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <=> CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('1' as binary) <=> cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <=> CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) <=> cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast(1 as decimal(3, 0)) <=> cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <=> cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast(1 as decimal(5, 0)) <=> cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast(1 as decimal(10, 0)) <=> cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <=> cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast(1 as decimal(20, 0)) <=> cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <=> cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <=> CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(3, 0)) <=> cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <=> cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <=> CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(5, 0)) <=> cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(10, 0)) <=> cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <=> cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <=> CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(20, 0)) <=> cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <=> CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "cast(1 as decimal(3, 0)) <=> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "cast(1 as decimal(5, 0)) <=> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "cast(1 as decimal(10, 0)) <=> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <=> CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "cast(1 as decimal(20, 0)) <=> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) < cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) < CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as tinyint) < cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) < CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as tinyint) < cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) < CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as tinyint) < cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) < CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as smallint) < cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) < CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as smallint) < cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) < CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as smallint) < cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) < CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as smallint) < cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) < CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as int) < cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) < CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as int) < cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) < CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as int) < cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) < CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as int) < cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) < CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as bigint) < cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) < CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as bigint) < cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) < CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as bigint) < cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) < CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as bigint) < cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) < CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as float) < cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) < CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as float) < cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) < CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as float) < cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) < CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as float) < cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) < CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as double) < cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) < CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as double) < cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) < CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as double) < cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) < CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as double) < cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) < CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast('1' as binary) < cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) < cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) < cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) < cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) < cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) < cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) < cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) < CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) < cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) < cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) < cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) < cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS TINYINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) < cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS TINYINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS TINYINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) < cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS TINYINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) < cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS SMALLINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) < cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS SMALLINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS SMALLINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) < cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS SMALLINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) < cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS INT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) < cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS INT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS INT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) < cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS INT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) < cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS BIGINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) < cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS BIGINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS BIGINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) < cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS BIGINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) < cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS FLOAT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) < cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS FLOAT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS FLOAT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) < cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS FLOAT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) < cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS DOUBLE)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) < cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS DOUBLE)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS DOUBLE)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) < cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS DOUBLE)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) < cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) < cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) < cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) < cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS STRING)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) < cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS STRING)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS STRING)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) < cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS STRING)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) < cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) < cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) < cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) < cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) < cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) < cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) < cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) < cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(3, 0)) < cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) < cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(5, 0)) < cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) < cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) < cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(20, 0)) < cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) < cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) < cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) < cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) < CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) < cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) <= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) <= CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) <= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) <= CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) <= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) <= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) <= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) <= CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) <= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) <= CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) <= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) <= CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) <= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) <= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) <= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) <= CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) <= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) <= CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) <= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) <= CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) <= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) <= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) <= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) <= CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) <= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) <= CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) <= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) <= CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) <= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) <= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) <= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) <= CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) <= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) <= CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) <= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) <= CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) <= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) <= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) <= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) <= CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) <= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) <= CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) <= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) <= CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) <= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) <= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) <= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) <= CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast('1' as binary) <= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) <= cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) <= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) <= cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) <= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) <= cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) <= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) <= CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) <= cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <= cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) <= cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <= cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) <= cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) <= cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) <= cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) <= cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) <= cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) <= cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) <= cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) <= cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) <= cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) <= cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) <= cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) <= cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) <= cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) <= cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) <= cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) <= cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) <= cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) <= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) <= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) <= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) <= cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) <= cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) <= cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) <= cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(3, 0)) <= cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <= cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(5, 0)) <= cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(10, 0)) <= cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <= cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(20, 0)) <= cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <= cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) <= cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <= cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) <= cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) <= cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <= cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) <= cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(3, 0)) <= cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(5, 0)) <= cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(10, 0)) <= cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) <= CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(20, 0)) <= cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) > cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) > CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as tinyint) > cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) > CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as tinyint) > cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) > CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as tinyint) > cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) > CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as smallint) > cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) > CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as smallint) > cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) > CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as smallint) > cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) > CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as smallint) > cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) > CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as int) > cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) > CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as int) > cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) > CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as int) > cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) > CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as int) > cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) > CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as bigint) > cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) > CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as bigint) > cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) > CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as bigint) > cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) > CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as bigint) > cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) > CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as float) > cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) > CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as float) > cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) > CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as float) > cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) > CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as float) > cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) > CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as double) > cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) > CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as double) > cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) > CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as double) > cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) > CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as double) > cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) > CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast('1' as binary) > cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) > cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) > cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast('1' as binary) > cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) > cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) > cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) > cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) > CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) > cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) > cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) > cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) > cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS TINYINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) > cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS TINYINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS TINYINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) > cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS TINYINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) > cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS SMALLINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) > cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS SMALLINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS SMALLINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) > cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS SMALLINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) > cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS INT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) > cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS INT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS INT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) > cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS INT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) > cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS BIGINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) > cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS BIGINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS BIGINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) > cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS BIGINT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) > cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS FLOAT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) > cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS FLOAT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS FLOAT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) > cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS FLOAT)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) > cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS DOUBLE)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) > cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS DOUBLE)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS DOUBLE)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) > cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS DOUBLE)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) > cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) > cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) > cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) > cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS STRING)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) > cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS STRING)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS STRING)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) > cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS STRING)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) > cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) > cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) > cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) > cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) > cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) > cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) > cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) > cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(3, 0)) > cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) > cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(5, 0)) > cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) > cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) > cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(20, 0)) > cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(3, 0)) > cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(5, 0)) > cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) > cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) > CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(20, 0)) > cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) >= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) >= CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) >= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) >= CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) >= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) >= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) >= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) >= CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) >= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) >= CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) >= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) >= CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) >= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) >= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) >= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) >= CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) >= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) >= CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) >= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) >= CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) >= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) >= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) >= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) >= CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) >= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) >= CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) >= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) >= CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) >= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) >= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) >= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) >= CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) >= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) >= CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) >= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) >= CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) >= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) >= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) >= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) >= CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) >= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) >= CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) >= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) >= CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) >= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) >= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) >= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) >= CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS DECIMAL(3,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS DECIMAL(5,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS DECIMAL(20,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast('1' as binary) >= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) >= cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) >= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) >= cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) >= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) >= cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) >= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) >= CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) >= cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) >= cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) >= cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) >= cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) >= cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) >= cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) >= cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) >= cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) >= cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) >= cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) >= cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) >= cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) >= cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) >= cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) >= cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) >= cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) >= cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) >= cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) >= cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) >= cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) >= cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) >= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) >= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) >= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) >= cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(5, 0)) >= cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(20, 0)) >= cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS STRING)):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(3, 0)) >= cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(3, 0)) >= cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) >= cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(5, 0)) >= cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(10, 0)) >= cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) >= cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(20, 0)) >= cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) >= cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) >= cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) >= cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) >= cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) >= cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) >= cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) >= cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(3, 0)) >= cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(5, 0)) >= cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(10, 0)) >= cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) >= CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(20, 0)) >= cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) <> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS TINYINT) = CAST(1 AS DECIMAL(3,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as tinyint) <> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS TINYINT) = CAST(1 AS DECIMAL(5,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as tinyint) <> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS TINYINT) = CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as tinyint) <> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS TINYINT) = CAST(1 AS DECIMAL(20,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as smallint) <> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS SMALLINT) = CAST(1 AS DECIMAL(3,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as smallint) <> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS SMALLINT) = CAST(1 AS DECIMAL(5,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as smallint) <> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS SMALLINT) = CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as smallint) <> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS SMALLINT) = CAST(1 AS DECIMAL(20,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as int) <> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS INT) = CAST(1 AS DECIMAL(3,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as int) <> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS INT) = CAST(1 AS DECIMAL(5,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as int) <> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS INT) = CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as int) <> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS INT) = CAST(1 AS DECIMAL(20,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as bigint) <> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS BIGINT) = CAST(1 AS DECIMAL(3,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as bigint) <> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS BIGINT) = CAST(1 AS DECIMAL(5,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as bigint) <> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS BIGINT) = CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as bigint) <> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS BIGINT) = CAST(1 AS DECIMAL(20,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as float) <> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS FLOAT) = CAST(1 AS DECIMAL(3,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as float) <> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS FLOAT) = CAST(1 AS DECIMAL(5,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as float) <> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS FLOAT) = CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as float) <> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS FLOAT) = CAST(1 AS DECIMAL(20,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as double) <> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DOUBLE) = CAST(1 AS DECIMAL(3,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as double) <> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DOUBLE) = CAST(1 AS DECIMAL(5,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as double) <> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DOUBLE) = CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as double) <> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DOUBLE) = CAST(1 AS DECIMAL(20,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(3,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(5,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(20,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast('1' as binary) <> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) <> cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) <> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) <> cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) <> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) <> cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) <> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) = CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) <> cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) = CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <> cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(3, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(3,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(3,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(3, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(5, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(5,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(5,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(5, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(20, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(20,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DECIMAL(20,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00' as date) <> cast(1 as decimal(20, 0))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <> cast(1 as tinyint) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(3,0)) = CAST(1 AS TINYINT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) <> cast(1 as tinyint) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(5,0)) = CAST(1 AS TINYINT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast(1 as tinyint) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS TINYINT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) <> cast(1 as tinyint) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(20,0)) = CAST(1 AS TINYINT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) <> cast(1 as smallint) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(3,0)) = CAST(1 AS SMALLINT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) <> cast(1 as smallint) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(5,0)) = CAST(1 AS SMALLINT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast(1 as smallint) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS SMALLINT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) <> cast(1 as smallint) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(20,0)) = CAST(1 AS SMALLINT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) <> cast(1 as int) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(3,0)) = CAST(1 AS INT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) <> cast(1 as int) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(5,0)) = CAST(1 AS INT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast(1 as int) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS INT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) <> cast(1 as int) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(20,0)) = CAST(1 AS INT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) <> cast(1 as bigint) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BIGINT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) <> cast(1 as bigint) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BIGINT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast(1 as bigint) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BIGINT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) <> cast(1 as bigint) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BIGINT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) <> cast(1 as float) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(3,0)) = CAST(1 AS FLOAT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) <> cast(1 as float) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(5,0)) = CAST(1 AS FLOAT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast(1 as float) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS FLOAT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) <> cast(1 as float) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(20,0)) = CAST(1 AS FLOAT))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) <> cast(1 as double) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(3,0)) = CAST(1 AS DOUBLE))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) <> cast(1 as double) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(5,0)) = CAST(1 AS DOUBLE))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast(1 as double) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DOUBLE))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) <> cast(1 as double) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(20,0)) = CAST(1 AS DOUBLE))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) <> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(3,0)) = CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) <> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(5,0)) = CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) <> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(20,0)) = CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) <> cast(1 as string) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(3,0)) = CAST(1 AS STRING))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(5, 0)) <> cast(1 as string) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(5,0)) = CAST(1 AS STRING))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast(1 as string) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = CAST(1 AS STRING))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(20, 0)) <> cast(1 as string) FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(20,0)) = CAST(1 AS STRING))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(3, 0)) <> cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(3, 0)) <> cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <> cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(5, 0)) <> cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(10, 0)) <> cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <> cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as decimal(20, 0)) <> cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <> cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(3, 0)) <> cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <> cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(5, 0)) <> cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) <> cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <> cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(20, 0)) <> cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(3,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(3,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(3, 0)) <> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(5,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(5,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(5, 0)) <> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(10, 0)) <> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(20,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(20,0)) = CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as decimal(20, 0)) <> cast('2017-12-11 09:30:00' as date)" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/division.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/division.sql.out new file mode 100644 index 000000000000..cf6931a4ffda --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/division.sql.out @@ -0,0 +1,2455 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT cast(1 as tinyint) / cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) / CAST(1 AS TINYINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as tinyint) / cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) / CAST(1 AS SMALLINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as tinyint) / cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) / CAST(1 AS INT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as tinyint) / cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) / CAST(1 AS BIGINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as tinyint) / cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) / CAST(1 AS FLOAT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as tinyint) / cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) / CAST(1 AS DOUBLE)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as tinyint) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) / CAST(1 AS DECIMAL(10,0))):decimal(14,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as tinyint) / cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) / CAST(1 AS STRING)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as tinyint) / cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast(1 as tinyint) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) / cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(1 as tinyint) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as tinyint) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TINYINT\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS TINYINT) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 63, + "fragment" : "cast(1 as tinyint) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as smallint) / cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) / CAST(1 AS TINYINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as smallint) / cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) / CAST(1 AS SMALLINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as smallint) / cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) / CAST(1 AS INT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as smallint) / cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) / CAST(1 AS BIGINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as smallint) / cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) / CAST(1 AS FLOAT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as smallint) / cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) / CAST(1 AS DOUBLE)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as smallint) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) / CAST(1 AS DECIMAL(10,0))):decimal(16,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as smallint) / cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) / CAST(1 AS STRING)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as smallint) / cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "cast(1 as smallint) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as smallint) / cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast(1 as smallint) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast(1 as smallint) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as smallint) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"SMALLINT\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "cast(1 as smallint) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as int) / cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS INT) / CAST(1 AS TINYINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as int) / cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS INT) / CAST(1 AS SMALLINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as int) / cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS INT) / CAST(1 AS INT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as int) / cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS INT) / CAST(1 AS BIGINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as int) / cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS INT) / CAST(1 AS FLOAT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as int) / cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS INT) / CAST(1 AS DOUBLE)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as int) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS INT) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as int) / cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS INT) / CAST(1 AS STRING)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as int) / cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS INT) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "cast(1 as int) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as int) / cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS INT) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "cast(1 as int) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as int) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS INT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 66, + "fragment" : "cast(1 as int) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as int) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS INT) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "cast(1 as int) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as bigint) / cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) / CAST(1 AS TINYINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as bigint) / cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) / CAST(1 AS SMALLINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as bigint) / cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) / CAST(1 AS INT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as bigint) / cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) / CAST(1 AS BIGINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as bigint) / cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) / CAST(1 AS FLOAT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as bigint) / cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) / CAST(1 AS DOUBLE)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as bigint) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) / CAST(1 AS DECIMAL(10,0))):decimal(31,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as bigint) / cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) / CAST(1 AS STRING)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as bigint) / cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(1 as bigint) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as bigint) / cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as bigint) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast(1 as bigint) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as bigint) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BIGINT\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS BIGINT) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "cast(1 as bigint) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as float) / cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) / CAST(1 AS TINYINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as float) / cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) / CAST(1 AS SMALLINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as float) / cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) / CAST(1 AS INT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as float) / cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) / CAST(1 AS BIGINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as float) / cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) / CAST(1 AS FLOAT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as float) / cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) / CAST(1 AS DOUBLE)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as float) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) / CAST(1 AS DECIMAL(10,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as float) / cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) / CAST(1 AS STRING)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as float) / cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as float) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as float) / cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "cast(1 as float) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as float) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 68, + "fragment" : "cast(1 as float) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as float) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"FLOAT\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS FLOAT) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "cast(1 as float) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as double) / cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) / CAST(1 AS TINYINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as double) / cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) / CAST(1 AS SMALLINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as double) / cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) / CAST(1 AS INT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as double) / cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) / CAST(1 AS BIGINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as double) / cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) / CAST(1 AS FLOAT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as double) / cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) / CAST(1 AS DOUBLE)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as double) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) / CAST(1 AS DECIMAL(10,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as double) / cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) / CAST(1 AS STRING)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as double) / cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(1 as double) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as double) / cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as double) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as double) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast(1 as double) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as double) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DOUBLE\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "cast(1 as double) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS TINYINT)):decimal(16,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS SMALLINT)):decimal(16,6)> +-- !query output +1.000000 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS INT)):decimal(21,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BIGINT)):decimal(31,21)> +-- !query output +1.000000000000000000000 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS FLOAT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DOUBLE)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS DECIMAL(10,0))):decimal(21,11)> +-- !query output +1.00000000000 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as string) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS STRING)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast(1 as decimal(10, 0)) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as decimal(10, 0)) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DECIMAL(10,0)\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as decimal(10, 0)) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as string) / cast(1 as tinyint) FROM t +-- !query schema +struct<(CAST(1 AS STRING) / CAST(1 AS TINYINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as string) / cast(1 as smallint) FROM t +-- !query schema +struct<(CAST(1 AS STRING) / CAST(1 AS SMALLINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as string) / cast(1 as int) FROM t +-- !query schema +struct<(CAST(1 AS STRING) / CAST(1 AS INT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as string) / cast(1 as bigint) FROM t +-- !query schema +struct<(CAST(1 AS STRING) / CAST(1 AS BIGINT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as string) / cast(1 as float) FROM t +-- !query schema +struct<(CAST(1 AS STRING) / CAST(1 AS FLOAT)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as string) / cast(1 as double) FROM t +-- !query schema +struct<(CAST(1 AS STRING) / CAST(1 AS DOUBLE)):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as string) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) / CAST(1 AS DECIMAL(10,0))):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as string) / cast(1 as string) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(1 AS STRING))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "cast(1 as string) / cast(1 as string)" + } ] +} + + +-- !query +SELECT cast(1 as string) / cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(1 as string) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as string) / cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as string) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as string) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast(1 as string) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as string) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"DATE\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(1 AS STRING) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "cast(1 as string) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as tinyint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast('1' as binary) / cast(1 as tinyint)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as smallint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "cast('1' as binary) / cast(1 as smallint)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as int) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 43, + "fragment" : "cast('1' as binary) / cast(1 as int)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as bigint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast('1' as binary) / cast(1 as bigint)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as float) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast('1' as binary) / cast(1 as float)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as double) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast('1' as binary) / cast(1 as double)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "cast('1' as binary) / cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as string) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS STRING))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast('1' as binary) / cast(1 as string)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "cast('1' as binary) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast('1' as binary) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('1' as binary) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast('1' as binary) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BINARY\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "cast('1' as binary) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast(1 as tinyint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(1 as boolean) / cast(1 as tinyint)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast(1 as smallint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast(1 as boolean) / cast(1 as smallint)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast(1 as int) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 42, + "fragment" : "cast(1 as boolean) / cast(1 as int)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast(1 as bigint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as boolean) / cast(1 as bigint)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast(1 as float) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 44, + "fragment" : "cast(1 as boolean) / cast(1 as float)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast(1 as double) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as boolean) / cast(1 as double)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "cast(1 as boolean) / cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast(1 as string) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS STRING))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 45, + "fragment" : "cast(1 as boolean) / cast(1 as string)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 47, + "fragment" : "cast(1 as boolean) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 46, + "fragment" : "cast(1 as boolean) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast(1 as boolean) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"BOOLEAN\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 63, + "fragment" : "cast(1 as boolean) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as tinyint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as tinyint)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as smallint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as smallint)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as int) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 66, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as int)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as bigint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as bigint)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as float) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 68, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as float)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as double) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as double)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as string) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS STRING))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 69, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as string)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 94, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"TIMESTAMP\"", + "right" : "\"DATE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 87, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as tinyint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"TINYINT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS TINYINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 63, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as tinyint)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as smallint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"SMALLINT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS SMALLINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as smallint)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as int) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"INT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS INT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as int)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as bigint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"BIGINT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS BIGINT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as bigint)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as float) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"FLOAT\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS FLOAT))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 61, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as float)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as double) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DOUBLE\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS DOUBLE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as double)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"DECIMAL(10,0)\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS DECIMAL(10,0)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as decimal(10, 0))" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as string) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"DATE\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS STRING))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as string)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"BINARY\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 64, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast('1' as binary)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 63, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"DATE\"", + "right" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 87, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"DATE\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "cast('2017-12-11 09:30:00' as date) / cast('2017-12-11 09:30:00' as date)" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/elt.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/elt.sql.out new file mode 100644 index 000000000000..61ea81f999b0 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/elt.sql.out @@ -0,0 +1,112 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT elt(2, col1, col2, col3, col4, col5) col +FROM ( + SELECT + 'prefix_' col1, + id col2, + string(id + 1) col3, + encode(string(id + 2), 'utf-8') col4, + CAST(id AS DOUBLE) col5 + FROM range(10) +) +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +SELECT elt(3, col1, col2, col3, col4) col +FROM ( + SELECT + string(id) col1, + string(id + 1) col2, + encode(string(id + 2), 'utf-8') col3, + encode(string(id + 3), 'utf-8') col4 + FROM range(10) +) +-- !query schema +struct +-- !query output +10 +11 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +set spark.sql.function.eltOutputAsString=true +-- !query schema +struct +-- !query output +spark.sql.function.eltOutputAsString true + + +-- !query +SELECT elt(1, col1, col2) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2 + FROM range(10) +) +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +set spark.sql.function.eltOutputAsString=false +-- !query schema +struct +-- !query output +spark.sql.function.eltOutputAsString false + + +-- !query +SELECT elt(2, col1, col2) col +FROM ( + SELECT + encode(string(id), 'utf-8') col1, + encode(string(id + 1), 'utf-8') col2 + FROM range(10) +) +-- !query schema +struct +-- !query output +1 +10 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/ifCoercion.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/ifCoercion.sql.out new file mode 100644 index 000000000000..ffcfba2717d6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/ifCoercion.sql.out @@ -0,0 +1,2249 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT IF(true, cast(1 as tinyint), cast(2 as tinyint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS TINYINT), CAST(2 AS TINYINT))):tinyint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as tinyint), cast(2 as smallint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS TINYINT), CAST(2 AS SMALLINT))):smallint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as tinyint), cast(2 as int)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS TINYINT), CAST(2 AS INT))):int> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as tinyint), cast(2 as bigint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS TINYINT), CAST(2 AS BIGINT))):bigint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as tinyint), cast(2 as float)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS TINYINT), CAST(2 AS FLOAT))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as tinyint), cast(2 as double)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS TINYINT), CAST(2 AS DOUBLE))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as tinyint), cast(2 as decimal(10, 0))) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS TINYINT), CAST(2 AS DECIMAL(10,0)))):decimal(10,0)> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as tinyint), cast(2 as string)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS TINYINT), CAST(2 AS STRING))):bigint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as tinyint), cast('2' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS TINYINT), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "IF(true, cast(1 as tinyint), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as tinyint), cast(2 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS TINYINT), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "IF(true, cast(1 as tinyint), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as tinyint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS TINYINT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "IF(true, cast(1 as tinyint), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as tinyint), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS TINYINT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "IF(true, cast(1 as tinyint), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as smallint), cast(2 as tinyint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS SMALLINT), CAST(2 AS TINYINT))):smallint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as smallint), cast(2 as smallint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS SMALLINT), CAST(2 AS SMALLINT))):smallint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as smallint), cast(2 as int)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS SMALLINT), CAST(2 AS INT))):int> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as smallint), cast(2 as bigint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS SMALLINT), CAST(2 AS BIGINT))):bigint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as smallint), cast(2 as float)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS SMALLINT), CAST(2 AS FLOAT))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as smallint), cast(2 as double)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS SMALLINT), CAST(2 AS DOUBLE))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as smallint), cast(2 as decimal(10, 0))) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS SMALLINT), CAST(2 AS DECIMAL(10,0)))):decimal(10,0)> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as smallint), cast(2 as string)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS SMALLINT), CAST(2 AS STRING))):bigint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as smallint), cast('2' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS SMALLINT), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "IF(true, cast(1 as smallint), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as smallint), cast(2 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS SMALLINT), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "IF(true, cast(1 as smallint), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as smallint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS SMALLINT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "IF(true, cast(1 as smallint), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as smallint), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS SMALLINT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "IF(true, cast(1 as smallint), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as int), cast(2 as tinyint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS INT), CAST(2 AS TINYINT))):int> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as int), cast(2 as smallint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS INT), CAST(2 AS SMALLINT))):int> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as int), cast(2 as int)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS INT), CAST(2 AS INT))):int> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as int), cast(2 as bigint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS INT), CAST(2 AS BIGINT))):bigint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as int), cast(2 as float)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS INT), CAST(2 AS FLOAT))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as int), cast(2 as double)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS INT), CAST(2 AS DOUBLE))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as int), cast(2 as decimal(10, 0))) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS INT), CAST(2 AS DECIMAL(10,0)))):decimal(10,0)> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as int), cast(2 as string)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS INT), CAST(2 AS STRING))):bigint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as int), cast('2' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS INT), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "IF(true, cast(1 as int), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as int), cast(2 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS INT), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "IF(true, cast(1 as int), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as int), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS INT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "IF(true, cast(1 as int), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as int), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS INT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 68, + "fragment" : "IF(true, cast(1 as int), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as bigint), cast(2 as tinyint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS BIGINT), CAST(2 AS TINYINT))):bigint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as bigint), cast(2 as smallint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS BIGINT), CAST(2 AS SMALLINT))):bigint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as bigint), cast(2 as int)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS BIGINT), CAST(2 AS INT))):bigint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as bigint), cast(2 as bigint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS BIGINT), CAST(2 AS BIGINT))):bigint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as bigint), cast(2 as float)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS BIGINT), CAST(2 AS FLOAT))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as bigint), cast(2 as double)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS BIGINT), CAST(2 AS DOUBLE))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as bigint), cast(2 as decimal(10, 0))) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS BIGINT), CAST(2 AS DECIMAL(10,0)))):decimal(20,0)> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as bigint), cast(2 as string)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS BIGINT), CAST(2 AS STRING))):bigint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as bigint), cast('2' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BIGINT), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "IF(true, cast(1 as bigint), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as bigint), cast(2 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BIGINT), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "IF(true, cast(1 as bigint), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as bigint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BIGINT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "IF(true, cast(1 as bigint), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as bigint), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BIGINT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "IF(true, cast(1 as bigint), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as float), cast(2 as tinyint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS TINYINT))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as float), cast(2 as smallint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS SMALLINT))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as float), cast(2 as int)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS INT))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as float), cast(2 as bigint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS BIGINT))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as float), cast(2 as float)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS FLOAT))):float> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as float), cast(2 as double)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS DOUBLE))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as float), cast(2 as decimal(10, 0))) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS DECIMAL(10,0)))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as float), cast(2 as string)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS FLOAT), CAST(2 AS STRING))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as float), cast('2' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS FLOAT), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "IF(true, cast(1 as float), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as float), cast(2 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS FLOAT), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "IF(true, cast(1 as float), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as float), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS FLOAT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "IF(true, cast(1 as float), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as float), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS FLOAT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "IF(true, cast(1 as float), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as double), cast(2 as tinyint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS DOUBLE), CAST(2 AS TINYINT))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as double), cast(2 as smallint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS DOUBLE), CAST(2 AS SMALLINT))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as double), cast(2 as int)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS DOUBLE), CAST(2 AS INT))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as double), cast(2 as bigint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS DOUBLE), CAST(2 AS BIGINT))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as double), cast(2 as float)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS DOUBLE), CAST(2 AS FLOAT))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as double), cast(2 as double)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS DOUBLE), CAST(2 AS DOUBLE))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as double), cast(2 as decimal(10, 0))) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS DOUBLE), CAST(2 AS DECIMAL(10,0)))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as double), cast(2 as string)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS DOUBLE), CAST(2 AS STRING))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as double), cast('2' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DOUBLE), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "IF(true, cast(1 as double), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as double), cast(2 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DOUBLE), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "IF(true, cast(1 as double), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as double), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DOUBLE), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "IF(true, cast(1 as double), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as double), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DOUBLE), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "IF(true, cast(1 as double), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as tinyint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS TINYINT))):decimal(10,0)> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as smallint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS SMALLINT))):decimal(10,0)> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as int)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS INT))):decimal(10,0)> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as bigint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS BIGINT))):decimal(20,0)> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as float)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS FLOAT))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as double)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS DOUBLE))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as decimal(10, 0))) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS DECIMAL(10,0)))):decimal(10,0)> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as string)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS STRING))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as decimal(10, 0)), cast('2' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 63, + "fragment" : "IF(true, cast(1 as decimal(10, 0)), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as decimal(10, 0)), cast(2 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "IF(true, cast(1 as decimal(10, 0)), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 86, + "fragment" : "IF(true, cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS DECIMAL(10,0)), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "IF(true, cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as string), cast(2 as tinyint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS TINYINT))):bigint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as string), cast(2 as smallint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS SMALLINT))):bigint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as string), cast(2 as int)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS INT))):bigint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as string), cast(2 as bigint)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS BIGINT))):bigint> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as string), cast(2 as float)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS FLOAT))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as string), cast(2 as double)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS DOUBLE))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as string), cast(2 as decimal(10, 0))) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS DECIMAL(10,0)))):double> +-- !query output +1.0 + + +-- !query +SELECT IF(true, cast(1 as string), cast(2 as string)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS STRING))):string> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as string), cast('2' as binary)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS BINARY))):binary> +-- !query output +1 + + +-- !query +SELECT IF(true, cast(1 as string), cast(2 as boolean)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS STRING), CAST(2 AS BOOLEAN))):boolean> +-- !query output +true + + +-- !query +SELECT IF(true, cast(1 as string), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "IF(true, cast(1 as string), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as string), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "IF(true, cast(1 as string), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast(2 as tinyint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"TINYINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "IF(true, cast('1' as binary), cast(2 as tinyint))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast(2 as smallint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"SMALLINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "IF(true, cast('1' as binary), cast(2 as smallint))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast(2 as int)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"INT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 52, + "fragment" : "IF(true, cast('1' as binary), cast(2 as int))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast(2 as bigint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BIGINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "IF(true, cast('1' as binary), cast(2 as bigint))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast(2 as float)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"FLOAT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "IF(true, cast('1' as binary), cast(2 as float))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast(2 as double)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"DOUBLE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "IF(true, cast('1' as binary), cast(2 as double))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast(2 as decimal(10, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"DECIMAL(10,0)\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 63, + "fragment" : "IF(true, cast('1' as binary), cast(2 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast(2 as string)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS BINARY), CAST(2 AS STRING))):binary> +-- !query output +1 + + +-- !query +SELECT IF(true, cast('1' as binary), cast('2' as binary)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS BINARY), CAST(2 AS BINARY))):binary> +-- !query output +1 + + +-- !query +SELECT IF(true, cast('1' as binary), cast(2 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "IF(true, cast('1' as binary), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "IF(true, cast('1' as binary), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast('1' as binary), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BINARY), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "IF(true, cast('1' as binary), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast(2 as tinyint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"TINYINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "IF(true, cast(1 as boolean), cast(2 as tinyint))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast(2 as smallint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"SMALLINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "IF(true, cast(1 as boolean), cast(2 as smallint))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast(2 as int)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"INT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "IF(true, cast(1 as boolean), cast(2 as int))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast(2 as bigint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BIGINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "IF(true, cast(1 as boolean), cast(2 as bigint))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast(2 as float)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"FLOAT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "IF(true, cast(1 as boolean), cast(2 as float))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast(2 as double)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"DOUBLE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 54, + "fragment" : "IF(true, cast(1 as boolean), cast(2 as double))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast(2 as decimal(10, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"DECIMAL(10,0)\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 62, + "fragment" : "IF(true, cast(1 as boolean), cast(2 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast(2 as string)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT IF(true, cast(1 as boolean), cast('2' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "IF(true, cast(1 as boolean), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast(2 as boolean)) FROM t +-- !query schema +struct<(IF(true, CAST(1 AS BOOLEAN), CAST(2 AS BOOLEAN))):boolean> +-- !query output +true + + +-- !query +SELECT IF(true, cast(1 as boolean), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"TIMESTAMP\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "IF(true, cast(1 as boolean), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT IF(true, cast(1 as boolean), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"DATE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(1 AS BOOLEAN), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "IF(true, cast(1 as boolean), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as tinyint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TINYINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as tinyint))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as smallint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"SMALLINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as smallint))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as int)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"INT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as int))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as bigint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"BIGINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as bigint))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as float)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"FLOAT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 77, + "fragment" : "IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as float))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as double)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"DOUBLE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 78, + "fragment" : "IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as double))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as decimal(10, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"DECIMAL(10,0)\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 86, + "fragment" : "IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as string)) FROM t +-- !query schema +struct<(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS STRING))):timestamp> +-- !query output +2017-12-12 09:30:00 + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast('2' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))):timestamp> +-- !query output +2017-12-12 09:30:00 + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00.0' as timestamp), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<(IF(true, CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2017-12-11 09:30:00 AS DATE))):timestamp> +-- !query output +2017-12-12 09:30:00 + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as tinyint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"TINYINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as tinyint))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as smallint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"SMALLINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as smallint))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as int)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"INT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 68, + "fragment" : "IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as int))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as bigint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"BIGINT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as bigint))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as float)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"FLOAT\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 70, + "fragment" : "IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as float))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as double)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DOUBLE\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 71, + "fragment" : "IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as double))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as decimal(10, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DECIMAL(10,0)\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as string)) FROM t +-- !query schema +struct<(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS STRING))):date> +-- !query output +2017-12-12 + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast('2' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"BINARY\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 73, + "fragment" : "IF(true, cast('2017-12-12 09:30:00' as date), cast('2' as binary))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"BOOLEAN\"]", + "functionName" : "`if`", + "sqlExpr" : "\"(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 72, + "fragment" : "IF(true, cast('2017-12-12 09:30:00' as date), cast(2 as boolean))" + } ] +} + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))):timestamp> +-- !query output +2017-12-12 00:00:00 + + +-- !query +SELECT IF(true, cast('2017-12-12 09:30:00' as date), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<(IF(true, CAST(2017-12-12 09:30:00 AS DATE), CAST(2017-12-11 09:30:00 AS DATE))):date> +-- !query output +2017-12-12 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out new file mode 100644 index 000000000000..bb75fe5991ac --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/implicitTypeCasts.sql.out @@ -0,0 +1,351 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 1 + '2' FROM t +-- !query schema +struct<(1 + 2):bigint> +-- !query output +3 + + +-- !query +SELECT 1 - '2' FROM t +-- !query schema +struct<(1 - 2):bigint> +-- !query output +-1 + + +-- !query +SELECT 1 * '2' FROM t +-- !query schema +struct<(1 * 2):bigint> +-- !query output +2 + + +-- !query +SELECT 4 / '2' FROM t +-- !query schema +struct<(4 / 2):double> +-- !query output +2.0 + + +-- !query +SELECT 1.1 + '2' FROM t +-- !query schema +struct<(1.1 + 2):double> +-- !query output +3.1 + + +-- !query +SELECT 1.1 - '2' FROM t +-- !query schema +struct<(1.1 - 2):double> +-- !query output +-0.8999999999999999 + + +-- !query +SELECT 1.1 * '2' FROM t +-- !query schema +struct<(1.1 * 2):double> +-- !query output +2.2 + + +-- !query +SELECT 4.4 / '2' FROM t +-- !query schema +struct<(4.4 / 2):double> +-- !query output +2.2 + + +-- !query +SELECT 1.1 + '2.2' FROM t +-- !query schema +struct<(1.1 + 2.2):double> +-- !query output +3.3000000000000003 + + +-- !query +SELECT 1.1 - '2.2' FROM t +-- !query schema +struct<(1.1 - 2.2):double> +-- !query output +-1.1 + + +-- !query +SELECT 1.1 * '2.2' FROM t +-- !query schema +struct<(1.1 * 2.2):double> +-- !query output +2.4200000000000004 + + +-- !query +SELECT 4.4 / '2.2' FROM t +-- !query schema +struct<(4.4 / 2.2):double> +-- !query output +2.0 + + +-- !query +SELECT '$' || cast(1 as smallint) || '$' FROM t +-- !query schema +struct +-- !query output +$1$ + + +-- !query +SELECT '$' || 1 || '$' FROM t +-- !query schema +struct +-- !query output +$1$ + + +-- !query +SELECT '$' || cast(1 as bigint) || '$' FROM t +-- !query schema +struct +-- !query output +$1$ + + +-- !query +SELECT '$' || cast(1.1 as float) || '$' FROM t +-- !query schema +struct +-- !query output +$1.1$ + + +-- !query +SELECT '$' || cast(1.1 as double) || '$' FROM t +-- !query schema +struct +-- !query output +$1.1$ + + +-- !query +SELECT '$' || 1.1 || '$' FROM t +-- !query schema +struct +-- !query output +$1.1$ + + +-- !query +SELECT '$' || cast(1.1 as decimal(8,3)) || '$' FROM t +-- !query schema +struct +-- !query output +$1.100$ + + +-- !query +SELECT '$' || 'abcd' || '$' FROM t +-- !query schema +struct +-- !query output +$abcd$ + + +-- !query +SELECT '$' || date('1996-09-09') || '$' FROM t +-- !query schema +struct +-- !query output +$1996-09-09$ + + +-- !query +SELECT '$' || timestamp('1996-09-09 10:11:12.4' )|| '$' FROM t +-- !query schema +struct +-- !query output +$1996-09-09 10:11:12.4$ + + +-- !query +SELECT length(cast(1 as smallint)) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT length(cast(1 as int)) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT length(cast(1 as bigint)) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT length(cast(1.1 as float)) FROM t +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT length(cast(1.1 as double)) FROM t +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT length(1.1) FROM t +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT length(cast(1.1 as decimal(8,3))) FROM t +-- !query schema +struct +-- !query output +5 + + +-- !query +SELECT length('four') FROM t +-- !query schema +struct +-- !query output +4 + + +-- !query +SELECT length(date('1996-09-10')) FROM t +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT length(timestamp('1996-09-10 10:11:12.4')) FROM t +-- !query schema +struct +-- !query output +21 + + +-- !query +SELECT year( '1996-01-10') FROM t +-- !query schema +struct +-- !query output +1996 + + +-- !query +SELECT month( '1996-01-10') FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT day( '1996-01-10') FROM t +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT hour( '10:11:12') FROM t +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT minute( '10:11:12') FROM t +-- !query schema +struct +-- !query output +11 + + +-- !query +SELECT second( '10:11:12') FROM t +-- !query schema +struct +-- !query output +12 + + +-- !query +select 1 like '%' FROM t +-- !query schema +struct<1 LIKE %:boolean> +-- !query output +true + + +-- !query +select date('1996-09-10') like '19%' FROM t +-- !query schema +struct<1996-09-10 LIKE 19%:boolean> +-- !query output +true + + +-- !query +select '1' like 1 FROM t +-- !query schema +struct<1 LIKE 1:boolean> +-- !query output +true + + +-- !query +select '1 ' like 1 FROM t +-- !query schema +struct<1 LIKE 1:boolean> +-- !query output +false + + +-- !query +select '1996-09-10' like date('1996-09-10') FROM t +-- !query schema +struct<1996-09-10 LIKE 1996-09-10:boolean> +-- !query output +true diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/inConversion.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/inConversion.sql.out new file mode 100644 index 000000000000..6005466b8e2f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/inConversion.sql.out @@ -0,0 +1,4525 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as smallint)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) IN (CAST(1 AS SMALLINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as int)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) IN (CAST(1 AS INT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as bigint)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) IN (CAST(1 AS BIGINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as float)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) IN (CAST(1 AS FLOAT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as double)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) IN (CAST(1 AS DOUBLE))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) IN (CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) IN (CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) in (cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 50, + "fragment" : "in (cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 49, + "fragment" : "in (cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 73, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 66, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as tinyint)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS TINYINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as int)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS INT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as bigint)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS BIGINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as float)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS FLOAT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as double)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS DOUBLE))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) in (cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 51, + "fragment" : "in (cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 50, + "fragment" : "in (cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as smallint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 74, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as smallint) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 67, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as int) in (cast(1 as tinyint)) FROM t +-- !query schema +struct<(CAST(1 AS INT) IN (CAST(1 AS TINYINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) in (cast(1 as smallint)) FROM t +-- !query schema +struct<(CAST(1 AS INT) IN (CAST(1 AS SMALLINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) in (cast(1 as int)) FROM t +-- !query schema +struct<(CAST(1 AS INT) IN (CAST(1 AS INT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) in (cast(1 as bigint)) FROM t +-- !query schema +struct<(CAST(1 AS INT) IN (CAST(1 AS BIGINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) in (cast(1 as float)) FROM t +-- !query schema +struct<(CAST(1 AS INT) IN (CAST(1 AS FLOAT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) in (cast(1 as double)) FROM t +-- !query schema +struct<(CAST(1 AS INT) IN (CAST(1 AS DOUBLE))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) in (cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<(CAST(1 AS INT) IN (CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) in (cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS INT) IN (CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) in (cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 46, + "fragment" : "in (cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as int) in (cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 45, + "fragment" : "in (cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as int) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 69, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as int) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 62, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as tinyint)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) IN (CAST(1 AS TINYINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as smallint)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) IN (CAST(1 AS SMALLINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as int)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) IN (CAST(1 AS INT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as bigint)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as float)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) IN (CAST(1 AS FLOAT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as double)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) IN (CAST(1 AS DOUBLE))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) IN (CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) IN (CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) in (cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 49, + "fragment" : "in (cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 48, + "fragment" : "in (cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as bigint) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 72, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as bigint) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 65, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as float) in (cast(1 as tinyint)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) IN (CAST(1 AS TINYINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) in (cast(1 as smallint)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) IN (CAST(1 AS SMALLINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) in (cast(1 as int)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) IN (CAST(1 AS INT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) in (cast(1 as bigint)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) IN (CAST(1 AS BIGINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) in (cast(1 as float)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) in (cast(1 as double)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) IN (CAST(1 AS DOUBLE))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) in (cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) IN (CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) in (cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) IN (CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) in (cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 48, + "fragment" : "in (cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as float) in (cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 47, + "fragment" : "in (cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as float) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 71, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as float) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 64, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as double) in (cast(1 as tinyint)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS TINYINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) in (cast(1 as smallint)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS SMALLINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) in (cast(1 as int)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS INT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) in (cast(1 as bigint)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS BIGINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) in (cast(1 as float)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS FLOAT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) in (cast(1 as double)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) in (cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) in (cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) in (cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 49, + "fragment" : "in (cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as double) in (cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 48, + "fragment" : "in (cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as double) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 72, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as double) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 65, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as tinyint)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS TINYINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as smallint)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS SMALLINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as int)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS INT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as bigint)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS BIGINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as float)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS FLOAT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as double)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DOUBLE))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 57, + "fragment" : "in (cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 56, + "fragment" : "in (cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 80, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 73, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as string) in (cast(1 as tinyint)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS TINYINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as smallint)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS SMALLINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as int)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS INT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as bigint)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS BIGINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as float)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS FLOAT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as double)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS DOUBLE))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast('1' as binary)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS BINARY))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as boolean)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS BOOLEAN))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 72, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as string) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 65, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast(1 as tinyint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 50, + "fragment" : "in (cast(1 as tinyint))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast(1 as smallint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 51, + "fragment" : "in (cast(1 as smallint))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast(1 as int)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 46, + "fragment" : "in (cast(1 as int))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast(1 as bigint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 49, + "fragment" : "in (cast(1 as bigint))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast(1 as float)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 48, + "fragment" : "in (cast(1 as float))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast(1 as double)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 49, + "fragment" : "in (cast(1 as double))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"DECIMAL(10,0)\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 57, + "fragment" : "in (cast(1 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS BINARY) IN (CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary)) FROM t +-- !query schema +struct<(CAST(1 AS BINARY) IN (CAST(1 AS BINARY))):boolean> +-- !query output +true + + +-- !query +SELECT cast('1' as binary) in (cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 50, + "fragment" : "in (cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 74, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 67, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT true in (cast(1 as tinyint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 35, + "fragment" : "in (cast(1 as tinyint))" + } ] +} + + +-- !query +SELECT true in (cast(1 as smallint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 36, + "fragment" : "in (cast(1 as smallint))" + } ] +} + + +-- !query +SELECT true in (cast(1 as int)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 31, + "fragment" : "in (cast(1 as int))" + } ] +} + + +-- !query +SELECT true in (cast(1 as bigint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 34, + "fragment" : "in (cast(1 as bigint))" + } ] +} + + +-- !query +SELECT true in (cast(1 as float)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 33, + "fragment" : "in (cast(1 as float))" + } ] +} + + +-- !query +SELECT true in (cast(1 as double)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 34, + "fragment" : "in (cast(1 as double))" + } ] +} + + +-- !query +SELECT true in (cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"DECIMAL(10,0)\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 42, + "fragment" : "in (cast(1 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT true in (cast(1 as string)) FROM t +-- !query schema +struct<(true IN (CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT true in (cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 36, + "fragment" : "in (cast('1' as binary))" + } ] +} + + +-- !query +SELECT true in (cast(1 as boolean)) FROM t +-- !query schema +struct<(true IN (CAST(1 AS BOOLEAN))):boolean> +-- !query output +true + + +-- !query +SELECT true in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 59, + "fragment" : "in (cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT true in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(true IN (CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 13, + "stopIndex" : 52, + "fragment" : "in (cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as tinyint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 73, + "fragment" : "in (cast(2 as tinyint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as smallint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 74, + "fragment" : "in (cast(2 as smallint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as int)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 69, + "fragment" : "in (cast(2 as int))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as bigint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 72, + "fragment" : "in (cast(2 as bigint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as float)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 71, + "fragment" : "in (cast(2 as float))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as double)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 72, + "fragment" : "in (cast(2 as double))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as decimal(10, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"DECIMAL(10,0)\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 80, + "fragment" : "in (cast(2 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as string)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 72, + "fragment" : "in (cast(2 as string))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 74, + "fragment" : "in (cast('2' as binary))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast(2 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 73, + "fragment" : "in (cast(2 as boolean))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))):boolean> +-- !query output +false + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-11 09:30:00 AS DATE))):boolean> +-- !query output +false + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as tinyint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 66, + "fragment" : "in (cast(2 as tinyint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as smallint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 67, + "fragment" : "in (cast(2 as smallint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as int)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 62, + "fragment" : "in (cast(2 as int))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as bigint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 65, + "fragment" : "in (cast(2 as bigint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as float)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 64, + "fragment" : "in (cast(2 as float))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as double)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 65, + "fragment" : "in (cast(2 as double))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as decimal(10, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DECIMAL(10,0)\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 73, + "fragment" : "in (cast(2 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as string)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 65, + "fragment" : "in (cast(2 as string))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 67, + "fragment" : "in (cast('2' as binary))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast(2 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 66, + "fragment" : "in (cast(2 as boolean))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))):boolean> +-- !query output +false + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-11 09:30:00 AS DATE))):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as tinyint)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS TINYINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as smallint)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS SMALLINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as int)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS INT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as bigint)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS BIGINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as float)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS FLOAT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as double)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS DOUBLE))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"TINYINT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 70, + "fragment" : "in (cast(1 as tinyint), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"TINYINT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 69, + "fragment" : "in (cast(1 as tinyint), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"TINYINT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 93, + "fragment" : "in (cast(1 as tinyint), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) in (cast(1 as tinyint), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TINYINT\", \"TINYINT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS TINYINT) IN (CAST(1 AS TINYINT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 86, + "fragment" : "in (cast(1 as tinyint), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as tinyint)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS TINYINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as smallint)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS SMALLINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as int)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS INT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as bigint)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS BIGINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as float)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS FLOAT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as double)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS DOUBLE))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint), cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"SMALLINT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 72, + "fragment" : "in (cast(1 as smallint), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint), cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"SMALLINT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 71, + "fragment" : "in (cast(1 as smallint), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"SMALLINT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 95, + "fragment" : "in (cast(1 as smallint), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as smallint) in (cast(1 as smallint), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"SMALLINT\", \"SMALLINT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS SMALLINT) IN (CAST(1 AS SMALLINT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 88, + "fragment" : "in (cast(1 as smallint), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as int) in (cast(1 as int), cast(1 as tinyint)) FROM t +-- !query schema +struct<(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS TINYINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) in (cast(1 as int), cast(1 as smallint)) FROM t +-- !query schema +struct<(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS SMALLINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) in (cast(1 as int), cast(1 as int)) FROM t +-- !query schema +struct<(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS INT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) in (cast(1 as int), cast(1 as bigint)) FROM t +-- !query schema +struct<(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS BIGINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) in (cast(1 as int), cast(1 as float)) FROM t +-- !query schema +struct<(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS FLOAT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) in (cast(1 as int), cast(1 as double)) FROM t +-- !query schema +struct<(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS DOUBLE))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) in (cast(1 as int), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) in (cast(1 as int), cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) in (cast(1 as int), cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"INT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 62, + "fragment" : "in (cast(1 as int), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as int) in (cast(1 as int), cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"INT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 61, + "fragment" : "in (cast(1 as int), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as int) in (cast(1 as int), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"INT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 85, + "fragment" : "in (cast(1 as int), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as int) in (cast(1 as int), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"INT\", \"INT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS INT) IN (CAST(1 AS INT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 23, + "stopIndex" : 78, + "fragment" : "in (cast(1 as int), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as tinyint)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS TINYINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as smallint)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS SMALLINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as int)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS INT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as bigint)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS BIGINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as float)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS FLOAT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as double)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS DOUBLE))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as bigint), cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BIGINT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 68, + "fragment" : "in (cast(1 as bigint), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as bigint), cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BIGINT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 67, + "fragment" : "in (cast(1 as bigint), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as bigint), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BIGINT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 91, + "fragment" : "in (cast(1 as bigint), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as bigint) in (cast(1 as bigint), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BIGINT\", \"BIGINT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BIGINT) IN (CAST(1 AS BIGINT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 84, + "fragment" : "in (cast(1 as bigint), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as float) in (cast(1 as float), cast(1 as tinyint)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS TINYINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) in (cast(1 as float), cast(1 as smallint)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS SMALLINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) in (cast(1 as float), cast(1 as int)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS INT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) in (cast(1 as float), cast(1 as bigint)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS BIGINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) in (cast(1 as float), cast(1 as float)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS FLOAT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) in (cast(1 as float), cast(1 as double)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS DOUBLE))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) in (cast(1 as float), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) in (cast(1 as float), cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) in (cast(1 as float), cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"FLOAT\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 66, + "fragment" : "in (cast(1 as float), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as float) in (cast(1 as float), cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"FLOAT\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 65, + "fragment" : "in (cast(1 as float), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as float) in (cast(1 as float), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"FLOAT\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 89, + "fragment" : "in (cast(1 as float), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as float) in (cast(1 as float), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"FLOAT\", \"FLOAT\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS FLOAT) IN (CAST(1 AS FLOAT), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 25, + "stopIndex" : 82, + "fragment" : "in (cast(1 as float), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as double) in (cast(1 as double), cast(1 as tinyint)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS TINYINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) in (cast(1 as double), cast(1 as smallint)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS SMALLINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) in (cast(1 as double), cast(1 as int)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS INT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) in (cast(1 as double), cast(1 as bigint)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS BIGINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) in (cast(1 as double), cast(1 as float)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS FLOAT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) in (cast(1 as double), cast(1 as double)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS DOUBLE))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) in (cast(1 as double), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) in (cast(1 as double), cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) in (cast(1 as double), cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"DOUBLE\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 68, + "fragment" : "in (cast(1 as double), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as double) in (cast(1 as double), cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"DOUBLE\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 67, + "fragment" : "in (cast(1 as double), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as double) in (cast(1 as double), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"DOUBLE\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 91, + "fragment" : "in (cast(1 as double), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as double) in (cast(1 as double), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DOUBLE\", \"DOUBLE\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DOUBLE) IN (CAST(1 AS DOUBLE), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 84, + "fragment" : "in (cast(1 as double), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as tinyint)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(1 AS TINYINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as smallint)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(1 AS SMALLINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as int)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(1 AS INT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as bigint)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(1 AS BIGINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as float)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(1 AS FLOAT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as double)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(1 AS DOUBLE))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"DECIMAL(10,0)\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 84, + "fragment" : "in (cast(1 as decimal(10, 0)), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"DECIMAL(10,0)\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 83, + "fragment" : "in (cast(1 as decimal(10, 0)), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"DECIMAL(10,0)\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 107, + "fragment" : "in (cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) in (cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DECIMAL(10,0)\", \"DECIMAL(10,0)\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS DECIMAL(10,0)) IN (CAST(1 AS DECIMAL(10,0)), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 100, + "fragment" : "in (cast(1 as decimal(10, 0)), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as string) in (cast(1 as string), cast(1 as tinyint)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS TINYINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as string), cast(1 as smallint)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS SMALLINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as string), cast(1 as int)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS INT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as string), cast(1 as bigint)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS BIGINT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as string), cast(1 as float)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS FLOAT))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as string), cast(1 as double)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS DOUBLE))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as string), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as string), cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as string), cast('1' as binary)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS BINARY))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as string), cast(1 as boolean)) FROM t +-- !query schema +struct<(CAST(1 AS STRING) IN (CAST(1 AS STRING), CAST(1 AS BOOLEAN))):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as string) in (cast(1 as string), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 91, + "fragment" : "in (cast(1 as string), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast(1 as string) in (cast(1 as string), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 84, + "fragment" : "in (cast(1 as string), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as tinyint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 71, + "fragment" : "in (cast('1' as binary), cast(1 as tinyint))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as smallint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 72, + "fragment" : "in (cast('1' as binary), cast(1 as smallint))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as int)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 67, + "fragment" : "in (cast('1' as binary), cast(1 as int))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as bigint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 70, + "fragment" : "in (cast('1' as binary), cast(1 as bigint))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as float)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 69, + "fragment" : "in (cast('1' as binary), cast(1 as float))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as double)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 70, + "fragment" : "in (cast('1' as binary), cast(1 as double))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"DECIMAL(10,0)\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 78, + "fragment" : "in (cast('1' as binary), cast(1 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast('1' as binary)) FROM t +-- !query schema +struct<(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS BINARY))):boolean> +-- !query output +true + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 71, + "fragment" : "in (cast('1' as binary), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 95, + "fragment" : "in (cast('1' as binary), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast('1' as binary) in (cast('1' as binary), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BINARY\", \"BINARY\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BINARY) IN (CAST(1 AS BINARY), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 88, + "fragment" : "in (cast('1' as binary), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as tinyint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 73, + "fragment" : "in (cast('1' as boolean), cast(1 as tinyint))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as smallint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 74, + "fragment" : "in (cast('1' as boolean), cast(1 as smallint))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as int)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 69, + "fragment" : "in (cast('1' as boolean), cast(1 as int))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as bigint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 72, + "fragment" : "in (cast('1' as boolean), cast(1 as bigint))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as float)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 71, + "fragment" : "in (cast('1' as boolean), cast(1 as float))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as double)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 72, + "fragment" : "in (cast('1' as boolean), cast(1 as double))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"DECIMAL(10,0)\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 80, + "fragment" : "in (cast('1' as boolean), cast(1 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as string)) FROM t +-- !query schema +struct<(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 74, + "fragment" : "in (cast('1' as boolean), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast(1 as boolean)) FROM t +-- !query schema +struct<(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(1 AS BOOLEAN))):boolean> +-- !query output +true + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"TIMESTAMP\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 97, + "fragment" : "in (cast('1' as boolean), cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT cast('1' as boolean) in (cast('1' as boolean), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"BOOLEAN\", \"BOOLEAN\", \"DATE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) IN (CAST(1 AS BOOLEAN), CAST(2017-12-11 09:30:00 AS DATE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 29, + "stopIndex" : 90, + "fragment" : "in (cast('1' as boolean), cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as tinyint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TIMESTAMP\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 117, + "fragment" : "in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as tinyint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as smallint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TIMESTAMP\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 118, + "fragment" : "in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as smallint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as int)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TIMESTAMP\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 113, + "fragment" : "in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as int))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as bigint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TIMESTAMP\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 116, + "fragment" : "in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as bigint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as float)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TIMESTAMP\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 115, + "fragment" : "in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as float))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as double)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TIMESTAMP\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 116, + "fragment" : "in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as double))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TIMESTAMP\", \"DECIMAL(10,0)\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 124, + "fragment" : "in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as string)) FROM t +-- !query schema +struct<(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TIMESTAMP\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 118, + "fragment" : "in (cast('2017-12-12 09:30:00.0' as timestamp), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"TIMESTAMP\", \"TIMESTAMP\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 117, + "fragment" : "in (cast('2017-12-12 09:30:00.0' as timestamp), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))):boolean> +-- !query output +true + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) in (cast('2017-12-12 09:30:00.0' as timestamp), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<(CAST(2017-12-12 09:30:00.0 AS TIMESTAMP) IN (CAST(2017-12-12 09:30:00.0 AS TIMESTAMP), CAST(2017-12-11 09:30:00 AS DATE))):boolean> +-- !query output +true + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as tinyint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DATE\", \"TINYINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS TINYINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 103, + "fragment" : "in (cast('2017-12-12 09:30:00' as date), cast(1 as tinyint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as smallint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DATE\", \"SMALLINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS SMALLINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 104, + "fragment" : "in (cast('2017-12-12 09:30:00' as date), cast(1 as smallint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as int)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DATE\", \"INT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS INT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 99, + "fragment" : "in (cast('2017-12-12 09:30:00' as date), cast(1 as int))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as bigint)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DATE\", \"BIGINT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS BIGINT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 102, + "fragment" : "in (cast('2017-12-12 09:30:00' as date), cast(1 as bigint))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as float)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DATE\", \"FLOAT\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS FLOAT)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 101, + "fragment" : "in (cast('2017-12-12 09:30:00' as date), cast(1 as float))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as double)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DATE\", \"DOUBLE\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS DOUBLE)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 102, + "fragment" : "in (cast('2017-12-12 09:30:00' as date), cast(1 as double))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DATE\", \"DECIMAL(10,0)\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS DECIMAL(10,0))))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 110, + "fragment" : "in (cast('2017-12-12 09:30:00' as date), cast(1 as decimal(10, 0)))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as string)) FROM t +-- !query schema +struct<(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS STRING))):boolean> +-- !query output +true + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DATE\", \"BINARY\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS BINARY)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 104, + "fragment" : "in (cast('2017-12-12 09:30:00' as date), cast('1' as binary))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "[\"DATE\", \"DATE\", \"BOOLEAN\"]", + "functionName" : "`in`", + "sqlExpr" : "\"(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(1 AS BOOLEAN)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 103, + "fragment" : "in (cast('2017-12-12 09:30:00' as date), cast(1 as boolean))" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))):boolean> +-- !query output +true + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) in (cast('2017-12-12 09:30:00' as date), cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<(CAST(2017-12-12 09:30:00 AS DATE) IN (CAST(2017-12-12 09:30:00 AS DATE), CAST(2017-12-11 09:30:00 AS DATE))):boolean> +-- !query output +true diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/mapZipWith.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/mapZipWith.sql.out new file mode 100644 index 000000000000..a0bd111f6ba6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/mapZipWith.sql.out @@ -0,0 +1,242 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES ( + map(true, false), + map(2Y, 1Y), + map(2S, 1S), + map(2, 1), + map(2L, 1L), + map(922337203685477897945456575809789456, 922337203685477897945456575809789456), + map(9.22337203685477897945456575809789456, 9.22337203685477897945456575809789456), + map(2.0D, 1.0D), + map(float(2.0), float(1.0)), + map(date '2016-03-14', date '2016-03-13'), + map(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), + map('true', 'false', '2', '1'), + map('2016-03-14', '2016-03-13'), + map('2016-11-15 20:54:00.000', '2016-11-12 20:54:00.000'), + map('922337203685477897945456575809789456', 'text'), + map(array(1L, 2L), array(1L, 2L)), map(array(1, 2), array(1, 2)), + map(struct(1S, 2L), struct(1S, 2L)), map(struct(1, 2), struct(1, 2)) +) AS various_maps( + boolean_map, + tinyint_map, + smallint_map, + int_map, + bigint_map, + decimal_map1, decimal_map2, + double_map, + float_map, + date_map, + timestamp_map, + string_map1, string_map2, string_map3, string_map4, + array_map1, array_map2, + struct_map1, struct_map2 +) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT map_zip_with(tinyint_map, smallint_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query schema +struct>> +-- !query output +{2:{"k":2,"v1":1,"v2":1}} + + +-- !query +SELECT map_zip_with(smallint_map, int_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query schema +struct>> +-- !query output +{2:{"k":2,"v1":1,"v2":1}} + + +-- !query +SELECT map_zip_with(int_map, bigint_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query schema +struct>> +-- !query output +{2:{"k":2,"v1":1,"v2":1}} + + +-- !query +SELECT map_zip_with(double_map, float_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query schema +struct>> +-- !query output +{2.0:{"k":2.0,"v1":1.0,"v2":1.0}} + + +-- !query +SELECT map_zip_with(decimal_map1, decimal_map2, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query schema +struct>> +-- !query output +{9.22:{"k":9.22,"v1":null,"v2":9.22337203685477897945456575809789456},922337203685477897945456575809789456.00:{"k":922337203685477897945456575809789456.00,"v1":922337203685477897945456575809789456,"v2":null}} + + +-- !query +SELECT map_zip_with(decimal_map1, int_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query schema +struct>> +-- !query output +{2:{"k":2,"v1":null,"v2":1},922337203685477897945456575809789456:{"k":922337203685477897945456575809789456,"v1":922337203685477897945456575809789456,"v2":null}} + + +-- !query +SELECT map_zip_with(decimal_map1, double_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query schema +struct>> +-- !query output +{2.0:{"k":2.0,"v1":null,"v2":1.0},9.223372036854779E35:{"k":9.223372036854779E35,"v1":922337203685477897945456575809789456,"v2":null}} + + +-- !query +SELECT map_zip_with(decimal_map2, int_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query schema +struct>> +-- !query output +{2.0000000000000000000000000000:{"k":2.0000000000000000000000000000,"v1":null,"v2":1},9.2233720368547789794545657581:{"k":9.2233720368547789794545657581,"v1":9.22337203685477897945456575809789456,"v2":null}} + + +-- !query +SELECT map_zip_with(decimal_map2, double_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query schema +struct>> +-- !query output +{2.0:{"k":2.0,"v1":null,"v2":1.0},9.223372036854778:{"k":9.223372036854778,"v1":9.22337203685477897945456575809789456,"v2":null}} + + +-- !query +SELECT map_zip_with(string_map1, int_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`map_zip_with`", + "leftType" : "\"STRING\"", + "rightType" : "\"INT\"", + "sqlExpr" : "\"map_zip_with(string_map1, int_map, lambdafunction(struct(k, v1, v2), k, v1, v2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 75, + "fragment" : "map_zip_with(string_map1, int_map, (k, v1, v2) -> struct(k, v1, v2))" + } ] +} + + +-- !query +SELECT map_zip_with(string_map2, date_map, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`map_zip_with`", + "leftType" : "\"STRING\"", + "rightType" : "\"DATE\"", + "sqlExpr" : "\"map_zip_with(string_map2, date_map, lambdafunction(struct(k, v1, v2), k, v1, v2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 76, + "fragment" : "map_zip_with(string_map2, date_map, (k, v1, v2) -> struct(k, v1, v2))" + } ] +} + + +-- !query +SELECT map_zip_with(timestamp_map, string_map3, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`map_zip_with`", + "leftType" : "\"TIMESTAMP\"", + "rightType" : "\"STRING\"", + "sqlExpr" : "\"map_zip_with(timestamp_map, string_map3, lambdafunction(struct(k, v1, v2), k, v1, v2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 81, + "fragment" : "map_zip_with(timestamp_map, string_map3, (k, v1, v2) -> struct(k, v1, v2))" + } ] +} + + +-- !query +SELECT map_zip_with(decimal_map1, string_map4, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.MAP_ZIP_WITH_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "functionName" : "`map_zip_with`", + "leftType" : "\"DECIMAL(36,0)\"", + "rightType" : "\"STRING\"", + "sqlExpr" : "\"map_zip_with(decimal_map1, string_map4, lambdafunction(struct(k, v1, v2), k, v1, v2))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 80, + "fragment" : "map_zip_with(decimal_map1, string_map4, (k, v1, v2) -> struct(k, v1, v2))" + } ] +} + + +-- !query +SELECT map_zip_with(array_map1, array_map2, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query schema +struct,struct,v1:array,v2:array>>> +-- !query output +{[1,2]:{"k":[1,2],"v1":[1,2],"v2":[1,2]}} + + +-- !query +SELECT map_zip_with(struct_map1, struct_map2, (k, v1, v2) -> struct(k, v1, v2)) m +FROM various_maps +-- !query schema +struct,struct,v1:struct,v2:struct>>> +-- !query output +{{"col1":1,"col2":2}:{"k":{"col1":1,"col2":2},"v1":{"col1":1,"col2":2},"v2":{"col1":1,"col2":2}}} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/mapconcat.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/mapconcat.sql.out new file mode 100644 index 000000000000..893e9b511986 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/mapconcat.sql.out @@ -0,0 +1,229 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES ( + map(true, false), map(false, true), + map(1Y, 2Y), map(3Y, 4Y), + map(1S, 2S), map(3S, 4S), + map(4, 6), map(7, 8), + map(6L, 7L), map(8L, 9L), + map(9223372036854775809, 9223372036854775808), map(9223372036854775808, 9223372036854775809), + map(1.0D, 2.0D), map(3.0D, 4.0D), + map(float(1.0D), float(2.0D)), map(float(3.0D), float(4.0D)), + map(date '2016-03-14', date '2016-03-13'), map(date '2016-03-12', date '2016-03-11'), + map(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), + map(timestamp '2016-11-11 20:54:00.000', timestamp '2016-11-09 20:54:00.000'), + map('a', 'b'), map('c', 'd'), + map(array('a', 'b'), array('c', 'd')), map(array('e'), array('f')), + map(struct('a', 1), struct('b', 2)), map(struct('c', 3), struct('d', 4)), + map('a', 1), map('c', 2), + map(1, 'a'), map(2, 'c') +) AS various_maps ( + boolean_map1, boolean_map2, + tinyint_map1, tinyint_map2, + smallint_map1, smallint_map2, + int_map1, int_map2, + bigint_map1, bigint_map2, + decimal_map1, decimal_map2, + double_map1, double_map2, + float_map1, float_map2, + date_map1, date_map2, + timestamp_map1, + timestamp_map2, + string_map1, string_map2, + array_map1, array_map2, + struct_map1, struct_map2, + string_int_map1, string_int_map2, + int_string_map1, int_string_map2 +) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + map_concat(boolean_map1, boolean_map2) boolean_map, + map_concat(tinyint_map1, tinyint_map2) tinyint_map, + map_concat(smallint_map1, smallint_map2) smallint_map, + map_concat(int_map1, int_map2) int_map, + map_concat(bigint_map1, bigint_map2) bigint_map, + map_concat(decimal_map1, decimal_map2) decimal_map, + map_concat(float_map1, float_map2) float_map, + map_concat(double_map1, double_map2) double_map, + map_concat(date_map1, date_map2) date_map, + map_concat(timestamp_map1, timestamp_map2) timestamp_map, + map_concat(string_map1, string_map2) string_map, + map_concat(array_map1, array_map2) array_map, + map_concat(struct_map1, struct_map2) struct_map, + map_concat(string_int_map1, string_int_map2) string_int_map, + map_concat(int_string_map1, int_string_map2) int_string_map +FROM various_maps +-- !query schema +struct,tinyint_map:map,smallint_map:map,int_map:map,bigint_map:map,decimal_map:map,float_map:map,double_map:map,date_map:map,timestamp_map:map,string_map:map,array_map:map,array>,struct_map:map,struct>,string_int_map:map,int_string_map:map> +-- !query output +{false:true,true:false} {1:2,3:4} {1:2,3:4} {4:6,7:8} {6:7,8:9} {9223372036854775808:9223372036854775809,9223372036854775809:9223372036854775808} {1.0:2.0,3.0:4.0} {1.0:2.0,3.0:4.0} {2016-03-12:2016-03-11,2016-03-14:2016-03-13} {2016-11-11 20:54:00:2016-11-09 20:54:00,2016-11-15 20:54:00:2016-11-12 20:54:00} {"a":"b","c":"d"} {["a","b"]:["c","d"],["e"]:["f"]} {{"col1":"a","col2":1}:{"col1":"b","col2":2},{"col1":"c","col2":3}:{"col1":"d","col2":4}} {"a":1,"c":2} {1:"a",2:"c"} + + +-- !query +SELECT + map_concat(tinyint_map1, smallint_map2) ts_map, + map_concat(smallint_map1, int_map2) si_map, + map_concat(int_map1, bigint_map2) ib_map, + map_concat(bigint_map1, decimal_map2) bd_map, + map_concat(decimal_map1, float_map2) df_map, + map_concat(string_map1, date_map2) std_map, + map_concat(timestamp_map1, string_map2) tst_map, + map_concat(string_map1, int_map2) sti_map, + map_concat(int_string_map1, tinyint_map2) istt_map +FROM various_maps +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"MAP\" or \"MAP\")", + "functionName" : "`map_concat`", + "sqlExpr" : "\"map_concat(string_map1, date_map2)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 257, + "stopIndex" : 290, + "fragment" : "map_concat(string_map1, date_map2)" + } ] +} + + +-- !query +SELECT + map_concat(tinyint_map1, array_map1) tm_map +FROM various_maps +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"MAP\" or \"MAP, ARRAY>\")", + "functionName" : "`map_concat`", + "sqlExpr" : "\"map_concat(tinyint_map1, array_map1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 47, + "fragment" : "map_concat(tinyint_map1, array_map1)" + } ] +} + + +-- !query +SELECT + map_concat(boolean_map1, int_map2) bi_map +FROM various_maps +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"MAP\" or \"MAP\")", + "functionName" : "`map_concat`", + "sqlExpr" : "\"map_concat(boolean_map1, int_map2)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 45, + "fragment" : "map_concat(boolean_map1, int_map2)" + } ] +} + + +-- !query +SELECT + map_concat(int_map1, struct_map2) is_map +FROM various_maps +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"MAP\" or \"MAP, STRUCT>\")", + "functionName" : "`map_concat`", + "sqlExpr" : "\"map_concat(int_map1, struct_map2)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 44, + "fragment" : "map_concat(int_map1, struct_map2)" + } ] +} + + +-- !query +SELECT + map_concat(struct_map1, array_map2) ma_map +FROM various_maps +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"MAP, STRUCT>\" or \"MAP, ARRAY>\")", + "functionName" : "`map_concat`", + "sqlExpr" : "\"map_concat(struct_map1, array_map2)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 46, + "fragment" : "map_concat(struct_map1, array_map2)" + } ] +} + + +-- !query +SELECT + map_concat(int_map1, array_map2) ms_map +FROM various_maps +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "dataType" : "(\"MAP\" or \"MAP, ARRAY>\")", + "functionName" : "`map_concat`", + "sqlExpr" : "\"map_concat(int_map1, array_map2)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 43, + "fragment" : "map_concat(int_map1, array_map2)" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/promoteStrings.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/promoteStrings.sql.out new file mode 100644 index 000000000000..6fb21f001447 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/promoteStrings.sql.out @@ -0,0 +1,3875 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT '1' + cast(1 as tinyint) FROM t +-- !query schema +struct<(1 + CAST(1 AS TINYINT)):bigint> +-- !query output +2 + + +-- !query +SELECT '1' + cast(1 as smallint) FROM t +-- !query schema +struct<(1 + CAST(1 AS SMALLINT)):bigint> +-- !query output +2 + + +-- !query +SELECT '1' + cast(1 as int) FROM t +-- !query schema +struct<(1 + CAST(1 AS INT)):bigint> +-- !query output +2 + + +-- !query +SELECT '1' + cast(1 as bigint) FROM t +-- !query schema +struct<(1 + CAST(1 AS BIGINT)):bigint> +-- !query output +2 + + +-- !query +SELECT '1' + cast(1 as float) FROM t +-- !query schema +struct<(1 + CAST(1 AS FLOAT)):double> +-- !query output +2.0 + + +-- !query +SELECT '1' + cast(1 as double) FROM t +-- !query schema +struct<(1 + CAST(1 AS DOUBLE)):double> +-- !query output +2.0 + + +-- !query +SELECT '1' + cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(1 + CAST(1 AS DECIMAL(10,0))):double> +-- !query output +2.0 + + +-- !query +SELECT '1' + '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(1 + 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "'1' + '1'" + } ] +} + + +-- !query +SELECT '1' + cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(1 + CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "'1' + cast('1' as binary)" + } ] +} + + +-- !query +SELECT '1' + cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(1 + CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "'1' + cast(1 as boolean)" + } ] +} + + +-- !query +SELECT '1' + cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(1 + CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "'1' + cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT '1' + cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"CAST(2017-12-11 09:30:00 AS DATE)\"", + "inputType" : "\"DATE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(1, CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "'1' + cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT '1' - cast(1 as tinyint) FROM t +-- !query schema +struct<(1 - CAST(1 AS TINYINT)):bigint> +-- !query output +0 + + +-- !query +SELECT '1' - cast(1 as smallint) FROM t +-- !query schema +struct<(1 - CAST(1 AS SMALLINT)):bigint> +-- !query output +0 + + +-- !query +SELECT '1' - cast(1 as int) FROM t +-- !query schema +struct<(1 - CAST(1 AS INT)):bigint> +-- !query output +0 + + +-- !query +SELECT '1' - cast(1 as bigint) FROM t +-- !query schema +struct<(1 - CAST(1 AS BIGINT)):bigint> +-- !query output +0 + + +-- !query +SELECT '1' - cast(1 as float) FROM t +-- !query schema +struct<(1 - CAST(1 AS FLOAT)):double> +-- !query output +0.0 + + +-- !query +SELECT '1' - cast(1 as double) FROM t +-- !query schema +struct<(1 - CAST(1 AS DOUBLE)):double> +-- !query output +0.0 + + +-- !query +SELECT '1' - cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(1 - CAST(1 AS DECIMAL(10,0))):double> +-- !query output +0.0 + + +-- !query +SELECT '1' - '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(1 - 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "'1' - '1'" + } ] +} + + +-- !query +SELECT '1' - cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(1 - CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "'1' - cast('1' as binary)" + } ] +} + + +-- !query +SELECT '1' - cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(1 - CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "'1' - cast(1 as boolean)" + } ] +} + + +-- !query +SELECT '1' - cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "'1' - cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT '1' - cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "'1' - cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT '1' * cast(1 as tinyint) FROM t +-- !query schema +struct<(1 * CAST(1 AS TINYINT)):bigint> +-- !query output +1 + + +-- !query +SELECT '1' * cast(1 as smallint) FROM t +-- !query schema +struct<(1 * CAST(1 AS SMALLINT)):bigint> +-- !query output +1 + + +-- !query +SELECT '1' * cast(1 as int) FROM t +-- !query schema +struct<(1 * CAST(1 AS INT)):bigint> +-- !query output +1 + + +-- !query +SELECT '1' * cast(1 as bigint) FROM t +-- !query schema +struct<(1 * CAST(1 AS BIGINT)):bigint> +-- !query output +1 + + +-- !query +SELECT '1' * cast(1 as float) FROM t +-- !query schema +struct<(1 * CAST(1 AS FLOAT)):double> +-- !query output +1.0 + + +-- !query +SELECT '1' * cast(1 as double) FROM t +-- !query schema +struct<(1 * CAST(1 AS DOUBLE)):double> +-- !query output +1.0 + + +-- !query +SELECT '1' * cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(1 * CAST(1 AS DECIMAL(10,0))):double> +-- !query output +1.0 + + +-- !query +SELECT '1' * '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(1 * 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "'1' * '1'" + } ] +} + + +-- !query +SELECT '1' * cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BINARY\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(1 * CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "'1' * cast('1' as binary)" + } ] +} + + +-- !query +SELECT '1' * cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(1 * CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "'1' * cast(1 as boolean)" + } ] +} + + +-- !query +SELECT '1' * cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(1 * CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "'1' * cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT '1' * cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"DATE\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(1 * CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "'1' * cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT '1' / cast(1 as tinyint) FROM t +-- !query schema +struct<(1 / CAST(1 AS TINYINT)):double> +-- !query output +1.0 + + +-- !query +SELECT '1' / cast(1 as smallint) FROM t +-- !query schema +struct<(1 / CAST(1 AS SMALLINT)):double> +-- !query output +1.0 + + +-- !query +SELECT '1' / cast(1 as int) FROM t +-- !query schema +struct<(1 / CAST(1 AS INT)):double> +-- !query output +1.0 + + +-- !query +SELECT '1' / cast(1 as bigint) FROM t +-- !query schema +struct<(1 / CAST(1 AS BIGINT)):double> +-- !query output +1.0 + + +-- !query +SELECT '1' / cast(1 as float) FROM t +-- !query schema +struct<(1 / CAST(1 AS FLOAT)):double> +-- !query output +1.0 + + +-- !query +SELECT '1' / cast(1 as double) FROM t +-- !query schema +struct<(1 / CAST(1 AS DOUBLE)):double> +-- !query output +1.0 + + +-- !query +SELECT '1' / cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(1 / CAST(1 AS DECIMAL(10,0))):double> +-- !query output +1.0 + + +-- !query +SELECT '1' / '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(1 / 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "'1' / '1'" + } ] +} + + +-- !query +SELECT '1' / cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(1 / CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "'1' / cast('1' as binary)" + } ] +} + + +-- !query +SELECT '1' / cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(1 / CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "'1' / cast(1 as boolean)" + } ] +} + + +-- !query +SELECT '1' / cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(1 / CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "'1' / cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT '1' / cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"DATE\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(1 / CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "'1' / cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT '1' % cast(1 as tinyint) FROM t +-- !query schema +struct<(1 % CAST(1 AS TINYINT)):bigint> +-- !query output +0 + + +-- !query +SELECT '1' % cast(1 as smallint) FROM t +-- !query schema +struct<(1 % CAST(1 AS SMALLINT)):bigint> +-- !query output +0 + + +-- !query +SELECT '1' % cast(1 as int) FROM t +-- !query schema +struct<(1 % CAST(1 AS INT)):bigint> +-- !query output +0 + + +-- !query +SELECT '1' % cast(1 as bigint) FROM t +-- !query schema +struct<(1 % CAST(1 AS BIGINT)):bigint> +-- !query output +0 + + +-- !query +SELECT '1' % cast(1 as float) FROM t +-- !query schema +struct<(1 % CAST(1 AS FLOAT)):double> +-- !query output +0.0 + + +-- !query +SELECT '1' % cast(1 as double) FROM t +-- !query schema +struct<(1 % CAST(1 AS DOUBLE)):double> +-- !query output +0.0 + + +-- !query +SELECT '1' % cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(1 % CAST(1 AS DECIMAL(10,0))):double> +-- !query output +0.0 + + +-- !query +SELECT '1' % '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(1 % 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "'1' % '1'" + } ] +} + + +-- !query +SELECT '1' % cast('1' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BINARY\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(1 % CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 32, + "fragment" : "'1' % cast('1' as binary)" + } ] +} + + +-- !query +SELECT '1' % cast(1 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(1 % CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "'1' % cast(1 as boolean)" + } ] +} + + +-- !query +SELECT '1' % cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(1 % CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "'1' % cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT '1' % cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"DATE\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(1 % CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "'1' % cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT pmod('1', cast(1 as tinyint)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod('1', cast(1 as smallint)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod('1', cast(1 as int)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod('1', cast(1 as bigint)) FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod('1', cast(1 as float)) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod('1', cast(1 as double)) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod('1', cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod('1', '1') FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"STRING\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"pmod(1, 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "pmod('1', '1')" + } ] +} + + +-- !query +SELECT pmod('1', cast('1' as binary)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BINARY\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"pmod(1, CAST(1 AS BINARY))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "pmod('1', cast('1' as binary))" + } ] +} + + +-- !query +SELECT pmod('1', cast(1 as boolean)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"pmod(1, CAST(1 AS BOOLEAN))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "pmod('1', cast(1 as boolean))" + } ] +} + + +-- !query +SELECT pmod('1', cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"pmod(1, CAST(2017-12-11 09:30:00.0 AS TIMESTAMP))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 60, + "fragment" : "pmod('1', cast('2017-12-11 09:30:00.0' as timestamp))" + } ] +} + + +-- !query +SELECT pmod('1', cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"DATE\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"pmod(1, CAST(2017-12-11 09:30:00 AS DATE))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "pmod('1', cast('2017-12-11 09:30:00' as date))" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) + '1' FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) + 1):bigint> +-- !query output +2 + + +-- !query +SELECT cast(1 as smallint) + '1' FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) + 1):bigint> +-- !query output +2 + + +-- !query +SELECT cast(1 as int) + '1' FROM t +-- !query schema +struct<(CAST(1 AS INT) + 1):bigint> +-- !query output +2 + + +-- !query +SELECT cast(1 as bigint) + '1' FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) + 1):bigint> +-- !query output +2 + + +-- !query +SELECT cast(1 as float) + '1' FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) + 1):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as double) + '1' FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) + 1):double> +-- !query output +2.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) + '1' FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) + 1):double> +-- !query output +2.0 + + +-- !query +SELECT cast('1' as binary) + '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(CAST(1 AS BINARY) + 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) + '1'" + } ] +} + + +-- !query +SELECT cast(1 as boolean) + '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) + 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as boolean) + '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) + '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) + 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) + '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) + '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"1\"", + "inputType" : "\"DATE\"", + "paramIndex" : "second", + "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")", + "sqlExpr" : "\"date_add(CAST(2017-12-11 09:30:00 AS DATE), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00' as date) + '1'" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) - '1' FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) - 1):bigint> +-- !query output +0 + + +-- !query +SELECT cast(1 as smallint) - '1' FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) - 1):bigint> +-- !query output +0 + + +-- !query +SELECT cast(1 as int) - '1' FROM t +-- !query schema +struct<(CAST(1 AS INT) - 1):bigint> +-- !query output +0 + + +-- !query +SELECT cast(1 as bigint) - '1' FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) - 1):bigint> +-- !query output +0 + + +-- !query +SELECT cast(1 as float) - '1' FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) - 1):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as double) - '1' FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) - 1):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) - '1' FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) - 1):double> +-- !query output +0.0 + + +-- !query +SELECT cast('1' as binary) - '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(CAST(1 AS BINARY) - 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) - '1'" + } ] +} + + +-- !query +SELECT cast(1 as boolean) - '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) - 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as boolean) - '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) - '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) - '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) - '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00' as date) - '1'" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) * '1' FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) * 1):bigint> +-- !query output +1 + + +-- !query +SELECT cast(1 as smallint) * '1' FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) * 1):bigint> +-- !query output +1 + + +-- !query +SELECT cast(1 as int) * '1' FROM t +-- !query schema +struct<(CAST(1 AS INT) * 1):bigint> +-- !query output +1 + + +-- !query +SELECT cast(1 as bigint) * '1' FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) * 1):bigint> +-- !query output +1 + + +-- !query +SELECT cast(1 as float) * '1' FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) * 1):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as double) * '1' FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) * 1):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) * '1' FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) * 1):double> +-- !query output +1.0 + + +-- !query +SELECT cast('1' as binary) * '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BINARY\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) * 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) * '1'" + } ] +} + + +-- !query +SELECT cast(1 as boolean) * '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) * 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as boolean) * '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) * '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) * 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) * '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) * '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"DATE\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) * 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00' as date) * '1'" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) / '1' FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) / 1):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as smallint) / '1' FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) / 1):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as int) / '1' FROM t +-- !query schema +struct<(CAST(1 AS INT) / 1):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as bigint) / '1' FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) / 1):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as float) / '1' FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) / 1):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as double) / '1' FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) / 1):double> +-- !query output +1.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) / '1' FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) / 1):double> +-- !query output +1.0 + + +-- !query +SELECT cast('1' as binary) / '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BINARY\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(1 AS BINARY) / 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) / '1'" + } ] +} + + +-- !query +SELECT cast(1 as boolean) / '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) / 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as boolean) / '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) / '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) / 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) / '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) / '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"DATE\"", + "inputType" : "(\"DOUBLE\" or \"DECIMAL\")", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) / 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00' as date) / '1'" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) % '1' FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) % 1):bigint> +-- !query output +0 + + +-- !query +SELECT cast(1 as smallint) % '1' FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) % 1):bigint> +-- !query output +0 + + +-- !query +SELECT cast(1 as int) % '1' FROM t +-- !query schema +struct<(CAST(1 AS INT) % 1):bigint> +-- !query output +0 + + +-- !query +SELECT cast(1 as bigint) % '1' FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) % 1):bigint> +-- !query output +0 + + +-- !query +SELECT cast(1 as float) % '1' FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) % 1):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as double) % '1' FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) % 1):double> +-- !query output +0.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) % '1' FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) % 1):double> +-- !query output +0.0 + + +-- !query +SELECT cast('1' as binary) % '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BINARY\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(CAST(1 AS BINARY) % 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('1' as binary) % '1'" + } ] +} + + +-- !query +SELECT cast(1 as boolean) % '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(CAST(1 AS BOOLEAN) % 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast(1 as boolean) % '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) % '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) % 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) % '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) % '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"DATE\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"(CAST(2017-12-11 09:30:00 AS DATE) % 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00' as date) % '1'" + } ] +} + + +-- !query +SELECT pmod(cast(1 as tinyint), '1') FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as smallint), '1') FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as int), '1') FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as bigint), '1') FROM t +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT pmod(cast(1 as float), '1') FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as double), '1') FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast(1 as decimal(10, 0)), '1') FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT pmod(cast('1' as binary), '1') FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BINARY\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"pmod(CAST(1 AS BINARY), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "pmod(cast('1' as binary), '1')" + } ] +} + + +-- !query +SELECT pmod(cast(1 as boolean), '1') FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"BOOLEAN\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"pmod(CAST(1 AS BOOLEAN), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 36, + "fragment" : "pmod(cast(1 as boolean), '1')" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00.0' as timestamp), '1') FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"TIMESTAMP\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 60, + "fragment" : "pmod(cast('2017-12-11 09:30:00.0' as timestamp), '1')" + } ] +} + + +-- !query +SELECT pmod(cast('2017-12-11 09:30:00' as date), '1') FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "actualDataType" : "\"DATE\"", + "inputType" : "\"NUMERIC\"", + "sqlExpr" : "\"pmod(CAST(2017-12-11 09:30:00 AS DATE), 1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 53, + "fragment" : "pmod(cast('2017-12-11 09:30:00' as date), '1')" + } ] +} + + +-- !query +SELECT '1' = cast(1 as tinyint) FROM t +-- !query schema +struct<(1 = CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' = cast(1 as smallint) FROM t +-- !query schema +struct<(1 = CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' = cast(1 as int) FROM t +-- !query schema +struct<(1 = CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' = cast(1 as bigint) FROM t +-- !query schema +struct<(1 = CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' = cast(1 as float) FROM t +-- !query schema +struct<(1 = CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' = cast(1 as double) FROM t +-- !query schema +struct<(1 = CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT '1' = cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(1 = CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT '1' = '1' FROM t +-- !query schema +struct<(1 = 1):boolean> +-- !query output +true + + +-- !query +SELECT '1' = cast('1' as binary) FROM t +-- !query schema +struct<(1 = CAST(1 AS BINARY)):boolean> +-- !query output +true + + +-- !query +SELECT '1' = cast(1 as boolean) FROM t +-- !query schema +struct<(1 = CAST(1 AS BOOLEAN)):boolean> +-- !query output +true + + +-- !query +SELECT '1' = cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "'1' = cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT '1' = cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "'1' = cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) = '1' FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) = 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) = '1' FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) = 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) = '1' FROM t +-- !query schema +struct<(CAST(1 AS INT) = 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) = '1' FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) = 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) = '1' FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) = 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) = '1' FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) = 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) = '1' FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) = 1):boolean> +-- !query output +true + + +-- !query +SELECT cast('1' as binary) = '1' FROM t +-- !query schema +struct<(CAST(1 AS BINARY) = 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as boolean) = '1' FROM t +-- !query schema +struct<(CAST(1 AS BOOLEAN) = 1):boolean> +-- !query output +true + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) = '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) = '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) = '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00' as date) = '1'" + } ] +} + + +-- !query +SELECT '1' <=> cast(1 as tinyint) FROM t +-- !query schema +struct<(1 <=> CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' <=> cast(1 as smallint) FROM t +-- !query schema +struct<(1 <=> CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' <=> cast(1 as int) FROM t +-- !query schema +struct<(1 <=> CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' <=> cast(1 as bigint) FROM t +-- !query schema +struct<(1 <=> CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' <=> cast(1 as float) FROM t +-- !query schema +struct<(1 <=> CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' <=> cast(1 as double) FROM t +-- !query schema +struct<(1 <=> CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT '1' <=> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(1 <=> CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT '1' <=> '1' FROM t +-- !query schema +struct<(1 <=> 1):boolean> +-- !query output +true + + +-- !query +SELECT '1' <=> cast('1' as binary) FROM t +-- !query schema +struct<(1 <=> CAST(1 AS BINARY)):boolean> +-- !query output +true + + +-- !query +SELECT '1' <=> cast(1 as boolean) FROM t +-- !query schema +struct<(1 <=> CAST(1 AS BOOLEAN)):boolean> +-- !query output +true + + +-- !query +SELECT '1' <=> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "'1' <=> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT '1' <=> cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "'1' <=> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) <=> '1' FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) <=> 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) <=> '1' FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) <=> 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) <=> '1' FROM t +-- !query schema +struct<(CAST(1 AS INT) <=> 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) <=> '1' FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) <=> 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) <=> '1' FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) <=> 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) <=> '1' FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) <=> 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <=> '1' FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <=> 1):boolean> +-- !query output +true + + +-- !query +SELECT cast('1' as binary) <=> '1' FROM t +-- !query schema +struct<(CAST(1 AS BINARY) <=> 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as boolean) <=> '1' FROM t +-- !query schema +struct<(CAST(1 AS BOOLEAN) <=> 1):boolean> +-- !query output +true + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <=> '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <=> '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <=> '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 57, + "fragment" : "cast('2017-12-11 09:30:00' as date) <=> '1'" + } ] +} + + +-- !query +SELECT '1' < cast(1 as tinyint) FROM t +-- !query schema +struct<(1 < CAST(1 AS TINYINT)):boolean> +-- !query output +false + + +-- !query +SELECT '1' < cast(1 as smallint) FROM t +-- !query schema +struct<(1 < CAST(1 AS SMALLINT)):boolean> +-- !query output +false + + +-- !query +SELECT '1' < cast(1 as int) FROM t +-- !query schema +struct<(1 < CAST(1 AS INT)):boolean> +-- !query output +false + + +-- !query +SELECT '1' < cast(1 as bigint) FROM t +-- !query schema +struct<(1 < CAST(1 AS BIGINT)):boolean> +-- !query output +false + + +-- !query +SELECT '1' < cast(1 as float) FROM t +-- !query schema +struct<(1 < CAST(1 AS FLOAT)):boolean> +-- !query output +false + + +-- !query +SELECT '1' < cast(1 as double) FROM t +-- !query schema +struct<(1 < CAST(1 AS DOUBLE)):boolean> +-- !query output +false + + +-- !query +SELECT '1' < cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(1 < CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT '1' < '1' FROM t +-- !query schema +struct<(1 < 1):boolean> +-- !query output +false + + +-- !query +SELECT '1' < cast('1' as binary) FROM t +-- !query schema +struct<(1 < CAST(1 AS BINARY)):boolean> +-- !query output +false + + +-- !query +SELECT '1' < cast(1 as boolean) FROM t +-- !query schema +struct<(1 < CAST(1 AS BOOLEAN)):boolean> +-- !query output +false + + +-- !query +SELECT '1' < cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "'1' < cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT '1' < cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "'1' < cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT '1' <= cast(1 as tinyint) FROM t +-- !query schema +struct<(1 <= CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' <= cast(1 as smallint) FROM t +-- !query schema +struct<(1 <= CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' <= cast(1 as int) FROM t +-- !query schema +struct<(1 <= CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' <= cast(1 as bigint) FROM t +-- !query schema +struct<(1 <= CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' <= cast(1 as float) FROM t +-- !query schema +struct<(1 <= CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' <= cast(1 as double) FROM t +-- !query schema +struct<(1 <= CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT '1' <= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(1 <= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT '1' <= '1' FROM t +-- !query schema +struct<(1 <= 1):boolean> +-- !query output +true + + +-- !query +SELECT '1' <= cast('1' as binary) FROM t +-- !query schema +struct<(1 <= CAST(1 AS BINARY)):boolean> +-- !query output +true + + +-- !query +SELECT '1' <= cast(1 as boolean) FROM t +-- !query schema +struct<(1 <= CAST(1 AS BOOLEAN)):boolean> +-- !query output +true + + +-- !query +SELECT '1' <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "'1' <= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT '1' <= cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "'1' <= cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT '1' > cast(1 as tinyint) FROM t +-- !query schema +struct<(1 > CAST(1 AS TINYINT)):boolean> +-- !query output +false + + +-- !query +SELECT '1' > cast(1 as smallint) FROM t +-- !query schema +struct<(1 > CAST(1 AS SMALLINT)):boolean> +-- !query output +false + + +-- !query +SELECT '1' > cast(1 as int) FROM t +-- !query schema +struct<(1 > CAST(1 AS INT)):boolean> +-- !query output +false + + +-- !query +SELECT '1' > cast(1 as bigint) FROM t +-- !query schema +struct<(1 > CAST(1 AS BIGINT)):boolean> +-- !query output +false + + +-- !query +SELECT '1' > cast(1 as float) FROM t +-- !query schema +struct<(1 > CAST(1 AS FLOAT)):boolean> +-- !query output +false + + +-- !query +SELECT '1' > cast(1 as double) FROM t +-- !query schema +struct<(1 > CAST(1 AS DOUBLE)):boolean> +-- !query output +false + + +-- !query +SELECT '1' > cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(1 > CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +false + + +-- !query +SELECT '1' > '1' FROM t +-- !query schema +struct<(1 > 1):boolean> +-- !query output +false + + +-- !query +SELECT '1' > cast('1' as binary) FROM t +-- !query schema +struct<(1 > CAST(1 AS BINARY)):boolean> +-- !query output +false + + +-- !query +SELECT '1' > cast(1 as boolean) FROM t +-- !query schema +struct<(1 > CAST(1 AS BOOLEAN)):boolean> +-- !query output +false + + +-- !query +SELECT '1' > cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "'1' > cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT '1' > cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 48, + "fragment" : "'1' > cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT '1' >= cast(1 as tinyint) FROM t +-- !query schema +struct<(1 >= CAST(1 AS TINYINT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' >= cast(1 as smallint) FROM t +-- !query schema +struct<(1 >= CAST(1 AS SMALLINT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' >= cast(1 as int) FROM t +-- !query schema +struct<(1 >= CAST(1 AS INT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' >= cast(1 as bigint) FROM t +-- !query schema +struct<(1 >= CAST(1 AS BIGINT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' >= cast(1 as float) FROM t +-- !query schema +struct<(1 >= CAST(1 AS FLOAT)):boolean> +-- !query output +true + + +-- !query +SELECT '1' >= cast(1 as double) FROM t +-- !query schema +struct<(1 >= CAST(1 AS DOUBLE)):boolean> +-- !query output +true + + +-- !query +SELECT '1' >= cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(1 >= CAST(1 AS DECIMAL(10,0))):boolean> +-- !query output +true + + +-- !query +SELECT '1' >= '1' FROM t +-- !query schema +struct<(1 >= 1):boolean> +-- !query output +true + + +-- !query +SELECT '1' >= cast('1' as binary) FROM t +-- !query schema +struct<(1 >= CAST(1 AS BINARY)):boolean> +-- !query output +true + + +-- !query +SELECT '1' >= cast(1 as boolean) FROM t +-- !query schema +struct<(1 >= CAST(1 AS BOOLEAN)):boolean> +-- !query output +true + + +-- !query +SELECT '1' >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "'1' >= cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT '1' >= cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "'1' >= cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT '1' <> cast(1 as tinyint) FROM t +-- !query schema +struct<(NOT (1 = CAST(1 AS TINYINT))):boolean> +-- !query output +false + + +-- !query +SELECT '1' <> cast(1 as smallint) FROM t +-- !query schema +struct<(NOT (1 = CAST(1 AS SMALLINT))):boolean> +-- !query output +false + + +-- !query +SELECT '1' <> cast(1 as int) FROM t +-- !query schema +struct<(NOT (1 = CAST(1 AS INT))):boolean> +-- !query output +false + + +-- !query +SELECT '1' <> cast(1 as bigint) FROM t +-- !query schema +struct<(NOT (1 = CAST(1 AS BIGINT))):boolean> +-- !query output +false + + +-- !query +SELECT '1' <> cast(1 as float) FROM t +-- !query schema +struct<(NOT (1 = CAST(1 AS FLOAT))):boolean> +-- !query output +false + + +-- !query +SELECT '1' <> cast(1 as double) FROM t +-- !query schema +struct<(NOT (1 = CAST(1 AS DOUBLE))):boolean> +-- !query output +false + + +-- !query +SELECT '1' <> cast(1 as decimal(10, 0)) FROM t +-- !query schema +struct<(NOT (1 = CAST(1 AS DECIMAL(10,0)))):boolean> +-- !query output +false + + +-- !query +SELECT '1' <> '1' FROM t +-- !query schema +struct<(NOT (1 = 1)):boolean> +-- !query output +false + + +-- !query +SELECT '1' <> cast('1' as binary) FROM t +-- !query schema +struct<(NOT (1 = CAST(1 AS BINARY))):boolean> +-- !query output +false + + +-- !query +SELECT '1' <> cast(1 as boolean) FROM t +-- !query schema +struct<(NOT (1 = CAST(1 AS BOOLEAN))):boolean> +-- !query output +false + + +-- !query +SELECT '1' <> cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "'1' <> cast('2017-12-11 09:30:00.0' as timestamp)" + } ] +} + + +-- !query +SELECT '1' <> cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "'1' <> cast('2017-12-11 09:30:00' as date)" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) < '1' FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) < 1):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as smallint) < '1' FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) < 1):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as int) < '1' FROM t +-- !query schema +struct<(CAST(1 AS INT) < 1):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as bigint) < '1' FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) < 1):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as float) < '1' FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) < 1):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as double) < '1' FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) < 1):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) < '1' FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) < 1):boolean> +-- !query output +false + + +-- !query +SELECT '1' < '1' FROM t +-- !query schema +struct<(1 < 1):boolean> +-- !query output +false + + +-- !query +SELECT cast('1' as binary) < '1' FROM t +-- !query schema +struct<(CAST(1 AS BINARY) < 1):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as boolean) < '1' FROM t +-- !query schema +struct<(CAST(1 AS BOOLEAN) < 1):boolean> +-- !query output +false + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) < '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) < '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) < '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00' as date) < '1'" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) <= '1' FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) <= 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) <= '1' FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) <= 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) <= '1' FROM t +-- !query schema +struct<(CAST(1 AS INT) <= 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) <= '1' FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) <= 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) <= '1' FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) <= 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) <= '1' FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) <= 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) <= '1' FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) <= 1):boolean> +-- !query output +true + + +-- !query +SELECT '1' <= '1' FROM t +-- !query schema +struct<(1 <= 1):boolean> +-- !query output +true + + +-- !query +SELECT cast('1' as binary) <= '1' FROM t +-- !query schema +struct<(CAST(1 AS BINARY) <= 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as boolean) <= '1' FROM t +-- !query schema +struct<(CAST(1 AS BOOLEAN) <= 1):boolean> +-- !query output +true + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <= '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <= '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('2017-12-11 09:30:00' as date) <= '1'" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) > '1' FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) > 1):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as smallint) > '1' FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) > 1):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as int) > '1' FROM t +-- !query schema +struct<(CAST(1 AS INT) > 1):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as bigint) > '1' FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) > 1):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as float) > '1' FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) > 1):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as double) > '1' FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) > 1):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) > '1' FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) > 1):boolean> +-- !query output +false + + +-- !query +SELECT '1' > '1' FROM t +-- !query schema +struct<(1 > 1):boolean> +-- !query output +false + + +-- !query +SELECT cast('1' as binary) > '1' FROM t +-- !query schema +struct<(CAST(1 AS BINARY) > 1):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as boolean) > '1' FROM t +-- !query schema +struct<(CAST(1 AS BOOLEAN) > 1):boolean> +-- !query output +false + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) > '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) > '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) > '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 55, + "fragment" : "cast('2017-12-11 09:30:00' as date) > '1'" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) >= '1' FROM t +-- !query schema +struct<(CAST(1 AS TINYINT) >= 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as smallint) >= '1' FROM t +-- !query schema +struct<(CAST(1 AS SMALLINT) >= 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as int) >= '1' FROM t +-- !query schema +struct<(CAST(1 AS INT) >= 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as bigint) >= '1' FROM t +-- !query schema +struct<(CAST(1 AS BIGINT) >= 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as float) >= '1' FROM t +-- !query schema +struct<(CAST(1 AS FLOAT) >= 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as double) >= '1' FROM t +-- !query schema +struct<(CAST(1 AS DOUBLE) >= 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as decimal(10, 0)) >= '1' FROM t +-- !query schema +struct<(CAST(1 AS DECIMAL(10,0)) >= 1):boolean> +-- !query output +true + + +-- !query +SELECT '1' >= '1' FROM t +-- !query schema +struct<(1 >= 1):boolean> +-- !query output +true + + +-- !query +SELECT cast('1' as binary) >= '1' FROM t +-- !query schema +struct<(CAST(1 AS BINARY) >= 1):boolean> +-- !query output +true + + +-- !query +SELECT cast(1 as boolean) >= '1' FROM t +-- !query schema +struct<(CAST(1 AS BOOLEAN) >= 1):boolean> +-- !query output +true + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) >= '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) >= '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('2017-12-11 09:30:00' as date) >= '1'" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) <> '1' FROM t +-- !query schema +struct<(NOT (CAST(1 AS TINYINT) = 1)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as smallint) <> '1' FROM t +-- !query schema +struct<(NOT (CAST(1 AS SMALLINT) = 1)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as int) <> '1' FROM t +-- !query schema +struct<(NOT (CAST(1 AS INT) = 1)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as bigint) <> '1' FROM t +-- !query schema +struct<(NOT (CAST(1 AS BIGINT) = 1)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as float) <> '1' FROM t +-- !query schema +struct<(NOT (CAST(1 AS FLOAT) = 1)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as double) <> '1' FROM t +-- !query schema +struct<(NOT (CAST(1 AS DOUBLE) = 1)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as decimal(10, 0)) <> '1' FROM t +-- !query schema +struct<(NOT (CAST(1 AS DECIMAL(10,0)) = 1)):boolean> +-- !query output +false + + +-- !query +SELECT '1' <> '1' FROM t +-- !query schema +struct<(NOT (1 = 1)):boolean> +-- !query output +false + + +-- !query +SELECT cast('1' as binary) <> '1' FROM t +-- !query schema +struct<(NOT (CAST(1 AS BINARY) = 1)):boolean> +-- !query output +false + + +-- !query +SELECT cast(1 as boolean) <> '1' FROM t +-- !query schema +struct<(NOT (CAST(1 AS BOOLEAN) = 1)):boolean> +-- !query output +false + + +-- !query +SELECT cast('2017-12-11 09:30:00.0' as timestamp) <> '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('2017-12-11 09:30:00.0' as timestamp) <> '1'" + } ] +} + + +-- !query +SELECT cast('2017-12-11 09:30:00' as date) <> '1' FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 56, + "fragment" : "cast('2017-12-11 09:30:00' as date) <> '1'" + } ] +} + + +-- !query +SELECT abs('1') FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT sum('1') FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT avg('1') FROM t +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT stddev_pop('1') FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT stddev_samp('1') FROM t +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT - '1' FROM t +-- !query schema +struct<(- 1):double> +-- !query output +-1.0 + + +-- !query +SELECT + '1' FROM t +-- !query schema +struct<(+ 1):double> +-- !query output +1.0 + + +-- !query +SELECT var_pop('1') FROM t +-- !query schema +struct +-- !query output +0.0 + + +-- !query +SELECT var_samp('1') FROM t +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT skewness('1') FROM t +-- !query schema +struct +-- !query output +NULL + + +-- !query +SELECT kurtosis('1') FROM t +-- !query schema +struct +-- !query output +NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out new file mode 100644 index 000000000000..e0b81c5ddd98 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out @@ -0,0 +1,577 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 'aa' as a +-- !query schema +struct<> +-- !query output + + + +-- !query +select cast(a as byte) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TINYINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "cast(a as byte)" + } ] +} + + +-- !query +select cast(a as short) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"SMALLINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "cast(a as short)" + } ] +} + + +-- !query +select cast(a as int) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "cast(a as int)" + } ] +} + + +-- !query +select cast(a as long) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "cast(a as long)" + } ] +} + + +-- !query +select cast(a as float) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"FLOAT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "cast(a as float)" + } ] +} + + +-- !query +select cast(a as double) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DOUBLE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "cast(a as double)" + } ] +} + + +-- !query +select cast(a as decimal) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DECIMAL(10,0)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "cast(a as decimal)" + } ] +} + + +-- !query +select cast(a as boolean) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 25, + "fragment" : "cast(a as boolean)" + } ] +} + + +-- !query +select cast(a as timestamp) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 27, + "fragment" : "cast(a as timestamp)" + } ] +} + + +-- !query +select cast(a as date) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "cast(a as date)" + } ] +} + + +-- !query +select cast(a as binary) from t +-- !query schema +struct +-- !query output +aa + + +-- !query +select cast(a as array) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"a\"", + "srcType" : "\"STRING\"", + "targetType" : "\"ARRAY\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 31, + "fragment" : "cast(a as array)" + } ] +} + + +-- !query +select cast(a as struct) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"a\"", + "srcType" : "\"STRING\"", + "targetType" : "\"STRUCT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 34, + "fragment" : "cast(a as struct)" + } ] +} + + +-- !query +select cast(a as map) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"a\"", + "srcType" : "\"STRING\"", + "targetType" : "\"MAP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "cast(a as map)" + } ] +} + + +-- !query +select to_timestamp(a) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 22, + "fragment" : "to_timestamp(a)" + } ] +} + + + +-- !query +select to_unix_timestamp(a) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text 'aa' could not be parsed at index 0" + } +} + + + +-- !query +select unix_timestamp(a) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CANNOT_PARSE_TIMESTAMP", + "sqlState" : "22007", + "messageParameters" : { + "func" : "`try_to_timestamp`", + "message" : "Text 'aa' could not be parsed at index 0" + } +} + + + +-- !query +select from_unixtime(a) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "from_unixtime(a)" + } ] +} + + +-- !query +select from_unixtime('2018-01-01', a) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'2018-01-01'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "from_unixtime('2018-01-01', a)" + } ] +} + + +-- !query +select next_day(a, 'MO') from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 24, + "fragment" : "next_day(a, 'MO')" + } ] +} + + +-- !query +select next_day('2018-01-01', a) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "ILLEGAL_DAY_OF_WEEK", + "sqlState" : "22009", + "messageParameters" : { + "string" : "aa" + } +} + + +-- !query +select trunc(a, 'MM') from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 21, + "fragment" : "trunc(a, 'MM')" + } ] +} + + +-- !query +select trunc('2018-01-01', a) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select unhex('-123') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select sha2(a, a) from t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'aa'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "sha2(a, a)" + } ] +} + + +-- !query +select get_json_object(a, a) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select json_tuple(a, a) from t +-- !query schema +struct +-- !query output +NULL + + +-- !query +select from_json(a, 'a INT') from t +-- !query schema +struct> +-- !query output +{"a":null} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out new file mode 100644 index 000000000000..4b5d8e71c1da --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out @@ -0,0 +1,2569 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as int) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as float) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as double) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as string) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"TINYINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 72, + "fragment" : "SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"TINYINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 71, + "fragment" : "SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TIMESTAMP\"", + "dataType2" : "\"TINYINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 95, + "fragment" : "SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DATE\"", + "dataType2" : "\"TINYINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as int) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as float) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as double) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as string) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as smallint) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"SMALLINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 73, + "fragment" : "SELECT cast(1 as smallint) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"SMALLINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 72, + "fragment" : "SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TIMESTAMP\"", + "dataType2" : "\"SMALLINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 96, + "fragment" : "SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DATE\"", + "dataType2" : "\"SMALLINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 89, + "fragment" : "SELECT cast(1 as smallint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as int) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as float) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as double) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as string) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as int) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"INT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 68, + "fragment" : "SELECT cast(1 as int) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as int) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"INT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 67, + "fragment" : "SELECT cast(1 as int) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TIMESTAMP\"", + "dataType2" : "\"INT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 91, + "fragment" : "SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DATE\"", + "dataType2" : "\"INT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as int) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as float) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as double) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as string) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"BIGINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 71, + "fragment" : "SELECT cast(1 as bigint) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"BIGINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 70, + "fragment" : "SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TIMESTAMP\"", + "dataType2" : "\"BIGINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 94, + "fragment" : "SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DATE\"", + "dataType2" : "\"BIGINT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 87, + "fragment" : "SELECT cast(1 as bigint) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as int) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as float) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as double) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as string) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as float) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"FLOAT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 70, + "fragment" : "SELECT cast(1 as float) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as float) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"FLOAT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 69, + "fragment" : "SELECT cast(1 as float) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TIMESTAMP\"", + "dataType2" : "\"FLOAT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 93, + "fragment" : "SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DATE\"", + "dataType2" : "\"FLOAT\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 86, + "fragment" : "SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as int) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as float) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as double) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as string) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"DOUBLE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 71, + "fragment" : "SELECT cast(1 as double) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"DOUBLE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 70, + "fragment" : "SELECT cast(1 as double) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TIMESTAMP\"", + "dataType2" : "\"DOUBLE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 94, + "fragment" : "SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DATE\"", + "dataType2" : "\"DOUBLE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 87, + "fragment" : "SELECT cast(1 as double) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as int) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as float) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as double) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as string) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"DECIMAL(10,0)\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 79, + "fragment" : "SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"DECIMAL(10,0)\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TIMESTAMP\"", + "dataType2" : "\"DECIMAL(10,0)\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 102, + "fragment" : "SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DATE\"", + "dataType2" : "\"DECIMAL(10,0)\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 95, + "fragment" : "SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as int) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as float) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as double) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query schema +struct +-- !query output +1.0 +2.0 + + +-- !query +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as string) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as string) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast(1 as string) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 94, + "fragment" : "SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'1'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 87, + "fragment" : "SELECT cast(1 as string) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TINYINT\"", + "dataType2" : "\"BINARY\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 72, + "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as tinyint) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"SMALLINT\"", + "dataType2" : "\"BINARY\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 73, + "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as smallint) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as int) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"INT\"", + "dataType2" : "\"BINARY\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 68, + "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as int) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BIGINT\"", + "dataType2" : "\"BINARY\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 71, + "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as bigint) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as float) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"FLOAT\"", + "dataType2" : "\"BINARY\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 70, + "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as float) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as double) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DOUBLE\"", + "dataType2" : "\"BINARY\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 71, + "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as double) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DECIMAL(10,0)\"", + "dataType2" : "\"BINARY\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 79, + "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as string) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"BINARY\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 72, + "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TIMESTAMP\"", + "dataType2" : "\"BINARY\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 96, + "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DATE\"", + "dataType2" : "\"BINARY\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 89, + "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TINYINT\"", + "dataType2" : "\"BOOLEAN\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 71, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as tinyint) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"SMALLINT\"", + "dataType2" : "\"BOOLEAN\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 72, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as smallint) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as int) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"INT\"", + "dataType2" : "\"BOOLEAN\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 67, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as int) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BIGINT\"", + "dataType2" : "\"BOOLEAN\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 70, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as bigint) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as float) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"FLOAT\"", + "dataType2" : "\"BOOLEAN\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 69, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as float) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as double) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DOUBLE\"", + "dataType2" : "\"BOOLEAN\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 70, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as double) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DECIMAL(10,0)\"", + "dataType2" : "\"BOOLEAN\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 78, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as string) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkRuntimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BOOLEAN\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 70, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as string) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"BOOLEAN\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 72, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TIMESTAMP\"", + "dataType2" : "\"BOOLEAN\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 95, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t" + } ] +} + + +-- !query +SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DATE\"", + "dataType2" : "\"BOOLEAN\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TINYINT\"", + "dataType2" : "\"TIMESTAMP\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 95, + "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as tinyint) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"SMALLINT\"", + "dataType2" : "\"TIMESTAMP\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 96, + "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as smallint) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as int) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"INT\"", + "dataType2" : "\"TIMESTAMP\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 91, + "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as int) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BIGINT\"", + "dataType2" : "\"TIMESTAMP\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 94, + "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as bigint) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as float) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"FLOAT\"", + "dataType2" : "\"TIMESTAMP\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 93, + "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as float) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as double) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DOUBLE\"", + "dataType2" : "\"TIMESTAMP\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 94, + "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as double) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DECIMAL(10,0)\"", + "dataType2" : "\"TIMESTAMP\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 102, + "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as string) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"TIMESTAMP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 94, + "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as string) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"TIMESTAMP\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 96, + "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"TIMESTAMP\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 95, + "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct +-- !query output +2017-12-11 09:30:00 +2017-12-12 09:30:00 + + +-- !query +SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct +-- !query output +2017-12-11 00:00:00 +2017-12-12 09:30:00 + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as tinyint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"TINYINT\"", + "dataType2" : "\"DATE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as tinyint) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as smallint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"SMALLINT\"", + "dataType2" : "\"DATE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 89, + "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as smallint) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as int) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"INT\"", + "dataType2" : "\"DATE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 84, + "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as int) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as bigint) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BIGINT\"", + "dataType2" : "\"DATE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 87, + "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as bigint) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as float) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"FLOAT\"", + "dataType2" : "\"DATE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 86, + "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as float) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as double) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DOUBLE\"", + "dataType2" : "\"DATE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 87, + "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as double) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"DECIMAL(10,0)\"", + "dataType2" : "\"DATE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 95, + "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as decimal(10, 0)) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as string) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkDateTimeException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'2'", + "sourceType" : "\"STRING\"", + "targetType" : "\"DATE\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 87, + "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as string) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2' as binary) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BINARY\"", + "dataType2" : "\"DATE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 89, + "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2' as binary) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as boolean) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"BOOLEAN\"", + "dataType2" : "\"DATE\"", + "hint" : "", + "operator" : "UNION", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 88, + "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast(2 as boolean) FROM t" + } ] +} + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2017-12-11 09:30:00.0' as timestamp) FROM t +-- !query schema +struct +-- !query output +2017-12-11 09:30:00 +2017-12-12 00:00:00 + + +-- !query +SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION SELECT cast('2017-12-11 09:30:00' as date) FROM t +-- !query schema +struct +-- !query output +2017-12-11 +2017-12-12 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out new file mode 100644 index 000000000000..67645cfb732f --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out @@ -0,0 +1,265 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as tinyint)) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as smallint)) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as int)) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as bigint)) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as float)) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as double)) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as decimal(10, 0))) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as string)) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('1' as binary)) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as boolean)) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00.0' as timestamp)) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00' as date)) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as tinyint) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as smallint) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as int) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as bigint) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as float) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as double) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as decimal(10, 0)) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as string) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "expectedType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "exprType" : "\"STRING\"", + "location" : "upper", + "sqlExpr" : "\"RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 111, + "fragment" : "(PARTITION BY 1 ORDER BY cast(1 as string) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('1' as binary) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpecType" : "\"BINARY\"", + "sqlExpr" : "\"(PARTITION BY 1 ORDER BY CAST(1 AS BINARY) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"", + "valueBoundaryType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 113, + "fragment" : "(PARTITION BY 1 ORDER BY cast('1' as binary) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as boolean) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "expectedType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "exprType" : "\"BOOLEAN\"", + "location" : "upper", + "sqlExpr" : "\"RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 112, + "fragment" : "(PARTITION BY 1 ORDER BY cast(1 as boolean) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00.0' as timestamp) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpecType" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(PARTITION BY 1 ORDER BY CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"", + "valueBoundaryType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 136, + "fragment" : "(PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00.0' as timestamp) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('2017-12-11 09:30:00' as date) DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM t +-- !query schema +struct +-- !query output +1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udaf/udaf-group-analytics.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udaf/udaf-group-analytics.sql.out new file mode 100644 index 000000000000..f0be6f436423 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udaf/udaf-group-analytics.sql.out @@ -0,0 +1,1164 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT a + b, b, udaf(a - b) FROM testData GROUP BY a + b, b WITH CUBE +-- !query schema +struct<(a + b):int,b:int,udaf((a - b)):int> +-- !query output +2 1 1 +2 NULL 1 +3 1 1 +3 2 1 +3 NULL 2 +4 1 1 +4 2 1 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +SELECT a, b, udaf(b) FROM testData GROUP BY a, b WITH CUBE +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +SELECT a + b, b, udaf(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP +-- !query schema +struct<(a + b):int,b:int,udaf((a - b)):int> +-- !query output +2 1 1 +2 NULL 1 +3 1 1 +3 2 1 +3 NULL 2 +4 1 1 +4 2 1 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL NULL 6 + + +-- !query +SELECT a, b, udaf(b) FROM testData GROUP BY a, b WITH ROLLUP +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL NULL 6 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW courseSales AS SELECT * FROM VALUES +("dotNET", 2012, 10000), ("Java", 2012, 20000), ("dotNET", 2012, 5000), ("dotNET", 2013, 48000), ("Java", 2013, 30000) +AS courseSales(course, year, earnings) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY course, year +-- !query schema +struct +-- !query output +NULL NULL 5 +Java NULL 2 +Java 2012 1 +Java 2013 1 +dotNET NULL 3 +dotNET 2012 2 +dotNET 2013 1 + + +-- !query +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY ROLLUP(course, year, (course, year)) ORDER BY course, year +-- !query schema +struct +-- !query output +NULL NULL 5 +Java NULL 2 +Java 2012 1 +Java 2012 1 +Java 2013 1 +Java 2013 1 +dotNET NULL 3 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2013 1 +dotNET 2013 1 + + +-- !query +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY ROLLUP(course, year, (course, year), ()) ORDER BY course, year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0051", + "messageParameters" : { + "element" : "ROLLUP" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 54, + "stopIndex" : 102, + "fragment" : "GROUP BY ROLLUP(course, year, (course, year), ())" + } ] +} + + +-- !query +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, year +-- !query schema +struct +-- !query output +NULL NULL 5 +NULL 2012 3 +NULL 2013 2 +Java NULL 2 +Java 2012 1 +Java 2013 1 +dotNET NULL 3 +dotNET 2012 2 +dotNET 2013 1 + + +-- !query +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY CUBE(course, year, (course, year)) ORDER BY course, year +-- !query schema +struct +-- !query output +NULL NULL 5 +NULL 2012 3 +NULL 2013 2 +Java NULL 2 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +dotNET NULL 3 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 + + +-- !query +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY CUBE(course, year, (course, year), ()) ORDER BY course, year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0051", + "messageParameters" : { + "element" : "CUBE" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 54, + "stopIndex" : 100, + "fragment" : "GROUP BY CUBE(course, year, (course, year), ())" + } ] +} + + +-- !query +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year) +-- !query schema +struct +-- !query output +Java NULL 2 +NULL 2012 3 +NULL 2013 2 +dotNET NULL 3 + + +-- !query +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year, ()) +-- !query schema +struct +-- !query output +Java NULL 2 +NULL 2012 3 +NULL 2013 2 +NULL NULL 5 +dotNET NULL 3 + + +-- !query +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course) +-- !query schema +struct +-- !query output +Java NULL 2 +dotNET NULL 3 + + +-- !query +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year) +-- !query schema +struct +-- !query output +NULL 2012 3 +NULL 2013 2 + + +-- !query +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY course, CUBE(course, year) ORDER BY course, year +-- !query schema +struct +-- !query output +Java NULL 2 +Java NULL 2 +Java 2012 1 +Java 2012 1 +Java 2013 1 +Java 2013 1 +dotNET NULL 3 +dotNET NULL 3 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2013 1 +dotNET 2013 1 + + +-- !query +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY CUBE(course, year), ROLLUP(course, year) ORDER BY course, year +-- !query schema +struct +-- !query output +NULL NULL 5 +NULL 2012 3 +NULL 2013 2 +Java NULL 2 +Java NULL 2 +Java NULL 2 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +dotNET NULL 3 +dotNET NULL 3 +dotNET NULL 3 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 + + +-- !query +SELECT course, year, udaf(earnings) FROM courseSales GROUP BY CUBE(course, year), ROLLUP(course, year), GROUPING SETS(course, year) ORDER BY course, year +-- !query schema +struct +-- !query output +NULL 2012 3 +NULL 2012 3 +NULL 2013 2 +NULL 2013 2 +Java NULL 2 +Java NULL 2 +Java NULL 2 +Java NULL 2 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2012 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +Java 2013 1 +dotNET NULL 3 +dotNET NULL 3 +dotNET NULL 3 +dotNET NULL 3 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2012 2 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 +dotNET 2013 1 + + +-- !query +SELECT course, udaf(earnings) AS sum FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum +-- !query schema +struct +-- !query output +NULL 5 +Java 1 +Java 1 +Java 2 +dotNET 1 +dotNET 1 +dotNET 1 +dotNET 3 + + +-- !query +SELECT course, udaf(earnings) AS sum, GROUPING_ID(course, earnings) FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum +-- !query schema +struct +-- !query output +NULL 5 3 +Java 1 0 +Java 1 0 +Java 2 1 +dotNET 1 0 +dotNET 1 0 +dotNET 1 0 +dotNET 3 1 + + +-- !query +SELECT a + b AS k1, b AS k2, udaf(a - b) FROM testData GROUP BY CUBE(k1, k2) +-- !query schema +struct +-- !query output +2 1 1 +2 NULL 1 +3 1 1 +3 2 1 +3 NULL 2 +4 1 1 +4 2 1 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +SELECT a + b AS k, b, udaf(a - b) FROM testData GROUP BY ROLLUP(k, b) +-- !query schema +struct +-- !query output +2 1 1 +2 NULL 1 +3 1 1 +3 2 1 +3 NULL 2 +4 1 1 +4 2 1 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL NULL 6 + + +-- !query +SELECT a + b, b AS k, udaf(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k) +-- !query schema +struct<(a + b):int,k:int,udaf((a - b)):int> +-- !query output +NULL 1 3 +NULL 2 3 + + +-- !query +SELECT a, b, udaf(1) FROM testData GROUP BY a, b, CUBE(a, b) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 + + +-- !query +SELECT a, b, udaf(1) FROM testData GROUP BY a, b, ROLLUP(a, b) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 + + +-- !query +SELECT a, b, udaf(1) FROM testData GROUP BY CUBE(a, b), ROLLUP(a, b) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 NULL 2 +1 NULL 2 +1 NULL 2 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 NULL 2 +2 NULL 2 +2 NULL 2 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 NULL 2 +3 NULL 2 +3 NULL 2 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +SELECT a, b, udaf(1) FROM testData GROUP BY a, CUBE(a, b), ROLLUP(b) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 NULL 2 +1 NULL 2 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 NULL 2 +2 NULL 2 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 NULL 2 +3 NULL 2 + + +-- !query +SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS((a, b), (a), ()) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +3 NULL 2 + + +-- !query +SELECT a, b, udaf(1) FROM testData GROUP BY a, CUBE(a, b), GROUPING SETS((a, b), (a), ()) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 NULL 2 +1 NULL 2 +1 NULL 2 +1 NULL 2 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 NULL 2 +2 NULL 2 +2 NULL 2 +2 NULL 2 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 NULL 2 +3 NULL 2 +3 NULL 2 +3 NULL 2 + + +-- !query +SELECT a, b, udaf(1) FROM testData GROUP BY a, CUBE(a, b), ROLLUP(a, b), GROUPING SETS((a, b), (a), ()) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 2 1 +1 NULL 2 +1 NULL 2 +1 NULL 2 +1 NULL 2 +1 NULL 2 +1 NULL 2 +1 NULL 2 +1 NULL 2 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 2 1 +2 NULL 2 +2 NULL 2 +2 NULL 2 +2 NULL 2 +2 NULL 2 +2 NULL 2 +2 NULL 2 +2 NULL 2 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 2 1 +3 NULL 2 +3 NULL 2 +3 NULL 2 +3 NULL 2 +3 NULL 2 +3 NULL 2 +3 NULL 2 +3 NULL 2 + + +-- !query +SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS(ROLLUP(a, b)) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +3 NULL 2 + + +-- !query +SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b), (a), ())) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +3 NULL 2 + + +-- !query +SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS((a, b), GROUPING SETS(ROLLUP(a, b))) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 NULL 2 +1 NULL 2 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 NULL 2 +2 NULL 2 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 NULL 2 +3 NULL 2 + + +-- !query +SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS((a, b, a, b), (a, b, a), (a, b)) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 + + +-- !query +SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b, a, b), (a, b, a), (a, b))) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 + + +-- !query +SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS(ROLLUP(a, b), CUBE(a, b)) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +1 NULL 2 +1 NULL 2 +1 NULL 2 +1 NULL 2 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +2 NULL 2 +2 NULL 2 +2 NULL 2 +2 NULL 2 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 +3 NULL 2 +3 NULL 2 +3 NULL 2 +3 NULL 2 + + +-- !query +SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b), (a), ()), GROUPING SETS((a, b), (a), (b), ())) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +1 NULL 2 +1 NULL 2 +1 NULL 2 +1 NULL 2 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +2 NULL 2 +2 NULL 2 +2 NULL 2 +2 NULL 2 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 +3 NULL 2 +3 NULL 2 +3 NULL 2 +3 NULL 2 + + +-- !query +SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS((a, b), (a), (), (a, b), (a), (b), ()) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 1 1 +1 2 1 +1 2 1 +1 2 1 +1 NULL 2 +1 NULL 2 +1 NULL 2 +1 NULL 2 +2 1 1 +2 1 1 +2 1 1 +2 2 1 +2 2 1 +2 2 1 +2 NULL 2 +2 NULL 2 +2 NULL 2 +2 NULL 2 +3 1 1 +3 1 1 +3 1 1 +3 2 1 +3 2 1 +3 2 1 +3 NULL 2 +3 NULL 2 +3 NULL 2 +3 NULL 2 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udaf/udaf-group-by-ordinal.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udaf/udaf-group-by-ordinal.sql.out new file mode 100644 index 000000000000..45a19ba2c3f1 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udaf/udaf-group-by-ordinal.sql.out @@ -0,0 +1,443 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view data as select * from values + (1, 1), + (1, 2), + (2, 1), + (2, 2), + (3, 1), + (3, 2) + as data(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +select a, udaf(b) from data group by 1 +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 + + +-- !query +select 1, 2, udaf(b) from data group by 1, 2 +-- !query schema +struct<1:int,2:int,udaf(b):int> +-- !query output +1 2 6 + + +-- !query +select a, 1, udaf(b) from data group by a, 1 +-- !query schema +struct +-- !query output +1 1 2 +2 1 2 +3 1 2 + + +-- !query +select a, 1, udaf(b) from data group by 1, 2 +-- !query schema +struct +-- !query output +1 1 2 +2 1 2 +3 1 2 + + +-- !query +select a, b + 2, udaf(2) from data group by a, 2 +-- !query schema +struct +-- !query output +1 3 1 +1 4 1 +2 3 1 +2 4 1 +3 3 1 +3 4 1 + + +-- !query +select a as aa, b + 2 as bb, udaf(2) from data group by 1, 2 +-- !query schema +struct +-- !query output +1 3 1 +1 4 1 +2 3 1 +2 4 1 +3 3 1 +3 4 1 + + +-- !query +select udaf(b) from data group by 1 + 0 +-- !query schema +struct +-- !query output +6 + + +-- !query +select a, b, udaf(b) from data group by 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "udaf(b#x) AS `udaf(b)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 41, + "stopIndex" : 41, + "fragment" : "3" + } ] +} + + +-- !query +select a, b, udaf(b) + 2 from data group by 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "(udaf(b#x) + 2) AS `(udaf(b) + 2)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 45, + "fragment" : "3" + } ] +} + + +-- !query +select a, rand(0), udaf(b) +from +(select /*+ REPARTITION(1) */ a, b from data) group by a, 2 +-- !query schema +struct +-- !query output +1 0.5234194256885571 1 +1 0.7604953758285915 1 +2 0.0953472826424725 1 +2 0.3163249920547614 1 +3 0.2710259815484829 1 +3 0.7141011170991605 1 + + +-- !query +select a, udaf(a) from (select 1 as a) tmp group by 1 order by 1 +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select udaf(a), a from (select 1 as a) tmp group by 2 having a > 0 +-- !query schema +struct +-- !query output +1 1 + + +-- !query +select a, a AS k, udaf(b) from data group by k, 1 +-- !query schema +struct +-- !query output +1 1 2 +2 2 2 +3 3 2 + + +-- !query +select a, b, udaf(1) from data group by cube(1, 2) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +select a, b, udaf(1) from data group by cube(1, b) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +select a, b, udaf(1) from data group by 1, 2 with cube +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 +NULL NULL 6 + + +-- !query +select a, b, udaf(1) from data group by rollup(1, 2) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL NULL 6 + + +-- !query +select a, b, udaf(1) from data group by rollup(1, b) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL NULL 6 + + +-- !query +select a, b, udaf(1) from data group by 1, 2 with rollup +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL NULL 6 + + +-- !query +select a, b, udaf(1) from data group by grouping sets((1), (2), (1, 2)) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 + + +-- !query +select a, b, udaf(1) from data group by grouping sets((1), (b), (a, 2)) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 + + +-- !query +select a, b, udaf(1) from data group by a, 2 grouping sets((1), (b), (a, 2)) +-- !query schema +struct +-- !query output +1 1 1 +1 2 1 +1 NULL 2 +2 1 1 +2 2 1 +2 NULL 2 +3 1 1 +3 2 1 +3 NULL 2 +NULL 1 3 +NULL 2 3 + + +-- !query +select a, b, udaf(1) from data group by a, -1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 45, + "fragment" : "-1" + } ] +} + + +-- !query +select a, b, udaf(1) from data group by a, 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "udaf(1) AS `udaf(1)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 44, + "stopIndex" : 44, + "fragment" : "3" + } ] +} + + +-- !query +select a, b, udaf(1) from data group by cube(-1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "-1", + "size" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 46, + "stopIndex" : 47, + "fragment" : "-1" + } ] +} + + +-- !query +select a, b, udaf(1) from data group by cube(1, 3) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "aggExpr" : "udaf(1) AS `udaf(1)`", + "index" : "3" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 49, + "stopIndex" : 49, + "fragment" : "3" + } ] +} + + +-- !query +set spark.sql.groupByOrdinal=false +-- !query schema +struct +-- !query output +spark.sql.groupByOrdinal false + + +-- !query +select udaf(b) from data group by -1 +-- !query schema +struct +-- !query output +6 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udaf/udaf-group-by.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udaf/udaf-group-by.sql.out new file mode 100644 index 000000000000..ad12f8bd03fd --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udaf/udaf-group-by.sql.out @@ -0,0 +1,441 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT a, udaf(b) FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 31, + "fragment" : "SELECT a, udaf(b) FROM testData" + } ] +} + + +-- !query +SELECT udaf(a), udaf(b) FROM testData +-- !query schema +struct +-- !query output +7 7 + + +-- !query +SELECT a, udaf(b) FROM testData GROUP BY a +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT a, udaf(b) FROM testData GROUP BY b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT udaf(a), udaf(b) FROM testData GROUP BY a +-- !query schema +struct +-- !query output +0 1 +2 2 +2 2 +3 2 + + +-- !query +SELECT 'foo', udaf(a) FROM testData GROUP BY 1 +-- !query schema +struct +-- !query output +foo 7 + + +-- !query +SELECT 'foo', udaf(a) FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', udaf(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT a + b, udaf(b) FROM testData GROUP BY a + b +-- !query schema +struct<(a + b):int,udaf(b):int> +-- !query output +2 1 +3 2 +4 2 +5 1 +NULL 1 + + +-- !query +SELECT a + 2, udaf(b) FROM testData GROUP BY a + 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT a + 1 + 1, udaf(b) FROM testData GROUP BY a + 1 +-- !query schema +struct<((a + 1) + 1):int,udaf(b):int> +-- !query output +3 2 +4 2 +5 2 +NULL 1 + + +-- !query +SELECT SKEWNESS(a), KURTOSIS(a), udaf(a), udaf(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), udaf(a) +FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_PANDAS_UDF_PLACEMENT", + "sqlState" : "0A000", + "messageParameters" : { + "functionList" : "`udaf`" + } +} + + +-- !query +SELECT udaf(DISTINCT b), udaf(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_SQL_SYNTAX.FUNCTION_WITH_UNSUPPORTED_SYNTAX", + "sqlState" : "42000", + "messageParameters" : { + "prettyName" : "`pythonudaf`", + "syntax" : "DISTINCT" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 23, + "fragment" : "udaf(DISTINCT b)" + } ] +} + + +-- !query +SELECT a AS k, udaf(b) FROM testData GROUP BY k +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT a AS k, udaf(b) FROM testData GROUP BY k HAVING k > 1 +-- !query schema +struct +-- !query output +2 2 +3 2 + + +-- !query +SELECT a AS k, udaf(non_existing) FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`non_existing`", + "proposal" : "`a`, `b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 21, + "stopIndex" : 32, + "fragment" : "non_existing" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT k AS a, udaf(v) FROM testDataHasSameNameWithAlias GROUP BY a +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"k\"", + "expressionAnyValue" : "\"any_value(k)\"" + } +} + + +-- !query +set spark.sql.groupByAliases=false +-- !query schema +struct +-- !query output +spark.sql.groupByAliases false + + +-- !query +SELECT a AS k, udaf(b) FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`k`", + "proposal" : "`a`, `b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 47, + "stopIndex" : 47, + "fragment" : "k" + } ] +} + + +-- !query +SELECT a, udaf(1) FROM testData WHERE false GROUP BY a +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udaf(1) FROM testData WHERE false +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 1 FROM (SELECT udaf(1) FROM testData WHERE false) t +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT 1 from ( + SELECT 1 AS z, + udaf(a.x) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z +-- !query schema +struct<1:int> +-- !query output + + + +-- !query +SELECT 1 FROM range(10) HAVING udaf(id) > 0 +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_PANDAS_UDF_PLACEMENT", + "sqlState" : "0A000", + "messageParameters" : { + "functionList" : "`udaf`" + } +} + + +-- !query +SELECT udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_PANDAS_UDF_PLACEMENT", + "sqlState" : "0A000", + "messageParameters" : { + "functionList" : "`udaf`" + } +} + + +-- !query +SELECT udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_PANDAS_UDF_PLACEMENT", + "sqlState" : "0A000", + "messageParameters" : { + "functionList" : "`udaf`" + } +} + + +-- !query +SELECT k, udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_PANDAS_UDF_PLACEMENT", + "sqlState" : "0A000", + "messageParameters" : { + "functionList" : "`udaf`" + } +} + + +-- !query +SELECT k, udaf(v) FROM test_agg GROUP BY k HAVING udaf(v) = false +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES", + "sqlState" : "42K09", + "messageParameters" : { + "left" : "\"INT\"", + "right" : "\"BOOLEAN\"", + "sqlExpr" : "\"(udaf(v) = false)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 51, + "stopIndex" : 65, + "fragment" : "udaf(v) = false" + } ] +} + + +-- !query +SELECT k, udaf(v) FROM test_agg GROUP BY k HAVING udaf(v) IS NULL +-- !query schema +struct +-- !query output + + + +-- !query +SELECT k, + udaf(v) AS count +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY k +-- !query schema +struct +-- !query output +2 1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udaf/udaf-grouping-set.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udaf/udaf-grouping-set.sql.out new file mode 100644 index 000000000000..db24bc7ec572 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udaf/udaf-grouping-set.sql.out @@ -0,0 +1,175 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW grouping AS SELECT * FROM VALUES + ("1", "2", "3", 1), + ("4", "5", "6", 1), + ("7", "8", "9", 1) + as grouping(a, b, c, d) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT a, b, c, udaf(d) FROM grouping GROUP BY a, b, c GROUPING SETS (()) +-- !query schema +struct +-- !query output +NULL NULL NULL 3 + + +-- !query +SELECT a, b, c, udaf(d) FROM grouping GROUP BY a, b, c GROUPING SETS ((a)) +-- !query schema +struct +-- !query output +1 NULL NULL 1 +4 NULL NULL 1 +7 NULL NULL 1 + + +-- !query +SELECT a, b, c, udaf(d) FROM grouping GROUP BY a, b, c GROUPING SETS ((c)) +-- !query schema +struct +-- !query output +NULL NULL 3 1 +NULL NULL 6 1 +NULL NULL 9 1 + + +-- !query +SELECT c1, udaf(c2) FROM (VALUES ('x', 10, 0), ('y', 20, 0)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1) +-- !query schema +struct +-- !query output +x 1 +y 1 + + +-- !query +SELECT c1, udaf(c2), grouping(c1) FROM (VALUES ('x', 10, 0), ('y', 20, 0)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1) +-- !query schema +struct +-- !query output +x 1 0 +y 1 0 + + +-- !query +SELECT c1, c2, udaf(c3), grouping__id +FROM (VALUES ('x', 'a', 10), ('y', 'b', 20) ) AS t (c1, c2, c3) +GROUP BY GROUPING SETS ( ( c1 ), ( c2 ) ) +HAVING GROUPING__ID > 1 +-- !query schema +struct +-- !query output +NULL a 1 2 +NULL b 1 2 + + +-- !query +SELECT a + b, b, udaf(c) FROM (VALUES (1,1,1),(2,2,2)) AS t(a,b,c) GROUP BY GROUPING SETS ( (a + b), (b)) +-- !query schema +struct<(a + b):int,b:int,udaf(c):int> +-- !query output +2 NULL 1 +4 NULL 1 +NULL 1 1 +NULL 2 1 + + +-- !query +SELECT a + b, b, udaf(c) FROM (VALUES (1,1,1),(2,2,2)) AS t(a,b,c) GROUP BY GROUPING SETS ( (a + b), (b + a), (b)) +-- !query schema +struct<(a + b):int,b:int,udaf(c):int> +-- !query output +2 NULL 1 +2 NULL 1 +4 NULL 1 +4 NULL 1 +NULL 1 1 +NULL 2 1 + + +-- !query +SELECT a, b, c, udaf(d) FROM grouping GROUP BY WITH ROLLUP +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'ROLLUP'", + "hint" : ": extra input 'ROLLUP'" + } +} + + +-- !query +SELECT a, b, c, udaf(d) FROM grouping GROUP BY WITH CUBE +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'CUBE'", + "hint" : ": extra input 'CUBE'" + } +} + + +-- !query +SELECT k1, k2, udaf(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)) +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +1 NULL 1 +2 2 1 +2 2 1 +2 NULL 1 + + +-- !query +SELECT grouping__id, k1, k2, udaf(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)) +-- !query schema +struct +-- !query output +0 1 1 1 +0 1 1 1 +0 2 2 1 +0 2 2 1 +1 1 NULL 1 +1 2 NULL 1 + + +-- !query +SELECT grouping(k1), k1, k2, udaf(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1)) +-- !query schema +struct +-- !query output +0 1 1 1 +0 1 1 1 +0 1 NULL 1 +0 2 2 1 +0 2 2 1 +0 2 NULL 1 + + +-- !query +SELECT grouping_id(k1, k2), udaf(v) from (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY k1, k2 GROUPING SETS ((k2, k1), k1) +-- !query schema +struct +-- !query output +0 1 +0 1 +1 1 +1 1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udaf/udaf.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udaf/udaf.sql.out new file mode 100644 index 000000000000..3ea85b1369fb --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udaf/udaf.sql.out @@ -0,0 +1,97 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(1), (2), (3), (4) +as t1(int_col1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION myDoubleAvg AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT default.myDoubleAvg(int_col1) as my_avg from t1 +-- !query schema +struct +-- !query output +102.5 + + +-- !query +SELECT default.myDoubleAvg(int_col1, 3) as my_avg from t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`spark_catalog`.`default`.`mydoubleavg`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 39, + "fragment" : "default.myDoubleAvg(int_col1, 3)" + } ] +} + + +-- !query +CREATE FUNCTION udaf1 AS 'test.non.existent.udaf' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT default.udaf1(int_col1) as udaf1 from t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CANNOT_LOAD_FUNCTION_CLASS", + "sqlState" : "46103", + "messageParameters" : { + "className" : "test.non.existent.udaf", + "functionName" : "`spark_catalog`.`default`.`udaf1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "default.udaf1(int_col1)" + } ] +} + + +-- !query +DROP FUNCTION myDoubleAvg +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION udaf1 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out new file mode 100644 index 000000000000..6e955c1e4f1e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out @@ -0,0 +1,516 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT avg(udf(four)) AS avg_1 FROM onek +-- !query schema +struct +-- !query output +1.5 + + +-- !query +SELECT udf(avg(a)) AS avg_32 FROM aggtest WHERE a < 100 +-- !query schema +struct +-- !query output +32.666666666666664 + + +-- !query +select CAST(avg(udf(b)) AS Decimal(10,3)) AS avg_107_943 FROM aggtest +-- !query schema +struct +-- !query output +107.943 + + +-- !query +SELECT sum(udf(four)) AS sum_1500 FROM onek +-- !query schema +struct +-- !query output +1500 + + +-- !query +SELECT udf(sum(a)) AS sum_198 FROM aggtest +-- !query schema +struct +-- !query output +198 + + +-- !query +SELECT udf(udf(sum(b))) AS avg_431_773 FROM aggtest +-- !query schema +struct +-- !query output +431.77260909229517 + + +-- !query +SELECT udf(max(four)) AS max_3 FROM onek +-- !query schema +struct +-- !query output +3 + + +-- !query +SELECT max(udf(a)) AS max_100 FROM aggtest +-- !query schema +struct +-- !query output +100 + + +-- !query +SELECT udf(udf(max(aggtest.b))) AS max_324_78 FROM aggtest +-- !query schema +struct +-- !query output +324.78 + + +-- !query +SELECT stddev_pop(udf(b)) FROM aggtest +-- !query schema +struct +-- !query output +131.10703231895047 + + +-- !query +SELECT udf(stddev_samp(b)) FROM aggtest +-- !query schema +struct +-- !query output +151.38936080399804 + + +-- !query +SELECT var_pop(udf(b)) FROM aggtest +-- !query schema +struct +-- !query output +17189.053923482323 + + +-- !query +SELECT udf(var_samp(b)) FROM aggtest +-- !query schema +struct +-- !query output +22918.738564643096 + + +-- !query +SELECT udf(stddev_pop(CAST(b AS Decimal(38,0)))) FROM aggtest +-- !query schema +struct +-- !query output +131.18117242958306 + + +-- !query +SELECT stddev_samp(CAST(udf(b) AS Decimal(38,0))) FROM aggtest +-- !query schema +struct +-- !query output +151.47497042966097 + + +-- !query +SELECT udf(var_pop(CAST(b AS Decimal(38,0)))) FROM aggtest +-- !query schema +struct +-- !query output +17208.5 + + +-- !query +SELECT var_samp(udf(CAST(b AS Decimal(38,0)))) FROM aggtest +-- !query schema +struct +-- !query output +22944.666666666668 + + +-- !query +SELECT udf(var_pop(1.0)), var_samp(udf(2.0)) +-- !query schema +struct +-- !query output +0.0 NULL + + +-- !query +SELECT stddev_pop(udf(CAST(3.0 AS Decimal(38,0)))), stddev_samp(CAST(udf(4.0) AS Decimal(38,0))) +-- !query schema +struct +-- !query output +0.0 NULL + + +-- !query +select sum(udf(CAST(null AS int))) from range(1,4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select sum(udf(CAST(null AS long))) from range(1,4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select sum(udf(CAST(null AS Decimal(38,0)))) from range(1,4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select sum(udf(CAST(null AS DOUBLE))) from range(1,4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select avg(udf(CAST(null AS int))) from range(1,4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select avg(udf(CAST(null AS long))) from range(1,4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select avg(udf(CAST(null AS Decimal(38,0)))) from range(1,4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select avg(udf(CAST(null AS DOUBLE))) from range(1,4) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select sum(CAST(udf('NaN') AS DOUBLE)) from range(1,4) +-- !query schema +struct +-- !query output +NaN + + +-- !query +select avg(CAST(udf('NaN') AS DOUBLE)) from range(1,4) +-- !query schema +struct +-- !query output +NaN + + +-- !query +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('Infinity'), ('1')) v(x) +-- !query schema +struct +-- !query output +Infinity NaN + + +-- !query +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('Infinity'), ('Infinity')) v(x) +-- !query schema +struct +-- !query output +Infinity NaN + + +-- !query +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('-Infinity'), ('Infinity')) v(x) +-- !query schema +struct +-- !query output +NaN NaN + + +-- !query +SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) +FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x) +-- !query schema +struct +-- !query output +1.00000005E8 2.5 + + +-- !query +SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) +FROM (VALUES (7000000000005), (7000000000007)) v(x) +-- !query schema +struct +-- !query output +7.000000000006E12 1.0 + + +-- !query +SELECT regr_count(b, a) FROM aggtest +-- !query schema +struct +-- !query output +4 + + +-- !query +SELECT regr_sxx(b, a) FROM aggtest +-- !query schema +struct +-- !query output +5099.0 + + +-- !query +SELECT regr_syy(b, a) FROM aggtest +-- !query schema +struct +-- !query output +68756.21569392929 + + +-- !query +SELECT regr_sxy(b, a) FROM aggtest +-- !query schema +struct +-- !query output +2614.5158215500414 + + +-- !query +SELECT regr_avgx(b, a), regr_avgy(b, a) FROM aggtest +-- !query schema +struct +-- !query output +49.5 107.94315227307379 + + +-- !query +SELECT regr_r2(b, a) FROM aggtest +-- !query schema +struct +-- !query output +0.019497798203180258 + + +-- !query +SELECT regr_slope(b, a), regr_intercept(b, a) FROM aggtest +-- !query schema +struct +-- !query output +0.5127507004412711 82.56199260123087 + + +-- !query +SELECT udf(covar_pop(b, udf(a))), covar_samp(udf(b), a) FROM aggtest +-- !query schema +struct +-- !query output +653.6289553875104 871.5052738500139 + + +-- !query +SELECT corr(b, udf(a)) FROM aggtest +-- !query schema +struct +-- !query output +0.1396345165178734 + + +-- !query +CREATE TEMPORARY VIEW regr_test AS SELECT * FROM VALUES (10,150),(20,250),(30,350),(80,540),(100,200) AS regr_test (x, y) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test WHERE x IN (10,20,30,80) +-- !query schema +struct +-- !query output +4 140 2900.0 1290 83075.0 15050.0 + + +-- !query +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test +-- !query schema +struct +-- !query output +5 240 6280.0 1490 95080.0 8680.0 + + +-- !query +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test WHERE x IN (10,20,30) +-- !query schema +struct +-- !query output +3 60 200.0 750 20000.0 2000.0 + + +-- !query +SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +FROM regr_test WHERE x IN (80,100) +-- !query schema +struct +-- !query output +2 180 200.0 740 57800.0 -3400.0 + + +-- !query +DROP VIEW regr_test +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT count(udf(four)) AS cnt_1000 FROM onek +-- !query schema +struct +-- !query output +1000 + + +-- !query +SELECT udf(count(DISTINCT four)) AS cnt_4 FROM onek +-- !query schema +struct +-- !query output +4 + + +-- !query +select ten, udf(count(*)), sum(udf(four)) from onek +group by ten order by ten +-- !query schema +struct +-- !query output +0 100 100 +1 100 200 +2 100 100 +3 100 200 +4 100 100 +5 100 200 +6 100 100 +7 100 200 +8 100 100 +9 100 200 + + +-- !query +select ten, count(udf(four)), udf(sum(DISTINCT four)) from onek +group by ten order by ten +-- !query schema +struct +-- !query output +0 100 2 +1 100 4 +2 100 2 +3 100 4 +4 100 2 +5 100 4 +6 100 2 +7 100 4 +8 100 2 +9 100 4 + + +-- !query +select ten, udf(sum(distinct four)) from onek a +group by ten +having exists (select 1 from onek b where udf(sum(distinct a.four)) = b.four) +-- !query schema +struct +-- !query output +0 2 +2 2 +4 2 +6 2 +8 2 + + +-- !query +select ten, sum(distinct four) from onek a +group by ten +having exists (select 1 from onek b + where sum(distinct a.four + b.four) = udf(b.four)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.AGGREGATE_FUNCTION_MIXED_OUTER_LOCAL_REFERENCES", + "sqlState" : "0A000", + "messageParameters" : { + "function" : "sum(DISTINCT (outer(a.four) + b.four))" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 114, + "stopIndex" : 142, + "fragment" : "sum(distinct a.four + b.four)" + } ] +} + + +-- !query +select + (select udf(max((select i.unique2 from tenk1 i where i.unique1 = o.unique1)))) +from tenk1 o +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`o`.`unique1`", + "proposal" : "`i`.`unique1`, `i`.`unique2`, `i`.`even`, `i`.`four`, `i`.`odd`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 75, + "stopIndex" : 83, + "fragment" : "o.unique1" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-aggregates_part2.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-aggregates_part2.sql.out new file mode 100644 index 000000000000..ff5dce8f0763 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-aggregates_part2.sql.out @@ -0,0 +1,261 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW bitwise_test AS SELECT * FROM VALUES + (1, 1, 1, 1L), + (3, 3, 3, null), + (7, 7, 7, 3L) AS bitwise_test(b1, b2, b3, b4) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT BIT_AND(b1) AS n1, BIT_OR(b2) AS n2 FROM bitwise_test where 1 = 0 +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +SELECT BIT_AND(b4) AS n1, BIT_OR(b4) AS n2 FROM bitwise_test where b4 is null +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +SELECT + BIT_AND(cast(b1 as tinyint)) AS a1, + BIT_AND(cast(b2 as smallint)) AS b1, + BIT_AND(b3) AS c1, + BIT_AND(b4) AS d1, + BIT_OR(cast(b1 as tinyint)) AS e7, + BIT_OR(cast(b2 as smallint)) AS f7, + BIT_OR(b3) AS g7, + BIT_OR(b4) AS h3 +FROM bitwise_test +-- !query schema +struct +-- !query output +1 1 1 1 7 7 7 3 + + +-- !query +SELECT + (NULL AND NULL) IS NULL AS `t`, + (TRUE AND NULL) IS NULL AS `t`, + (FALSE AND NULL) IS NULL AS `t`, + (NULL AND TRUE) IS NULL AS `t`, + (NULL AND FALSE) IS NULL AS `t`, + (TRUE AND TRUE) AS `t`, + NOT (TRUE AND FALSE) AS `t`, + NOT (FALSE AND TRUE) AS `t`, + NOT (FALSE AND FALSE) AS `t` +-- !query schema +struct +-- !query output +true true false true false true true true true + + +-- !query +SELECT + (NULL OR NULL) IS NULL AS `t`, + (TRUE OR NULL) IS NULL AS `t`, + (FALSE OR NULL) IS NULL AS `t`, + (NULL OR TRUE) IS NULL AS `t`, + (NULL OR FALSE) IS NULL AS `t`, + (TRUE OR TRUE) AS `t`, + (TRUE OR FALSE) AS `t`, + (FALSE OR TRUE) AS `t`, + NOT (FALSE OR FALSE) AS `t` +-- !query schema +struct +-- !query output +true false true false true true true true true + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW bool_test AS SELECT * FROM VALUES + (TRUE, null, FALSE, null), + (FALSE, TRUE, null, null), + (null, TRUE, FALSE, null) AS bool_test(b1, b2, b3, b4) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT BOOL_AND(b1) AS n1, BOOL_OR(b3) AS n2 FROM bool_test WHERE 1 = 0 +-- !query schema +struct +-- !query output +NULL NULL + + +-- !query +SELECT + BOOL_AND(b1) AS f1, + BOOL_AND(b2) AS t2, + BOOL_AND(b3) AS f3, + BOOL_AND(b4) AS n4, + BOOL_AND(NOT b2) AS f5, + BOOL_AND(NOT b3) AS t6 +FROM bool_test +-- !query schema +struct +-- !query output +false true false NULL false true + + +-- !query +SELECT + EVERY(b1) AS f1, + EVERY(b2) AS t2, + EVERY(b3) AS f3, + EVERY(b4) AS n4, + EVERY(NOT b2) AS f5, + EVERY(NOT b3) AS t6 +FROM bool_test +-- !query schema +struct +-- !query output +false true false NULL false true + + +-- !query +SELECT + BOOL_OR(b1) AS t1, + BOOL_OR(b2) AS t2, + BOOL_OR(b3) AS f3, + BOOL_OR(b4) AS n4, + BOOL_OR(NOT b2) AS f5, + BOOL_OR(NOT b3) AS t6 +FROM bool_test +-- !query schema +struct +-- !query output +true true false NULL false true + + +-- !query +select min(udf(unique1)) from tenk1 +-- !query schema +struct +-- !query output +0 + + +-- !query +select udf(max(unique1)) from tenk1 +-- !query schema +struct +-- !query output +9999 + + +-- !query +select max(unique1) from tenk1 where udf(unique1) < 42 +-- !query schema +struct +-- !query output +41 + + +-- !query +select max(unique1) from tenk1 where unique1 > udf(42) +-- !query schema +struct +-- !query output +9999 + + +-- !query +select max(unique1) from tenk1 where udf(unique1) > 42000 +-- !query schema +struct +-- !query output +NULL + + +-- !query +select max(tenthous) from tenk1 where udf(thousand) = 33 +-- !query schema +struct +-- !query output +9033 + + +-- !query +select min(tenthous) from tenk1 where udf(thousand) = 33 +-- !query schema +struct +-- !query output +33 + + +-- !query +select distinct max(udf(unique2)) from tenk1 +-- !query schema +struct +-- !query output +9999 + + +-- !query +select max(unique2) from tenk1 order by udf(1) +-- !query schema +struct +-- !query output +9999 + + +-- !query +select max(unique2) from tenk1 order by max(udf(unique2)) +-- !query schema +struct +-- !query output +9999 + + +-- !query +select udf(max(udf(unique2))) from tenk1 order by udf(max(unique2))+1 +-- !query schema +struct +-- !query output +9999 + + +-- !query +select t1.max_unique2, udf(g) from (select max(udf(unique2)) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc +-- !query schema +struct +-- !query output +9999 3 +9999 2 +9999 1 + + +-- !query +select udf(max(100)) from tenk1 +-- !query schema +struct +-- !query output +100 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-aggregates_part3.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-aggregates_part3.sql.out new file mode 100644 index 000000000000..e31538c16642 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-aggregates_part3.sql.out @@ -0,0 +1,29 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select udf(max(min(unique1))) from tenk1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NESTED_AGGREGATE_FUNCTION", + "sqlState" : "42607", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 27, + "fragment" : "min(unique1)" + } ] +} + + +-- !query +select udf((select udf(count(*)) + from (values (1)) t0(inner_c))) as col +from (values (2),(3)) t1(outer_c) +-- !query schema +struct +-- !query output +1 +1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-aggregates_part4.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-aggregates_part4.sql.out new file mode 100644 index 000000000000..1aaa514eb139 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-aggregates_part4.sql.out @@ -0,0 +1,31 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select percentile_cont(0.5) within group (order by b) from aggtest +-- !query schema +struct +-- !query output +53.44850015640259 + + +-- !query +select percentile_cont(0.5) within group (order by b), sum(b) from aggtest +-- !query schema +struct +-- !query output +53.44850015640259 431.77260909229517 + + +-- !query +select percentile_cont(0.5) within group (order by thousand) from tenk1 +-- !query schema +struct +-- !query output +499.5 + + +-- !query +select percentile_disc(0.5) within group (order by thousand) from tenk1 +-- !query schema +struct +-- !query output +499.0 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-case.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-case.sql.out new file mode 100755 index 000000000000..cf2bdc1efab4 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-case.sql.out @@ -0,0 +1,395 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE CASE_TBL ( + i integer, + f double +) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE CASE2_TBL ( + i integer, + j integer +) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE_TBL VALUES (1, 10.1) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE_TBL VALUES (2, 20.2) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE_TBL VALUES (3, -30.3) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE_TBL VALUES (4, NULL) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE2_TBL VALUES (1, -1) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE2_TBL VALUES (2, -2) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE2_TBL VALUES (3, -3) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE2_TBL VALUES (2, -4) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE2_TBL VALUES (1, NULL) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO CASE2_TBL VALUES (NULL, -6) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT '3' AS `One`, + CASE + WHEN udf(1 < 2) THEN 3 + END AS `Simple WHEN` +-- !query schema +struct +-- !query output +3 3 + + +-- !query +SELECT '' AS `One`, + CASE + WHEN 1 > 2 THEN udf(3) + END AS `Simple default` +-- !query schema +struct +-- !query output + NULL + + +-- !query +SELECT '3' AS `One`, + CASE + WHEN udf(1) < 2 THEN udf(3) + ELSE udf(4) + END AS `Simple ELSE` +-- !query schema +struct +-- !query output +3 3 + + +-- !query +SELECT udf('4') AS `One`, + CASE + WHEN 1 > 2 THEN 3 + ELSE 4 + END AS `ELSE default` +-- !query schema +struct +-- !query output +4 4 + + +-- !query +SELECT udf('6') AS `One`, + CASE + WHEN udf(1 > 2) THEN 3 + WHEN udf(4) < 5 THEN 6 + ELSE 7 + END AS `Two WHEN with default` +-- !query schema +struct +-- !query output +6 6 + + +-- !query +SELECT '7' AS `None`, + CASE WHEN rand() < udf(0) THEN 1 + END AS `NULL on no matches` +-- !query schema +struct +-- !query output +7 NULL + + +-- !query +SELECT CASE WHEN udf(1=0) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END +-- !query schema +struct +-- !query output +1.0 + + +-- !query +SELECT CASE 'a' WHEN 'a' THEN udf(1) ELSE udf(2) END +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT '' AS `Five`, + CASE + WHEN i >= 3 THEN i + END AS `>= 3 or Null` + FROM CASE_TBL +-- !query schema +struct= 3 or Null:int> +-- !query output + 3 + 4 + NULL + NULL + + +-- !query +SELECT '' AS `Five`, + CASE WHEN i >= 3 THEN (i + i) + ELSE i + END AS `Simplest Math` + FROM CASE_TBL +-- !query schema +struct +-- !query output + 1 + 2 + 6 + 8 + + +-- !query +SELECT '' AS `Five`, i AS `Value`, + CASE WHEN (i < 0) THEN 'small' + WHEN (i = 0) THEN 'zero' + WHEN (i = 1) THEN 'one' + WHEN (i = 2) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL +-- !query schema +struct +-- !query output + 1 one + 2 two + 3 big + 4 big + + +-- !query +SELECT '' AS `Five`, + CASE WHEN ((i < 0) or (i < 0)) THEN 'small' + WHEN ((i = 0) or (i = 0)) THEN 'zero' + WHEN ((i = 1) or (i = 1)) THEN 'one' + WHEN ((i = 2) or (i = 2)) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL +-- !query schema +struct +-- !query output + big + big + one + two + + +-- !query +SELECT * FROM CASE_TBL WHERE udf(COALESCE(f,i)) = 4 +-- !query schema +struct +-- !query output +4 NULL + + +-- !query +SELECT * FROM CASE_TBL WHERE udf(NULLIF(f,i)) = 2 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(COALESCE(a.f, b.i, b.j)) + FROM CASE_TBL a, CASE2_TBL b +-- !query schema +struct +-- !query output +-30.3 +-30.3 +-30.3 +-30.3 +-30.3 +-30.3 +-6.0 +1.0 +1.0 +10.1 +10.1 +10.1 +10.1 +10.1 +10.1 +2.0 +2.0 +20.2 +20.2 +20.2 +20.2 +20.2 +20.2 +3.0 + + +-- !query +SELECT * + FROM CASE_TBL a, CASE2_TBL b + WHERE udf(COALESCE(a.f, b.i, b.j)) = 2 +-- !query schema +struct +-- !query output +4 NULL 2 -2 +4 NULL 2 -4 + + +-- !query +SELECT udf('') AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, + NULLIF(b.i, 4) AS `NULLIF(b.i,4)` + FROM CASE_TBL a, CASE2_TBL b +-- !query schema +struct +-- !query output + 1 2 + 1 2 + 1 3 + 1 NULL + 2 1 + 2 1 + 2 3 + 2 NULL + 3 1 + 3 1 + 3 2 + 3 2 + 3 NULL + 4 1 + 4 1 + 4 2 + 4 2 + 4 3 + 4 NULL + NULL 1 + NULL 1 + NULL 2 + NULL 2 + NULL 3 + + +-- !query +SELECT '' AS `Two`, * + FROM CASE_TBL a, CASE2_TBL b + WHERE udf(COALESCE(f,b.i) = 2) +-- !query schema +struct +-- !query output + 4 NULL 2 -2 + 4 NULL 2 -4 + + +-- !query +SELECT CASE + (CASE vol('bar') + WHEN udf('foo') THEN 'it was foo!' + WHEN udf(vol(null)) THEN 'null input' + WHEN 'bar' THEN 'it was bar!' END + ) + WHEN udf('it was foo!') THEN 'foo recognized' + WHEN 'it was bar!' THEN udf('bar recognized') + ELSE 'unrecognized' END AS col +-- !query schema +struct +-- !query output +bar recognized + + +-- !query +DROP TABLE CASE_TBL +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE CASE2_TBL +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-join.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-join.sql.out new file mode 100644 index 000000000000..0eb291c07c74 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-join.sql.out @@ -0,0 +1,3745 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW INT2_TBL(f1) AS VALUES + (smallint(trim('0 '))), + (smallint(trim(' 1234 '))), + (smallint(trim(' -1234'))), + (smallint('32767')), + (smallint('-32767')) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW INT4_TBL AS SELECT * FROM + (VALUES (0), (123456), (-123456), (2147483647), (-2147483647)) + AS v(f1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM + (VALUES + (123, 456), + (123, 4567890123456789), + (4567890123456789, 123), + (4567890123456789, 4567890123456789), + (4567890123456789, -4567890123456789)) + AS v(q1, q2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW FLOAT8_TBL AS SELECT * FROM + (VALUES (0.0), (1004.30), (-34.84), + (cast('1.2345678901234e+200' as double)), (cast('1.2345678901234e-200' as double))) + AS v(f1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW TEXT_TBL AS SELECT * FROM + (VALUES ('doh!'), ('hi de ho neighbor')) + AS v(f1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW tenk2 AS SELECT * FROM tenk1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE J1_TBL ( + i integer, + j integer, + t string +) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE J2_TBL ( + i integer, + k integer +) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (1, 4, 'one') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (2, 3, 'two') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (3, 2, 'three') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (4, 1, 'four') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (5, 0, 'five') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (6, 6, 'six') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (7, 7, 'seven') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (8, 8, 'eight') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (0, NULL, 'zero') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (NULL, NULL, 'null') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J1_TBL VALUES (NULL, 0, 'zero') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J2_TBL VALUES (1, -1) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J2_TBL VALUES (2, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J2_TBL VALUES (3, -3) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J2_TBL VALUES (2, 4) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J2_TBL VALUES (5, -5) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J2_TBL VALUES (5, -5) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J2_TBL VALUES (0, NULL) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J2_TBL VALUES (NULL, NULL) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO J2_TBL VALUES (NULL, 0) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t) + FROM J1_TBL AS tx +-- !query schema +struct +-- !query output + 0 NULL zero + 1 4 one + 2 3 two + 3 2 three + 4 1 four + 5 0 five + 6 6 six + 7 7 seven + 8 8 eight + NULL 0 zero + NULL NULL null + + +-- !query +SELECT udf(udf('')) AS `xxx`, udf(udf(i)), udf(j), udf(t) + FROM J1_TBL tx +-- !query schema +struct +-- !query output + 0 NULL zero + 1 4 one + 2 3 two + 3 2 three + 4 1 four + 5 0 five + 6 6 six + 7 7 seven + 8 8 eight + NULL 0 zero + NULL NULL null + + +-- !query +SELECT udf('') AS `xxx`, a, udf(udf(b)), c + FROM J1_TBL AS t1 (a, b, c) +-- !query schema +struct +-- !query output + 0 NULL zero + 1 4 one + 2 3 two + 3 2 three + 4 1 four + 5 0 five + 6 6 six + 7 7 seven + 8 8 eight + NULL 0 zero + NULL NULL null + + +-- !query +SELECT udf('') AS `xxx`, udf(a), udf(b), udf(udf(c)) + FROM J1_TBL t1 (a, b, c) +-- !query schema +struct +-- !query output + 0 NULL zero + 1 4 one + 2 3 two + 3 2 three + 4 1 four + 5 0 five + 6 6 six + 7 7 seven + 8 8 eight + NULL 0 zero + NULL NULL null + + +-- !query +SELECT udf('') AS `xxx`, udf(a), b, udf(c), udf(d), e + FROM J1_TBL t1 (a, b, c), J2_TBL t2 (d, e) +-- !query schema +struct +-- !query output + 0 NULL zero 0 NULL + 0 NULL zero 1 -1 + 0 NULL zero 2 2 + 0 NULL zero 2 4 + 0 NULL zero 3 -3 + 0 NULL zero 5 -5 + 0 NULL zero 5 -5 + 0 NULL zero NULL 0 + 0 NULL zero NULL NULL + 1 4 one 0 NULL + 1 4 one 1 -1 + 1 4 one 2 2 + 1 4 one 2 4 + 1 4 one 3 -3 + 1 4 one 5 -5 + 1 4 one 5 -5 + 1 4 one NULL 0 + 1 4 one NULL NULL + 2 3 two 0 NULL + 2 3 two 1 -1 + 2 3 two 2 2 + 2 3 two 2 4 + 2 3 two 3 -3 + 2 3 two 5 -5 + 2 3 two 5 -5 + 2 3 two NULL 0 + 2 3 two NULL NULL + 3 2 three 0 NULL + 3 2 three 1 -1 + 3 2 three 2 2 + 3 2 three 2 4 + 3 2 three 3 -3 + 3 2 three 5 -5 + 3 2 three 5 -5 + 3 2 three NULL 0 + 3 2 three NULL NULL + 4 1 four 0 NULL + 4 1 four 1 -1 + 4 1 four 2 2 + 4 1 four 2 4 + 4 1 four 3 -3 + 4 1 four 5 -5 + 4 1 four 5 -5 + 4 1 four NULL 0 + 4 1 four NULL NULL + 5 0 five 0 NULL + 5 0 five 1 -1 + 5 0 five 2 2 + 5 0 five 2 4 + 5 0 five 3 -3 + 5 0 five 5 -5 + 5 0 five 5 -5 + 5 0 five NULL 0 + 5 0 five NULL NULL + 6 6 six 0 NULL + 6 6 six 1 -1 + 6 6 six 2 2 + 6 6 six 2 4 + 6 6 six 3 -3 + 6 6 six 5 -5 + 6 6 six 5 -5 + 6 6 six NULL 0 + 6 6 six NULL NULL + 7 7 seven 0 NULL + 7 7 seven 1 -1 + 7 7 seven 2 2 + 7 7 seven 2 4 + 7 7 seven 3 -3 + 7 7 seven 5 -5 + 7 7 seven 5 -5 + 7 7 seven NULL 0 + 7 7 seven NULL NULL + 8 8 eight 0 NULL + 8 8 eight 1 -1 + 8 8 eight 2 2 + 8 8 eight 2 4 + 8 8 eight 3 -3 + 8 8 eight 5 -5 + 8 8 eight 5 -5 + 8 8 eight NULL 0 + 8 8 eight NULL NULL + NULL 0 zero 0 NULL + NULL 0 zero 1 -1 + NULL 0 zero 2 2 + NULL 0 zero 2 4 + NULL 0 zero 3 -3 + NULL 0 zero 5 -5 + NULL 0 zero 5 -5 + NULL 0 zero NULL 0 + NULL 0 zero NULL NULL + NULL NULL null 0 NULL + NULL NULL null 1 -1 + NULL NULL null 2 2 + NULL NULL null 2 4 + NULL NULL null 3 -3 + NULL NULL null 5 -5 + NULL NULL null 5 -5 + NULL NULL null NULL 0 + NULL NULL null NULL NULL + + +-- !query +SELECT udf('') AS `xxx`, * + FROM J1_TBL CROSS JOIN J2_TBL +-- !query schema +struct +-- !query output + 0 NULL zero 0 NULL + 0 NULL zero 1 -1 + 0 NULL zero 2 2 + 0 NULL zero 2 4 + 0 NULL zero 3 -3 + 0 NULL zero 5 -5 + 0 NULL zero 5 -5 + 0 NULL zero NULL 0 + 0 NULL zero NULL NULL + 1 4 one 0 NULL + 1 4 one 1 -1 + 1 4 one 2 2 + 1 4 one 2 4 + 1 4 one 3 -3 + 1 4 one 5 -5 + 1 4 one 5 -5 + 1 4 one NULL 0 + 1 4 one NULL NULL + 2 3 two 0 NULL + 2 3 two 1 -1 + 2 3 two 2 2 + 2 3 two 2 4 + 2 3 two 3 -3 + 2 3 two 5 -5 + 2 3 two 5 -5 + 2 3 two NULL 0 + 2 3 two NULL NULL + 3 2 three 0 NULL + 3 2 three 1 -1 + 3 2 three 2 2 + 3 2 three 2 4 + 3 2 three 3 -3 + 3 2 three 5 -5 + 3 2 three 5 -5 + 3 2 three NULL 0 + 3 2 three NULL NULL + 4 1 four 0 NULL + 4 1 four 1 -1 + 4 1 four 2 2 + 4 1 four 2 4 + 4 1 four 3 -3 + 4 1 four 5 -5 + 4 1 four 5 -5 + 4 1 four NULL 0 + 4 1 four NULL NULL + 5 0 five 0 NULL + 5 0 five 1 -1 + 5 0 five 2 2 + 5 0 five 2 4 + 5 0 five 3 -3 + 5 0 five 5 -5 + 5 0 five 5 -5 + 5 0 five NULL 0 + 5 0 five NULL NULL + 6 6 six 0 NULL + 6 6 six 1 -1 + 6 6 six 2 2 + 6 6 six 2 4 + 6 6 six 3 -3 + 6 6 six 5 -5 + 6 6 six 5 -5 + 6 6 six NULL 0 + 6 6 six NULL NULL + 7 7 seven 0 NULL + 7 7 seven 1 -1 + 7 7 seven 2 2 + 7 7 seven 2 4 + 7 7 seven 3 -3 + 7 7 seven 5 -5 + 7 7 seven 5 -5 + 7 7 seven NULL 0 + 7 7 seven NULL NULL + 8 8 eight 0 NULL + 8 8 eight 1 -1 + 8 8 eight 2 2 + 8 8 eight 2 4 + 8 8 eight 3 -3 + 8 8 eight 5 -5 + 8 8 eight 5 -5 + 8 8 eight NULL 0 + 8 8 eight NULL NULL + NULL 0 zero 0 NULL + NULL 0 zero 1 -1 + NULL 0 zero 2 2 + NULL 0 zero 2 4 + NULL 0 zero 3 -3 + NULL 0 zero 5 -5 + NULL 0 zero 5 -5 + NULL 0 zero NULL 0 + NULL 0 zero NULL NULL + NULL NULL null 0 NULL + NULL NULL null 1 -1 + NULL NULL null 2 2 + NULL NULL null 2 4 + NULL NULL null 3 -3 + NULL NULL null 5 -5 + NULL NULL null 5 -5 + NULL NULL null NULL 0 + NULL NULL null NULL NULL + + +-- !query +SELECT udf('') AS `xxx`, udf(i) AS i, udf(k), udf(t) AS t + FROM J1_TBL CROSS JOIN J2_TBL +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`i`", + "referenceNames" : "[`spark_catalog`.`default`.`j1_tbl`.`i`, `spark_catalog`.`default`.`j2_tbl`.`i`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 30, + "stopIndex" : 30, + "fragment" : "i" + } ] +} + + +-- !query +SELECT udf('') AS `xxx`, udf(t1.i) AS i, udf(k), udf(t) + FROM J1_TBL t1 CROSS JOIN J2_TBL t2 +-- !query schema +struct +-- !query output + 0 -1 zero + 0 -3 zero + 0 -5 zero + 0 -5 zero + 0 0 zero + 0 2 zero + 0 4 zero + 0 NULL zero + 0 NULL zero + 1 -1 one + 1 -3 one + 1 -5 one + 1 -5 one + 1 0 one + 1 2 one + 1 4 one + 1 NULL one + 1 NULL one + 2 -1 two + 2 -3 two + 2 -5 two + 2 -5 two + 2 0 two + 2 2 two + 2 4 two + 2 NULL two + 2 NULL two + 3 -1 three + 3 -3 three + 3 -5 three + 3 -5 three + 3 0 three + 3 2 three + 3 4 three + 3 NULL three + 3 NULL three + 4 -1 four + 4 -3 four + 4 -5 four + 4 -5 four + 4 0 four + 4 2 four + 4 4 four + 4 NULL four + 4 NULL four + 5 -1 five + 5 -3 five + 5 -5 five + 5 -5 five + 5 0 five + 5 2 five + 5 4 five + 5 NULL five + 5 NULL five + 6 -1 six + 6 -3 six + 6 -5 six + 6 -5 six + 6 0 six + 6 2 six + 6 4 six + 6 NULL six + 6 NULL six + 7 -1 seven + 7 -3 seven + 7 -5 seven + 7 -5 seven + 7 0 seven + 7 2 seven + 7 4 seven + 7 NULL seven + 7 NULL seven + 8 -1 eight + 8 -3 eight + 8 -5 eight + 8 -5 eight + 8 0 eight + 8 2 eight + 8 4 eight + 8 NULL eight + 8 NULL eight + NULL -1 null + NULL -1 zero + NULL -3 null + NULL -3 zero + NULL -5 null + NULL -5 null + NULL -5 zero + NULL -5 zero + NULL 0 null + NULL 0 zero + NULL 2 null + NULL 2 zero + NULL 4 null + NULL 4 zero + NULL NULL null + NULL NULL null + NULL NULL zero + NULL NULL zero + + +-- !query +SELECT udf(udf('')) AS `xxx`, udf(udf(ii)) AS ii, udf(udf(tt)) AS tt, udf(udf(kk)) + FROM (J1_TBL CROSS JOIN J2_TBL) + AS tx (ii, jj, tt, ii2, kk) +-- !query schema +struct +-- !query output + 0 zero -1 + 0 zero -3 + 0 zero -5 + 0 zero -5 + 0 zero 0 + 0 zero 2 + 0 zero 4 + 0 zero NULL + 0 zero NULL + 1 one -1 + 1 one -3 + 1 one -5 + 1 one -5 + 1 one 0 + 1 one 2 + 1 one 4 + 1 one NULL + 1 one NULL + 2 two -1 + 2 two -3 + 2 two -5 + 2 two -5 + 2 two 0 + 2 two 2 + 2 two 4 + 2 two NULL + 2 two NULL + 3 three -1 + 3 three -3 + 3 three -5 + 3 three -5 + 3 three 0 + 3 three 2 + 3 three 4 + 3 three NULL + 3 three NULL + 4 four -1 + 4 four -3 + 4 four -5 + 4 four -5 + 4 four 0 + 4 four 2 + 4 four 4 + 4 four NULL + 4 four NULL + 5 five -1 + 5 five -3 + 5 five -5 + 5 five -5 + 5 five 0 + 5 five 2 + 5 five 4 + 5 five NULL + 5 five NULL + 6 six -1 + 6 six -3 + 6 six -5 + 6 six -5 + 6 six 0 + 6 six 2 + 6 six 4 + 6 six NULL + 6 six NULL + 7 seven -1 + 7 seven -3 + 7 seven -5 + 7 seven -5 + 7 seven 0 + 7 seven 2 + 7 seven 4 + 7 seven NULL + 7 seven NULL + 8 eight -1 + 8 eight -3 + 8 eight -5 + 8 eight -5 + 8 eight 0 + 8 eight 2 + 8 eight 4 + 8 eight NULL + 8 eight NULL + NULL null -1 + NULL null -3 + NULL null -5 + NULL null -5 + NULL null 0 + NULL null 2 + NULL null 4 + NULL null NULL + NULL null NULL + NULL zero -1 + NULL zero -3 + NULL zero -5 + NULL zero -5 + NULL zero 0 + NULL zero 2 + NULL zero 4 + NULL zero NULL + NULL zero NULL + + +-- !query +SELECT udf('') AS `xxx`, udf(udf(j1_tbl.i)), udf(j), udf(t), udf(a.i), udf(a.k), udf(b.i), udf(b.k) + FROM J1_TBL CROSS JOIN J2_TBL a CROSS JOIN J2_TBL b +-- !query schema +struct +-- !query output + 0 NULL zero 0 NULL 0 NULL + 0 NULL zero 0 NULL 1 -1 + 0 NULL zero 0 NULL 2 2 + 0 NULL zero 0 NULL 2 4 + 0 NULL zero 0 NULL 3 -3 + 0 NULL zero 0 NULL 5 -5 + 0 NULL zero 0 NULL 5 -5 + 0 NULL zero 0 NULL NULL 0 + 0 NULL zero 0 NULL NULL NULL + 0 NULL zero 1 -1 0 NULL + 0 NULL zero 1 -1 1 -1 + 0 NULL zero 1 -1 2 2 + 0 NULL zero 1 -1 2 4 + 0 NULL zero 1 -1 3 -3 + 0 NULL zero 1 -1 5 -5 + 0 NULL zero 1 -1 5 -5 + 0 NULL zero 1 -1 NULL 0 + 0 NULL zero 1 -1 NULL NULL + 0 NULL zero 2 2 0 NULL + 0 NULL zero 2 2 1 -1 + 0 NULL zero 2 2 2 2 + 0 NULL zero 2 2 2 4 + 0 NULL zero 2 2 3 -3 + 0 NULL zero 2 2 5 -5 + 0 NULL zero 2 2 5 -5 + 0 NULL zero 2 2 NULL 0 + 0 NULL zero 2 2 NULL NULL + 0 NULL zero 2 4 0 NULL + 0 NULL zero 2 4 1 -1 + 0 NULL zero 2 4 2 2 + 0 NULL zero 2 4 2 4 + 0 NULL zero 2 4 3 -3 + 0 NULL zero 2 4 5 -5 + 0 NULL zero 2 4 5 -5 + 0 NULL zero 2 4 NULL 0 + 0 NULL zero 2 4 NULL NULL + 0 NULL zero 3 -3 0 NULL + 0 NULL zero 3 -3 1 -1 + 0 NULL zero 3 -3 2 2 + 0 NULL zero 3 -3 2 4 + 0 NULL zero 3 -3 3 -3 + 0 NULL zero 3 -3 5 -5 + 0 NULL zero 3 -3 5 -5 + 0 NULL zero 3 -3 NULL 0 + 0 NULL zero 3 -3 NULL NULL + 0 NULL zero 5 -5 0 NULL + 0 NULL zero 5 -5 0 NULL + 0 NULL zero 5 -5 1 -1 + 0 NULL zero 5 -5 1 -1 + 0 NULL zero 5 -5 2 2 + 0 NULL zero 5 -5 2 2 + 0 NULL zero 5 -5 2 4 + 0 NULL zero 5 -5 2 4 + 0 NULL zero 5 -5 3 -3 + 0 NULL zero 5 -5 3 -3 + 0 NULL zero 5 -5 5 -5 + 0 NULL zero 5 -5 5 -5 + 0 NULL zero 5 -5 5 -5 + 0 NULL zero 5 -5 5 -5 + 0 NULL zero 5 -5 NULL 0 + 0 NULL zero 5 -5 NULL 0 + 0 NULL zero 5 -5 NULL NULL + 0 NULL zero 5 -5 NULL NULL + 0 NULL zero NULL 0 0 NULL + 0 NULL zero NULL 0 1 -1 + 0 NULL zero NULL 0 2 2 + 0 NULL zero NULL 0 2 4 + 0 NULL zero NULL 0 3 -3 + 0 NULL zero NULL 0 5 -5 + 0 NULL zero NULL 0 5 -5 + 0 NULL zero NULL 0 NULL 0 + 0 NULL zero NULL 0 NULL NULL + 0 NULL zero NULL NULL 0 NULL + 0 NULL zero NULL NULL 1 -1 + 0 NULL zero NULL NULL 2 2 + 0 NULL zero NULL NULL 2 4 + 0 NULL zero NULL NULL 3 -3 + 0 NULL zero NULL NULL 5 -5 + 0 NULL zero NULL NULL 5 -5 + 0 NULL zero NULL NULL NULL 0 + 0 NULL zero NULL NULL NULL NULL + 1 4 one 0 NULL 0 NULL + 1 4 one 0 NULL 1 -1 + 1 4 one 0 NULL 2 2 + 1 4 one 0 NULL 2 4 + 1 4 one 0 NULL 3 -3 + 1 4 one 0 NULL 5 -5 + 1 4 one 0 NULL 5 -5 + 1 4 one 0 NULL NULL 0 + 1 4 one 0 NULL NULL NULL + 1 4 one 1 -1 0 NULL + 1 4 one 1 -1 1 -1 + 1 4 one 1 -1 2 2 + 1 4 one 1 -1 2 4 + 1 4 one 1 -1 3 -3 + 1 4 one 1 -1 5 -5 + 1 4 one 1 -1 5 -5 + 1 4 one 1 -1 NULL 0 + 1 4 one 1 -1 NULL NULL + 1 4 one 2 2 0 NULL + 1 4 one 2 2 1 -1 + 1 4 one 2 2 2 2 + 1 4 one 2 2 2 4 + 1 4 one 2 2 3 -3 + 1 4 one 2 2 5 -5 + 1 4 one 2 2 5 -5 + 1 4 one 2 2 NULL 0 + 1 4 one 2 2 NULL NULL + 1 4 one 2 4 0 NULL + 1 4 one 2 4 1 -1 + 1 4 one 2 4 2 2 + 1 4 one 2 4 2 4 + 1 4 one 2 4 3 -3 + 1 4 one 2 4 5 -5 + 1 4 one 2 4 5 -5 + 1 4 one 2 4 NULL 0 + 1 4 one 2 4 NULL NULL + 1 4 one 3 -3 0 NULL + 1 4 one 3 -3 1 -1 + 1 4 one 3 -3 2 2 + 1 4 one 3 -3 2 4 + 1 4 one 3 -3 3 -3 + 1 4 one 3 -3 5 -5 + 1 4 one 3 -3 5 -5 + 1 4 one 3 -3 NULL 0 + 1 4 one 3 -3 NULL NULL + 1 4 one 5 -5 0 NULL + 1 4 one 5 -5 0 NULL + 1 4 one 5 -5 1 -1 + 1 4 one 5 -5 1 -1 + 1 4 one 5 -5 2 2 + 1 4 one 5 -5 2 2 + 1 4 one 5 -5 2 4 + 1 4 one 5 -5 2 4 + 1 4 one 5 -5 3 -3 + 1 4 one 5 -5 3 -3 + 1 4 one 5 -5 5 -5 + 1 4 one 5 -5 5 -5 + 1 4 one 5 -5 5 -5 + 1 4 one 5 -5 5 -5 + 1 4 one 5 -5 NULL 0 + 1 4 one 5 -5 NULL 0 + 1 4 one 5 -5 NULL NULL + 1 4 one 5 -5 NULL NULL + 1 4 one NULL 0 0 NULL + 1 4 one NULL 0 1 -1 + 1 4 one NULL 0 2 2 + 1 4 one NULL 0 2 4 + 1 4 one NULL 0 3 -3 + 1 4 one NULL 0 5 -5 + 1 4 one NULL 0 5 -5 + 1 4 one NULL 0 NULL 0 + 1 4 one NULL 0 NULL NULL + 1 4 one NULL NULL 0 NULL + 1 4 one NULL NULL 1 -1 + 1 4 one NULL NULL 2 2 + 1 4 one NULL NULL 2 4 + 1 4 one NULL NULL 3 -3 + 1 4 one NULL NULL 5 -5 + 1 4 one NULL NULL 5 -5 + 1 4 one NULL NULL NULL 0 + 1 4 one NULL NULL NULL NULL + 2 3 two 0 NULL 0 NULL + 2 3 two 0 NULL 1 -1 + 2 3 two 0 NULL 2 2 + 2 3 two 0 NULL 2 4 + 2 3 two 0 NULL 3 -3 + 2 3 two 0 NULL 5 -5 + 2 3 two 0 NULL 5 -5 + 2 3 two 0 NULL NULL 0 + 2 3 two 0 NULL NULL NULL + 2 3 two 1 -1 0 NULL + 2 3 two 1 -1 1 -1 + 2 3 two 1 -1 2 2 + 2 3 two 1 -1 2 4 + 2 3 two 1 -1 3 -3 + 2 3 two 1 -1 5 -5 + 2 3 two 1 -1 5 -5 + 2 3 two 1 -1 NULL 0 + 2 3 two 1 -1 NULL NULL + 2 3 two 2 2 0 NULL + 2 3 two 2 2 1 -1 + 2 3 two 2 2 2 2 + 2 3 two 2 2 2 4 + 2 3 two 2 2 3 -3 + 2 3 two 2 2 5 -5 + 2 3 two 2 2 5 -5 + 2 3 two 2 2 NULL 0 + 2 3 two 2 2 NULL NULL + 2 3 two 2 4 0 NULL + 2 3 two 2 4 1 -1 + 2 3 two 2 4 2 2 + 2 3 two 2 4 2 4 + 2 3 two 2 4 3 -3 + 2 3 two 2 4 5 -5 + 2 3 two 2 4 5 -5 + 2 3 two 2 4 NULL 0 + 2 3 two 2 4 NULL NULL + 2 3 two 3 -3 0 NULL + 2 3 two 3 -3 1 -1 + 2 3 two 3 -3 2 2 + 2 3 two 3 -3 2 4 + 2 3 two 3 -3 3 -3 + 2 3 two 3 -3 5 -5 + 2 3 two 3 -3 5 -5 + 2 3 two 3 -3 NULL 0 + 2 3 two 3 -3 NULL NULL + 2 3 two 5 -5 0 NULL + 2 3 two 5 -5 0 NULL + 2 3 two 5 -5 1 -1 + 2 3 two 5 -5 1 -1 + 2 3 two 5 -5 2 2 + 2 3 two 5 -5 2 2 + 2 3 two 5 -5 2 4 + 2 3 two 5 -5 2 4 + 2 3 two 5 -5 3 -3 + 2 3 two 5 -5 3 -3 + 2 3 two 5 -5 5 -5 + 2 3 two 5 -5 5 -5 + 2 3 two 5 -5 5 -5 + 2 3 two 5 -5 5 -5 + 2 3 two 5 -5 NULL 0 + 2 3 two 5 -5 NULL 0 + 2 3 two 5 -5 NULL NULL + 2 3 two 5 -5 NULL NULL + 2 3 two NULL 0 0 NULL + 2 3 two NULL 0 1 -1 + 2 3 two NULL 0 2 2 + 2 3 two NULL 0 2 4 + 2 3 two NULL 0 3 -3 + 2 3 two NULL 0 5 -5 + 2 3 two NULL 0 5 -5 + 2 3 two NULL 0 NULL 0 + 2 3 two NULL 0 NULL NULL + 2 3 two NULL NULL 0 NULL + 2 3 two NULL NULL 1 -1 + 2 3 two NULL NULL 2 2 + 2 3 two NULL NULL 2 4 + 2 3 two NULL NULL 3 -3 + 2 3 two NULL NULL 5 -5 + 2 3 two NULL NULL 5 -5 + 2 3 two NULL NULL NULL 0 + 2 3 two NULL NULL NULL NULL + 3 2 three 0 NULL 0 NULL + 3 2 three 0 NULL 1 -1 + 3 2 three 0 NULL 2 2 + 3 2 three 0 NULL 2 4 + 3 2 three 0 NULL 3 -3 + 3 2 three 0 NULL 5 -5 + 3 2 three 0 NULL 5 -5 + 3 2 three 0 NULL NULL 0 + 3 2 three 0 NULL NULL NULL + 3 2 three 1 -1 0 NULL + 3 2 three 1 -1 1 -1 + 3 2 three 1 -1 2 2 + 3 2 three 1 -1 2 4 + 3 2 three 1 -1 3 -3 + 3 2 three 1 -1 5 -5 + 3 2 three 1 -1 5 -5 + 3 2 three 1 -1 NULL 0 + 3 2 three 1 -1 NULL NULL + 3 2 three 2 2 0 NULL + 3 2 three 2 2 1 -1 + 3 2 three 2 2 2 2 + 3 2 three 2 2 2 4 + 3 2 three 2 2 3 -3 + 3 2 three 2 2 5 -5 + 3 2 three 2 2 5 -5 + 3 2 three 2 2 NULL 0 + 3 2 three 2 2 NULL NULL + 3 2 three 2 4 0 NULL + 3 2 three 2 4 1 -1 + 3 2 three 2 4 2 2 + 3 2 three 2 4 2 4 + 3 2 three 2 4 3 -3 + 3 2 three 2 4 5 -5 + 3 2 three 2 4 5 -5 + 3 2 three 2 4 NULL 0 + 3 2 three 2 4 NULL NULL + 3 2 three 3 -3 0 NULL + 3 2 three 3 -3 1 -1 + 3 2 three 3 -3 2 2 + 3 2 three 3 -3 2 4 + 3 2 three 3 -3 3 -3 + 3 2 three 3 -3 5 -5 + 3 2 three 3 -3 5 -5 + 3 2 three 3 -3 NULL 0 + 3 2 three 3 -3 NULL NULL + 3 2 three 5 -5 0 NULL + 3 2 three 5 -5 0 NULL + 3 2 three 5 -5 1 -1 + 3 2 three 5 -5 1 -1 + 3 2 three 5 -5 2 2 + 3 2 three 5 -5 2 2 + 3 2 three 5 -5 2 4 + 3 2 three 5 -5 2 4 + 3 2 three 5 -5 3 -3 + 3 2 three 5 -5 3 -3 + 3 2 three 5 -5 5 -5 + 3 2 three 5 -5 5 -5 + 3 2 three 5 -5 5 -5 + 3 2 three 5 -5 5 -5 + 3 2 three 5 -5 NULL 0 + 3 2 three 5 -5 NULL 0 + 3 2 three 5 -5 NULL NULL + 3 2 three 5 -5 NULL NULL + 3 2 three NULL 0 0 NULL + 3 2 three NULL 0 1 -1 + 3 2 three NULL 0 2 2 + 3 2 three NULL 0 2 4 + 3 2 three NULL 0 3 -3 + 3 2 three NULL 0 5 -5 + 3 2 three NULL 0 5 -5 + 3 2 three NULL 0 NULL 0 + 3 2 three NULL 0 NULL NULL + 3 2 three NULL NULL 0 NULL + 3 2 three NULL NULL 1 -1 + 3 2 three NULL NULL 2 2 + 3 2 three NULL NULL 2 4 + 3 2 three NULL NULL 3 -3 + 3 2 three NULL NULL 5 -5 + 3 2 three NULL NULL 5 -5 + 3 2 three NULL NULL NULL 0 + 3 2 three NULL NULL NULL NULL + 4 1 four 0 NULL 0 NULL + 4 1 four 0 NULL 1 -1 + 4 1 four 0 NULL 2 2 + 4 1 four 0 NULL 2 4 + 4 1 four 0 NULL 3 -3 + 4 1 four 0 NULL 5 -5 + 4 1 four 0 NULL 5 -5 + 4 1 four 0 NULL NULL 0 + 4 1 four 0 NULL NULL NULL + 4 1 four 1 -1 0 NULL + 4 1 four 1 -1 1 -1 + 4 1 four 1 -1 2 2 + 4 1 four 1 -1 2 4 + 4 1 four 1 -1 3 -3 + 4 1 four 1 -1 5 -5 + 4 1 four 1 -1 5 -5 + 4 1 four 1 -1 NULL 0 + 4 1 four 1 -1 NULL NULL + 4 1 four 2 2 0 NULL + 4 1 four 2 2 1 -1 + 4 1 four 2 2 2 2 + 4 1 four 2 2 2 4 + 4 1 four 2 2 3 -3 + 4 1 four 2 2 5 -5 + 4 1 four 2 2 5 -5 + 4 1 four 2 2 NULL 0 + 4 1 four 2 2 NULL NULL + 4 1 four 2 4 0 NULL + 4 1 four 2 4 1 -1 + 4 1 four 2 4 2 2 + 4 1 four 2 4 2 4 + 4 1 four 2 4 3 -3 + 4 1 four 2 4 5 -5 + 4 1 four 2 4 5 -5 + 4 1 four 2 4 NULL 0 + 4 1 four 2 4 NULL NULL + 4 1 four 3 -3 0 NULL + 4 1 four 3 -3 1 -1 + 4 1 four 3 -3 2 2 + 4 1 four 3 -3 2 4 + 4 1 four 3 -3 3 -3 + 4 1 four 3 -3 5 -5 + 4 1 four 3 -3 5 -5 + 4 1 four 3 -3 NULL 0 + 4 1 four 3 -3 NULL NULL + 4 1 four 5 -5 0 NULL + 4 1 four 5 -5 0 NULL + 4 1 four 5 -5 1 -1 + 4 1 four 5 -5 1 -1 + 4 1 four 5 -5 2 2 + 4 1 four 5 -5 2 2 + 4 1 four 5 -5 2 4 + 4 1 four 5 -5 2 4 + 4 1 four 5 -5 3 -3 + 4 1 four 5 -5 3 -3 + 4 1 four 5 -5 5 -5 + 4 1 four 5 -5 5 -5 + 4 1 four 5 -5 5 -5 + 4 1 four 5 -5 5 -5 + 4 1 four 5 -5 NULL 0 + 4 1 four 5 -5 NULL 0 + 4 1 four 5 -5 NULL NULL + 4 1 four 5 -5 NULL NULL + 4 1 four NULL 0 0 NULL + 4 1 four NULL 0 1 -1 + 4 1 four NULL 0 2 2 + 4 1 four NULL 0 2 4 + 4 1 four NULL 0 3 -3 + 4 1 four NULL 0 5 -5 + 4 1 four NULL 0 5 -5 + 4 1 four NULL 0 NULL 0 + 4 1 four NULL 0 NULL NULL + 4 1 four NULL NULL 0 NULL + 4 1 four NULL NULL 1 -1 + 4 1 four NULL NULL 2 2 + 4 1 four NULL NULL 2 4 + 4 1 four NULL NULL 3 -3 + 4 1 four NULL NULL 5 -5 + 4 1 four NULL NULL 5 -5 + 4 1 four NULL NULL NULL 0 + 4 1 four NULL NULL NULL NULL + 5 0 five 0 NULL 0 NULL + 5 0 five 0 NULL 1 -1 + 5 0 five 0 NULL 2 2 + 5 0 five 0 NULL 2 4 + 5 0 five 0 NULL 3 -3 + 5 0 five 0 NULL 5 -5 + 5 0 five 0 NULL 5 -5 + 5 0 five 0 NULL NULL 0 + 5 0 five 0 NULL NULL NULL + 5 0 five 1 -1 0 NULL + 5 0 five 1 -1 1 -1 + 5 0 five 1 -1 2 2 + 5 0 five 1 -1 2 4 + 5 0 five 1 -1 3 -3 + 5 0 five 1 -1 5 -5 + 5 0 five 1 -1 5 -5 + 5 0 five 1 -1 NULL 0 + 5 0 five 1 -1 NULL NULL + 5 0 five 2 2 0 NULL + 5 0 five 2 2 1 -1 + 5 0 five 2 2 2 2 + 5 0 five 2 2 2 4 + 5 0 five 2 2 3 -3 + 5 0 five 2 2 5 -5 + 5 0 five 2 2 5 -5 + 5 0 five 2 2 NULL 0 + 5 0 five 2 2 NULL NULL + 5 0 five 2 4 0 NULL + 5 0 five 2 4 1 -1 + 5 0 five 2 4 2 2 + 5 0 five 2 4 2 4 + 5 0 five 2 4 3 -3 + 5 0 five 2 4 5 -5 + 5 0 five 2 4 5 -5 + 5 0 five 2 4 NULL 0 + 5 0 five 2 4 NULL NULL + 5 0 five 3 -3 0 NULL + 5 0 five 3 -3 1 -1 + 5 0 five 3 -3 2 2 + 5 0 five 3 -3 2 4 + 5 0 five 3 -3 3 -3 + 5 0 five 3 -3 5 -5 + 5 0 five 3 -3 5 -5 + 5 0 five 3 -3 NULL 0 + 5 0 five 3 -3 NULL NULL + 5 0 five 5 -5 0 NULL + 5 0 five 5 -5 0 NULL + 5 0 five 5 -5 1 -1 + 5 0 five 5 -5 1 -1 + 5 0 five 5 -5 2 2 + 5 0 five 5 -5 2 2 + 5 0 five 5 -5 2 4 + 5 0 five 5 -5 2 4 + 5 0 five 5 -5 3 -3 + 5 0 five 5 -5 3 -3 + 5 0 five 5 -5 5 -5 + 5 0 five 5 -5 5 -5 + 5 0 five 5 -5 5 -5 + 5 0 five 5 -5 5 -5 + 5 0 five 5 -5 NULL 0 + 5 0 five 5 -5 NULL 0 + 5 0 five 5 -5 NULL NULL + 5 0 five 5 -5 NULL NULL + 5 0 five NULL 0 0 NULL + 5 0 five NULL 0 1 -1 + 5 0 five NULL 0 2 2 + 5 0 five NULL 0 2 4 + 5 0 five NULL 0 3 -3 + 5 0 five NULL 0 5 -5 + 5 0 five NULL 0 5 -5 + 5 0 five NULL 0 NULL 0 + 5 0 five NULL 0 NULL NULL + 5 0 five NULL NULL 0 NULL + 5 0 five NULL NULL 1 -1 + 5 0 five NULL NULL 2 2 + 5 0 five NULL NULL 2 4 + 5 0 five NULL NULL 3 -3 + 5 0 five NULL NULL 5 -5 + 5 0 five NULL NULL 5 -5 + 5 0 five NULL NULL NULL 0 + 5 0 five NULL NULL NULL NULL + 6 6 six 0 NULL 0 NULL + 6 6 six 0 NULL 1 -1 + 6 6 six 0 NULL 2 2 + 6 6 six 0 NULL 2 4 + 6 6 six 0 NULL 3 -3 + 6 6 six 0 NULL 5 -5 + 6 6 six 0 NULL 5 -5 + 6 6 six 0 NULL NULL 0 + 6 6 six 0 NULL NULL NULL + 6 6 six 1 -1 0 NULL + 6 6 six 1 -1 1 -1 + 6 6 six 1 -1 2 2 + 6 6 six 1 -1 2 4 + 6 6 six 1 -1 3 -3 + 6 6 six 1 -1 5 -5 + 6 6 six 1 -1 5 -5 + 6 6 six 1 -1 NULL 0 + 6 6 six 1 -1 NULL NULL + 6 6 six 2 2 0 NULL + 6 6 six 2 2 1 -1 + 6 6 six 2 2 2 2 + 6 6 six 2 2 2 4 + 6 6 six 2 2 3 -3 + 6 6 six 2 2 5 -5 + 6 6 six 2 2 5 -5 + 6 6 six 2 2 NULL 0 + 6 6 six 2 2 NULL NULL + 6 6 six 2 4 0 NULL + 6 6 six 2 4 1 -1 + 6 6 six 2 4 2 2 + 6 6 six 2 4 2 4 + 6 6 six 2 4 3 -3 + 6 6 six 2 4 5 -5 + 6 6 six 2 4 5 -5 + 6 6 six 2 4 NULL 0 + 6 6 six 2 4 NULL NULL + 6 6 six 3 -3 0 NULL + 6 6 six 3 -3 1 -1 + 6 6 six 3 -3 2 2 + 6 6 six 3 -3 2 4 + 6 6 six 3 -3 3 -3 + 6 6 six 3 -3 5 -5 + 6 6 six 3 -3 5 -5 + 6 6 six 3 -3 NULL 0 + 6 6 six 3 -3 NULL NULL + 6 6 six 5 -5 0 NULL + 6 6 six 5 -5 0 NULL + 6 6 six 5 -5 1 -1 + 6 6 six 5 -5 1 -1 + 6 6 six 5 -5 2 2 + 6 6 six 5 -5 2 2 + 6 6 six 5 -5 2 4 + 6 6 six 5 -5 2 4 + 6 6 six 5 -5 3 -3 + 6 6 six 5 -5 3 -3 + 6 6 six 5 -5 5 -5 + 6 6 six 5 -5 5 -5 + 6 6 six 5 -5 5 -5 + 6 6 six 5 -5 5 -5 + 6 6 six 5 -5 NULL 0 + 6 6 six 5 -5 NULL 0 + 6 6 six 5 -5 NULL NULL + 6 6 six 5 -5 NULL NULL + 6 6 six NULL 0 0 NULL + 6 6 six NULL 0 1 -1 + 6 6 six NULL 0 2 2 + 6 6 six NULL 0 2 4 + 6 6 six NULL 0 3 -3 + 6 6 six NULL 0 5 -5 + 6 6 six NULL 0 5 -5 + 6 6 six NULL 0 NULL 0 + 6 6 six NULL 0 NULL NULL + 6 6 six NULL NULL 0 NULL + 6 6 six NULL NULL 1 -1 + 6 6 six NULL NULL 2 2 + 6 6 six NULL NULL 2 4 + 6 6 six NULL NULL 3 -3 + 6 6 six NULL NULL 5 -5 + 6 6 six NULL NULL 5 -5 + 6 6 six NULL NULL NULL 0 + 6 6 six NULL NULL NULL NULL + 7 7 seven 0 NULL 0 NULL + 7 7 seven 0 NULL 1 -1 + 7 7 seven 0 NULL 2 2 + 7 7 seven 0 NULL 2 4 + 7 7 seven 0 NULL 3 -3 + 7 7 seven 0 NULL 5 -5 + 7 7 seven 0 NULL 5 -5 + 7 7 seven 0 NULL NULL 0 + 7 7 seven 0 NULL NULL NULL + 7 7 seven 1 -1 0 NULL + 7 7 seven 1 -1 1 -1 + 7 7 seven 1 -1 2 2 + 7 7 seven 1 -1 2 4 + 7 7 seven 1 -1 3 -3 + 7 7 seven 1 -1 5 -5 + 7 7 seven 1 -1 5 -5 + 7 7 seven 1 -1 NULL 0 + 7 7 seven 1 -1 NULL NULL + 7 7 seven 2 2 0 NULL + 7 7 seven 2 2 1 -1 + 7 7 seven 2 2 2 2 + 7 7 seven 2 2 2 4 + 7 7 seven 2 2 3 -3 + 7 7 seven 2 2 5 -5 + 7 7 seven 2 2 5 -5 + 7 7 seven 2 2 NULL 0 + 7 7 seven 2 2 NULL NULL + 7 7 seven 2 4 0 NULL + 7 7 seven 2 4 1 -1 + 7 7 seven 2 4 2 2 + 7 7 seven 2 4 2 4 + 7 7 seven 2 4 3 -3 + 7 7 seven 2 4 5 -5 + 7 7 seven 2 4 5 -5 + 7 7 seven 2 4 NULL 0 + 7 7 seven 2 4 NULL NULL + 7 7 seven 3 -3 0 NULL + 7 7 seven 3 -3 1 -1 + 7 7 seven 3 -3 2 2 + 7 7 seven 3 -3 2 4 + 7 7 seven 3 -3 3 -3 + 7 7 seven 3 -3 5 -5 + 7 7 seven 3 -3 5 -5 + 7 7 seven 3 -3 NULL 0 + 7 7 seven 3 -3 NULL NULL + 7 7 seven 5 -5 0 NULL + 7 7 seven 5 -5 0 NULL + 7 7 seven 5 -5 1 -1 + 7 7 seven 5 -5 1 -1 + 7 7 seven 5 -5 2 2 + 7 7 seven 5 -5 2 2 + 7 7 seven 5 -5 2 4 + 7 7 seven 5 -5 2 4 + 7 7 seven 5 -5 3 -3 + 7 7 seven 5 -5 3 -3 + 7 7 seven 5 -5 5 -5 + 7 7 seven 5 -5 5 -5 + 7 7 seven 5 -5 5 -5 + 7 7 seven 5 -5 5 -5 + 7 7 seven 5 -5 NULL 0 + 7 7 seven 5 -5 NULL 0 + 7 7 seven 5 -5 NULL NULL + 7 7 seven 5 -5 NULL NULL + 7 7 seven NULL 0 0 NULL + 7 7 seven NULL 0 1 -1 + 7 7 seven NULL 0 2 2 + 7 7 seven NULL 0 2 4 + 7 7 seven NULL 0 3 -3 + 7 7 seven NULL 0 5 -5 + 7 7 seven NULL 0 5 -5 + 7 7 seven NULL 0 NULL 0 + 7 7 seven NULL 0 NULL NULL + 7 7 seven NULL NULL 0 NULL + 7 7 seven NULL NULL 1 -1 + 7 7 seven NULL NULL 2 2 + 7 7 seven NULL NULL 2 4 + 7 7 seven NULL NULL 3 -3 + 7 7 seven NULL NULL 5 -5 + 7 7 seven NULL NULL 5 -5 + 7 7 seven NULL NULL NULL 0 + 7 7 seven NULL NULL NULL NULL + 8 8 eight 0 NULL 0 NULL + 8 8 eight 0 NULL 1 -1 + 8 8 eight 0 NULL 2 2 + 8 8 eight 0 NULL 2 4 + 8 8 eight 0 NULL 3 -3 + 8 8 eight 0 NULL 5 -5 + 8 8 eight 0 NULL 5 -5 + 8 8 eight 0 NULL NULL 0 + 8 8 eight 0 NULL NULL NULL + 8 8 eight 1 -1 0 NULL + 8 8 eight 1 -1 1 -1 + 8 8 eight 1 -1 2 2 + 8 8 eight 1 -1 2 4 + 8 8 eight 1 -1 3 -3 + 8 8 eight 1 -1 5 -5 + 8 8 eight 1 -1 5 -5 + 8 8 eight 1 -1 NULL 0 + 8 8 eight 1 -1 NULL NULL + 8 8 eight 2 2 0 NULL + 8 8 eight 2 2 1 -1 + 8 8 eight 2 2 2 2 + 8 8 eight 2 2 2 4 + 8 8 eight 2 2 3 -3 + 8 8 eight 2 2 5 -5 + 8 8 eight 2 2 5 -5 + 8 8 eight 2 2 NULL 0 + 8 8 eight 2 2 NULL NULL + 8 8 eight 2 4 0 NULL + 8 8 eight 2 4 1 -1 + 8 8 eight 2 4 2 2 + 8 8 eight 2 4 2 4 + 8 8 eight 2 4 3 -3 + 8 8 eight 2 4 5 -5 + 8 8 eight 2 4 5 -5 + 8 8 eight 2 4 NULL 0 + 8 8 eight 2 4 NULL NULL + 8 8 eight 3 -3 0 NULL + 8 8 eight 3 -3 1 -1 + 8 8 eight 3 -3 2 2 + 8 8 eight 3 -3 2 4 + 8 8 eight 3 -3 3 -3 + 8 8 eight 3 -3 5 -5 + 8 8 eight 3 -3 5 -5 + 8 8 eight 3 -3 NULL 0 + 8 8 eight 3 -3 NULL NULL + 8 8 eight 5 -5 0 NULL + 8 8 eight 5 -5 0 NULL + 8 8 eight 5 -5 1 -1 + 8 8 eight 5 -5 1 -1 + 8 8 eight 5 -5 2 2 + 8 8 eight 5 -5 2 2 + 8 8 eight 5 -5 2 4 + 8 8 eight 5 -5 2 4 + 8 8 eight 5 -5 3 -3 + 8 8 eight 5 -5 3 -3 + 8 8 eight 5 -5 5 -5 + 8 8 eight 5 -5 5 -5 + 8 8 eight 5 -5 5 -5 + 8 8 eight 5 -5 5 -5 + 8 8 eight 5 -5 NULL 0 + 8 8 eight 5 -5 NULL 0 + 8 8 eight 5 -5 NULL NULL + 8 8 eight 5 -5 NULL NULL + 8 8 eight NULL 0 0 NULL + 8 8 eight NULL 0 1 -1 + 8 8 eight NULL 0 2 2 + 8 8 eight NULL 0 2 4 + 8 8 eight NULL 0 3 -3 + 8 8 eight NULL 0 5 -5 + 8 8 eight NULL 0 5 -5 + 8 8 eight NULL 0 NULL 0 + 8 8 eight NULL 0 NULL NULL + 8 8 eight NULL NULL 0 NULL + 8 8 eight NULL NULL 1 -1 + 8 8 eight NULL NULL 2 2 + 8 8 eight NULL NULL 2 4 + 8 8 eight NULL NULL 3 -3 + 8 8 eight NULL NULL 5 -5 + 8 8 eight NULL NULL 5 -5 + 8 8 eight NULL NULL NULL 0 + 8 8 eight NULL NULL NULL NULL + NULL 0 zero 0 NULL 0 NULL + NULL 0 zero 0 NULL 1 -1 + NULL 0 zero 0 NULL 2 2 + NULL 0 zero 0 NULL 2 4 + NULL 0 zero 0 NULL 3 -3 + NULL 0 zero 0 NULL 5 -5 + NULL 0 zero 0 NULL 5 -5 + NULL 0 zero 0 NULL NULL 0 + NULL 0 zero 0 NULL NULL NULL + NULL 0 zero 1 -1 0 NULL + NULL 0 zero 1 -1 1 -1 + NULL 0 zero 1 -1 2 2 + NULL 0 zero 1 -1 2 4 + NULL 0 zero 1 -1 3 -3 + NULL 0 zero 1 -1 5 -5 + NULL 0 zero 1 -1 5 -5 + NULL 0 zero 1 -1 NULL 0 + NULL 0 zero 1 -1 NULL NULL + NULL 0 zero 2 2 0 NULL + NULL 0 zero 2 2 1 -1 + NULL 0 zero 2 2 2 2 + NULL 0 zero 2 2 2 4 + NULL 0 zero 2 2 3 -3 + NULL 0 zero 2 2 5 -5 + NULL 0 zero 2 2 5 -5 + NULL 0 zero 2 2 NULL 0 + NULL 0 zero 2 2 NULL NULL + NULL 0 zero 2 4 0 NULL + NULL 0 zero 2 4 1 -1 + NULL 0 zero 2 4 2 2 + NULL 0 zero 2 4 2 4 + NULL 0 zero 2 4 3 -3 + NULL 0 zero 2 4 5 -5 + NULL 0 zero 2 4 5 -5 + NULL 0 zero 2 4 NULL 0 + NULL 0 zero 2 4 NULL NULL + NULL 0 zero 3 -3 0 NULL + NULL 0 zero 3 -3 1 -1 + NULL 0 zero 3 -3 2 2 + NULL 0 zero 3 -3 2 4 + NULL 0 zero 3 -3 3 -3 + NULL 0 zero 3 -3 5 -5 + NULL 0 zero 3 -3 5 -5 + NULL 0 zero 3 -3 NULL 0 + NULL 0 zero 3 -3 NULL NULL + NULL 0 zero 5 -5 0 NULL + NULL 0 zero 5 -5 0 NULL + NULL 0 zero 5 -5 1 -1 + NULL 0 zero 5 -5 1 -1 + NULL 0 zero 5 -5 2 2 + NULL 0 zero 5 -5 2 2 + NULL 0 zero 5 -5 2 4 + NULL 0 zero 5 -5 2 4 + NULL 0 zero 5 -5 3 -3 + NULL 0 zero 5 -5 3 -3 + NULL 0 zero 5 -5 5 -5 + NULL 0 zero 5 -5 5 -5 + NULL 0 zero 5 -5 5 -5 + NULL 0 zero 5 -5 5 -5 + NULL 0 zero 5 -5 NULL 0 + NULL 0 zero 5 -5 NULL 0 + NULL 0 zero 5 -5 NULL NULL + NULL 0 zero 5 -5 NULL NULL + NULL 0 zero NULL 0 0 NULL + NULL 0 zero NULL 0 1 -1 + NULL 0 zero NULL 0 2 2 + NULL 0 zero NULL 0 2 4 + NULL 0 zero NULL 0 3 -3 + NULL 0 zero NULL 0 5 -5 + NULL 0 zero NULL 0 5 -5 + NULL 0 zero NULL 0 NULL 0 + NULL 0 zero NULL 0 NULL NULL + NULL 0 zero NULL NULL 0 NULL + NULL 0 zero NULL NULL 1 -1 + NULL 0 zero NULL NULL 2 2 + NULL 0 zero NULL NULL 2 4 + NULL 0 zero NULL NULL 3 -3 + NULL 0 zero NULL NULL 5 -5 + NULL 0 zero NULL NULL 5 -5 + NULL 0 zero NULL NULL NULL 0 + NULL 0 zero NULL NULL NULL NULL + NULL NULL null 0 NULL 0 NULL + NULL NULL null 0 NULL 1 -1 + NULL NULL null 0 NULL 2 2 + NULL NULL null 0 NULL 2 4 + NULL NULL null 0 NULL 3 -3 + NULL NULL null 0 NULL 5 -5 + NULL NULL null 0 NULL 5 -5 + NULL NULL null 0 NULL NULL 0 + NULL NULL null 0 NULL NULL NULL + NULL NULL null 1 -1 0 NULL + NULL NULL null 1 -1 1 -1 + NULL NULL null 1 -1 2 2 + NULL NULL null 1 -1 2 4 + NULL NULL null 1 -1 3 -3 + NULL NULL null 1 -1 5 -5 + NULL NULL null 1 -1 5 -5 + NULL NULL null 1 -1 NULL 0 + NULL NULL null 1 -1 NULL NULL + NULL NULL null 2 2 0 NULL + NULL NULL null 2 2 1 -1 + NULL NULL null 2 2 2 2 + NULL NULL null 2 2 2 4 + NULL NULL null 2 2 3 -3 + NULL NULL null 2 2 5 -5 + NULL NULL null 2 2 5 -5 + NULL NULL null 2 2 NULL 0 + NULL NULL null 2 2 NULL NULL + NULL NULL null 2 4 0 NULL + NULL NULL null 2 4 1 -1 + NULL NULL null 2 4 2 2 + NULL NULL null 2 4 2 4 + NULL NULL null 2 4 3 -3 + NULL NULL null 2 4 5 -5 + NULL NULL null 2 4 5 -5 + NULL NULL null 2 4 NULL 0 + NULL NULL null 2 4 NULL NULL + NULL NULL null 3 -3 0 NULL + NULL NULL null 3 -3 1 -1 + NULL NULL null 3 -3 2 2 + NULL NULL null 3 -3 2 4 + NULL NULL null 3 -3 3 -3 + NULL NULL null 3 -3 5 -5 + NULL NULL null 3 -3 5 -5 + NULL NULL null 3 -3 NULL 0 + NULL NULL null 3 -3 NULL NULL + NULL NULL null 5 -5 0 NULL + NULL NULL null 5 -5 0 NULL + NULL NULL null 5 -5 1 -1 + NULL NULL null 5 -5 1 -1 + NULL NULL null 5 -5 2 2 + NULL NULL null 5 -5 2 2 + NULL NULL null 5 -5 2 4 + NULL NULL null 5 -5 2 4 + NULL NULL null 5 -5 3 -3 + NULL NULL null 5 -5 3 -3 + NULL NULL null 5 -5 5 -5 + NULL NULL null 5 -5 5 -5 + NULL NULL null 5 -5 5 -5 + NULL NULL null 5 -5 5 -5 + NULL NULL null 5 -5 NULL 0 + NULL NULL null 5 -5 NULL 0 + NULL NULL null 5 -5 NULL NULL + NULL NULL null 5 -5 NULL NULL + NULL NULL null NULL 0 0 NULL + NULL NULL null NULL 0 1 -1 + NULL NULL null NULL 0 2 2 + NULL NULL null NULL 0 2 4 + NULL NULL null NULL 0 3 -3 + NULL NULL null NULL 0 5 -5 + NULL NULL null NULL 0 5 -5 + NULL NULL null NULL 0 NULL 0 + NULL NULL null NULL 0 NULL NULL + NULL NULL null NULL NULL 0 NULL + NULL NULL null NULL NULL 1 -1 + NULL NULL null NULL NULL 2 2 + NULL NULL null NULL NULL 2 4 + NULL NULL null NULL NULL 3 -3 + NULL NULL null NULL NULL 5 -5 + NULL NULL null NULL NULL 5 -5 + NULL NULL null NULL NULL NULL 0 + NULL NULL null NULL NULL NULL NULL + + +-- !query +SELECT udf('') AS `xxx`, udf(i) AS i, udf(j), udf(t) AS t, udf(k) + FROM J1_TBL INNER JOIN J2_TBL USING (i) +-- !query schema +struct +-- !query output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + + +-- !query +SELECT udf(udf('')) AS `xxx`, udf(i), udf(j) AS j, udf(t), udf(k) AS k + FROM J1_TBL JOIN J2_TBL USING (i) +-- !query schema +struct +-- !query output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + + +-- !query +SELECT udf('') AS `xxx`, * + FROM J1_TBL t1 (a, b, c) JOIN J2_TBL t2 (a, d) USING (a) + ORDER BY udf(udf(a)), udf(d) +-- !query schema +struct +-- !query output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + + +-- !query +SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL NATURAL JOIN J2_TBL +-- !query schema +struct +-- !query output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + + +-- !query +SELECT udf('') AS `xxx`, udf(udf(udf(a))) AS a, udf(b), udf(c), udf(d) + FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (a, d) +-- !query schema +struct +-- !query output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + + +-- !query +SELECT udf('') AS `xxx`, udf(udf(a)), udf(udf(b)), udf(udf(c)) AS c, udf(udf(udf(d))) AS d + FROM J1_TBL t1 (a, b, c) NATURAL JOIN J2_TBL t2 (d, a) +-- !query schema +struct +-- !query output + 0 NULL zero NULL + 2 3 two 2 + 4 1 four 2 + + +-- !query +SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(udf(J1_TBL.j)), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k) + FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) = J2_TBL.i) +-- !query schema +struct +-- !query output + 0 NULL zero 0 NULL + 1 4 one 1 -1 + 2 3 two 2 2 + 2 3 two 2 4 + 3 2 three 3 -3 + 5 0 five 5 -5 + 5 0 five 5 -5 + + +-- !query +SELECT udf('') AS `xxx`, udf(udf(J1_TBL.i)), udf(udf(J1_TBL.j)), udf(udf(J1_TBL.t)), J2_TBL.i, J2_TBL.k + FROM J1_TBL JOIN J2_TBL ON (J1_TBL.i = udf(J2_TBL.k)) +-- !query schema +struct +-- !query output + 0 NULL zero NULL 0 + 2 3 two 2 2 + 4 1 four 2 4 + + +-- !query +SELECT udf('') AS `xxx`, udf(J1_TBL.i), udf(J1_TBL.j), udf(J1_TBL.t), udf(J2_TBL.i), udf(J2_TBL.k) + FROM J1_TBL JOIN J2_TBL ON (udf(J1_TBL.i) <= udf(udf(J2_TBL.k))) +-- !query schema +struct +-- !query output + 0 NULL zero 2 2 + 0 NULL zero 2 4 + 0 NULL zero NULL 0 + 1 4 one 2 2 + 1 4 one 2 4 + 2 3 two 2 2 + 2 3 two 2 4 + 3 2 three 2 4 + 4 1 four 2 4 + + +-- !query +SELECT udf(udf('')) AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL LEFT OUTER JOIN J2_TBL USING (i) + ORDER BY udf(udf(i)), udf(k), udf(t) +-- !query schema +struct +-- !query output + NULL NULL null NULL + NULL 0 zero NULL + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 4 1 four NULL + 5 0 five -5 + 5 0 five -5 + 6 6 six NULL + 7 7 seven NULL + 8 8 eight NULL + + +-- !query +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL LEFT JOIN J2_TBL USING (i) + ORDER BY udf(i), udf(udf(k)), udf(t) +-- !query schema +struct +-- !query output + NULL NULL null NULL + NULL 0 zero NULL + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 4 1 four NULL + 5 0 five -5 + 5 0 five -5 + 6 6 six NULL + 7 7 seven NULL + 8 8 eight NULL + + +-- !query +SELECT udf('') AS `xxx`, udf(udf(i)), udf(j), udf(t), udf(k) + FROM J1_TBL RIGHT OUTER JOIN J2_TBL USING (i) +-- !query schema +struct +-- !query output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + NULL NULL NULL 0 + NULL NULL NULL NULL + + +-- !query +SELECT udf('') AS `xxx`, udf(i), udf(udf(j)), udf(t), udf(k) + FROM J1_TBL RIGHT JOIN J2_TBL USING (i) +-- !query schema +struct +-- !query output + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 5 0 five -5 + 5 0 five -5 + NULL NULL NULL 0 + NULL NULL NULL NULL + + +-- !query +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(udf(t)), udf(k) + FROM J1_TBL FULL OUTER JOIN J2_TBL USING (i) + ORDER BY udf(udf(i)), udf(k), udf(t) +-- !query schema +struct +-- !query output + NULL NULL NULL NULL + NULL NULL null NULL + NULL 0 zero NULL + NULL NULL NULL 0 + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 4 1 four NULL + 5 0 five -5 + 5 0 five -5 + 6 6 six NULL + 7 7 seven NULL + 8 8 eight NULL + + +-- !query +SELECT udf('') AS `xxx`, udf(i), udf(j), t, udf(udf(k)) + FROM J1_TBL FULL JOIN J2_TBL USING (i) + ORDER BY udf(udf(i)), udf(k), udf(udf(t)) +-- !query schema +struct +-- !query output + NULL NULL NULL NULL + NULL NULL null NULL + NULL 0 zero NULL + NULL NULL NULL 0 + 0 NULL zero NULL + 1 4 one -1 + 2 3 two 2 + 2 3 two 4 + 3 2 three -3 + 4 1 four NULL + 5 0 five -5 + 5 0 five -5 + 6 6 six NULL + 7 7 seven NULL + 8 8 eight NULL + + +-- !query +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(udf(k)) + FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(k) = 1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf('') AS `xxx`, udf(i), udf(j), udf(t), udf(k) + FROM J1_TBL LEFT JOIN J2_TBL USING (i) WHERE (udf(udf(i)) = udf(1)) +-- !query schema +struct +-- !query output + 1 4 one -1 + + +-- !query +CREATE TABLE t1 (name STRING, n INTEGER) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t2 (name STRING, n INTEGER) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t3 (name STRING, n INTEGER) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t1 VALUES ( 'bb', 11 ) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t2 VALUES ( 'bb', 12 ) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t2 VALUES ( 'cc', 22 ) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t2 VALUES ( 'ee', 42 ) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t3 VALUES ( 'bb', 13 ) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t3 VALUES ( 'cc', 23 ) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t3 VALUES ( 'dd', 33 ) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM t1 FULL JOIN t2 USING (name) FULL JOIN t3 USING (name) +-- !query schema +struct +-- !query output +bb 11 12 13 +cc NULL 22 23 +dd NULL NULL 33 +ee NULL 42 NULL + + +-- !query +SELECT * FROM +(SELECT udf(name) as name, t2.n FROM t2) as s2 +INNER JOIN +(SELECT udf(udf(name)) as name, t3.n FROM t3) s3 +USING (name) +-- !query schema +struct +-- !query output +bb 12 13 +cc 22 23 + + +-- !query +SELECT * FROM +(SELECT udf(udf(name)) as name, t2.n FROM t2) as s2 +LEFT JOIN +(SELECT udf(name) as name, t3.n FROM t3) s3 +USING (name) +-- !query schema +struct +-- !query output +bb 12 13 +cc 22 23 +ee 42 NULL + + +-- !query +SELECT udf(name), udf(udf(s2.n)), udf(s3.n) FROM +(SELECT * FROM t2) as s2 +FULL JOIN +(SELECT * FROM t3) s3 +USING (name) +-- !query schema +struct +-- !query output +bb 12 13 +cc 22 23 +dd NULL 33 +ee 42 NULL + + +-- !query +SELECT * FROM +(SELECT udf(udf(name)) as name, udf(n) as s2_n, udf(2) as s2_2 FROM t2) as s2 +NATURAL INNER JOIN +(SELECT udf(name) as name, udf(udf(n)) as s3_n, udf(3) as s3_2 FROM t3) s3 +-- !query schema +struct +-- !query output +bb 12 2 13 3 +cc 22 2 23 3 + + +-- !query +SELECT * FROM +(SELECT udf(name) as name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL LEFT JOIN +(SELECT udf(udf(name)) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3 +-- !query schema +struct +-- !query output +bb 12 2 13 3 +cc 22 2 23 3 +ee 42 2 NULL NULL + + +-- !query +SELECT * FROM +(SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL FULL JOIN +(SELECT udf(udf(name)) as name, udf(udf(n)) as s3_n, 3 as s3_2 FROM t3) s3 +-- !query schema +struct +-- !query output +bb 12 2 13 3 +cc 22 2 23 3 +dd NULL NULL 33 3 +ee 42 2 NULL NULL + + +-- !query +SELECT * FROM +(SELECT udf(udf(name)) as name, udf(n) as s1_n, 1 as s1_1 FROM t1) as s1 +NATURAL INNER JOIN +(SELECT udf(name) as name, udf(n) as s2_n, 2 as s2_2 FROM t2) as s2 +NATURAL INNER JOIN +(SELECT udf(udf(udf(name))) as name, udf(n) as s3_n, 3 as s3_2 FROM t3) s3 +-- !query schema +struct +-- !query output +bb 11 1 12 2 13 3 + + +-- !query +SELECT * FROM +(SELECT udf(name) as name, udf(n) as s1_n, udf(udf(1)) as s1_1 FROM t1) as s1 +NATURAL FULL JOIN +(SELECT udf(name) as name, udf(udf(n)) as s2_n, udf(2) as s2_2 FROM t2) as s2 +NATURAL FULL JOIN +(SELECT udf(udf(name)) as name, udf(n) as s3_n, udf(3) as s3_2 FROM t3) s3 +-- !query schema +struct +-- !query output +bb 11 1 12 2 13 3 +cc NULL NULL 22 2 23 3 +dd NULL NULL NULL NULL 33 3 +ee NULL NULL 42 2 NULL NULL + + +-- !query +SELECT name, udf(udf(s1_n)), udf(s2_n), udf(s3_n) FROM +(SELECT name, udf(udf(n)) as s1_n FROM t1) as s1 +NATURAL FULL JOIN + (SELECT * FROM + (SELECT name, udf(n) as s2_n FROM t2) as s2 + NATURAL FULL JOIN + (SELECT name, udf(udf(n)) as s3_n FROM t3) as s3 + ) ss2 +-- !query schema +struct +-- !query output +bb 11 12 13 +cc NULL 22 23 +dd NULL NULL 33 +ee NULL 42 NULL + + +-- !query +SELECT * FROM +(SELECT name, n as s1_n FROM t1) as s1 +NATURAL FULL JOIN + (SELECT * FROM + (SELECT name, udf(udf(n)) as s2_n, 2 as s2_2 FROM t2) as s2 + NATURAL FULL JOIN + (SELECT name, udf(n) as s3_n FROM t3) as s3 + ) ss2 +-- !query schema +struct +-- !query output +bb 11 12 2 13 +cc NULL 22 2 23 +dd NULL NULL NULL 33 +ee NULL 42 2 NULL + + +-- !query +SELECT s1.name, udf(s1_n), s2.name, udf(udf(s2_n)) FROM + (SELECT name, udf(n) as s1_n FROM t1) as s1 +FULL JOIN + (SELECT name, 2 as s2_n FROM t2) as s2 +ON (udf(udf(s1_n)) = udf(s2_n)) +-- !query schema +struct +-- !query output +NULL NULL bb 2 +NULL NULL cc 2 +NULL NULL ee 2 +bb 11 NULL NULL + + +-- !query +create or replace temporary view x as select * from + (values (1,11), (2,22), (3,null), (4,44), (5,null)) + as v(x1, x2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view y as select * from + (values (1,111), (2,222), (3,333), (4,null)) + as v(y1, y2) +-- !query schema +struct<> +-- !query output + + + +-- !query +select udf(udf(x1)), udf(x2) from x +-- !query schema +struct +-- !query output +1 11 +2 22 +3 NULL +4 44 +5 NULL + + +-- !query +select udf(y1), udf(udf(y2)) from y +-- !query schema +struct +-- !query output +1 111 +2 222 +3 333 +4 NULL + + +-- !query +select * from x left join y on (udf(x1) = udf(udf(y1)) and udf(x2) is not null) +-- !query schema +struct +-- !query output +1 11 1 111 +2 22 2 222 +3 NULL NULL NULL +4 44 4 NULL +5 NULL NULL NULL + + +-- !query +select * from x left join y on (udf(udf(x1)) = udf(y1) and udf(y2) is not null) +-- !query schema +struct +-- !query output +1 11 1 111 +2 22 2 222 +3 NULL 3 333 +4 44 NULL NULL +5 NULL NULL NULL + + +-- !query +select * from (x left join y on (udf(x1) = udf(udf(y1)))) left join x xx(xx1,xx2) +on (udf(udf(x1)) = udf(xx1)) +-- !query schema +struct +-- !query output +1 11 1 111 1 11 +2 22 2 222 2 22 +3 NULL 3 333 3 NULL +4 44 4 NULL 4 44 +5 NULL NULL NULL 5 NULL + + +-- !query +select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) +on (udf(x1) = xx1 and udf(x2) is not null) +-- !query schema +struct +-- !query output +1 11 1 111 1 11 +2 22 2 222 2 22 +3 NULL 3 333 NULL NULL +4 44 4 NULL 4 44 +5 NULL NULL NULL NULL NULL + + +-- !query +select * from (x left join y on (x1 = udf(y1))) left join x xx(xx1,xx2) +on (udf(x1) = udf(udf(xx1)) and udf(y2) is not null) +-- !query schema +struct +-- !query output +1 11 1 111 1 11 +2 22 2 222 2 22 +3 NULL 3 333 3 NULL +4 44 4 NULL NULL NULL +5 NULL NULL NULL NULL NULL + + +-- !query +select * from (x left join y on (udf(x1) = y1)) left join x xx(xx1,xx2) +on (udf(udf(x1)) = udf(xx1) and udf(udf(xx2)) is not null) +-- !query schema +struct +-- !query output +1 11 1 111 1 11 +2 22 2 222 2 22 +3 NULL 3 333 NULL NULL +4 44 4 NULL 4 44 +5 NULL NULL NULL NULL NULL + + +-- !query +select * from (x left join y on (udf(udf(x1)) = udf(udf(y1)))) left join x xx(xx1,xx2) +on (udf(x1) = udf(xx1)) where (udf(x2) is not null) +-- !query schema +struct +-- !query output +1 11 1 111 1 11 +2 22 2 222 2 22 +4 44 4 NULL 4 44 + + +-- !query +select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) +on (udf(x1) = xx1) where (udf(y2) is not null) +-- !query schema +struct +-- !query output +1 11 1 111 1 11 +2 22 2 222 2 22 +3 NULL 3 333 3 NULL + + +-- !query +select * from (x left join y on (udf(x1) = udf(y1))) left join x xx(xx1,xx2) +on (x1 = udf(xx1)) where (xx2 is not null) +-- !query schema +struct +-- !query output +1 11 1 111 1 11 +2 22 2 222 2 22 +4 44 4 NULL 4 44 + + +-- !query +select udf(udf(count(*))) from tenk1 a where udf(udf(unique1)) in + (select udf(unique1) from tenk1 b join tenk1 c using (unique1) + where udf(udf(b.unique2)) = udf(42)) +-- !query schema +struct +-- !query output +1 + + +-- !query +select udf(count(*)) from tenk1 x where + udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and + udf(x.unique1) = 0 and + udf(x.unique1) in (select aa.f1 from int4_tbl aa,float8_tbl bb where aa.f1=udf(udf(bb.f1))) +-- !query schema +struct +-- !query output +1 + + +-- !query +select udf(udf(count(*))) from tenk1 x where + udf(x.unique1) in (select udf(a.f1) from int4_tbl a,float8_tbl b where udf(udf(a.f1))=b.f1) and + udf(x.unique1) = 0 and + udf(udf(x.unique1)) in (select udf(aa.f1) from int4_tbl aa,float8_tbl bb where udf(aa.f1)=udf(udf(bb.f1))) +-- !query schema +struct +-- !query output +1 + + +-- !query +select * from int8_tbl i1 left join (int8_tbl i2 join + (select udf(123) as x) ss on udf(udf(i2.q1)) = udf(x)) on udf(udf(i1.q2)) = udf(udf(i2.q2)) +order by udf(udf(1)), 2 +-- !query schema +struct +-- !query output +4567890123456789 -4567890123456789 NULL NULL NULL +4567890123456789 123 NULL NULL NULL +123 456 123 456 123 +123 4567890123456789 123 4567890123456789 123 +4567890123456789 4567890123456789 123 4567890123456789 123 + + +-- !query +select udf(count(*)) +from + (select udf(t3.tenthous) as x1, udf(coalesce(udf(t1.stringu1), udf(t2.stringu1))) as x2 + from tenk1 t1 + left join tenk1 t2 on udf(t1.unique1) = udf(t2.unique1) + join tenk1 t3 on t1.unique2 = udf(t3.unique2)) ss, + tenk1 t4, + tenk1 t5 +where udf(t4.thousand) = udf(t5.unique1) and udf(udf(ss.x1)) = t4.tenthous and udf(ss.x2) = udf(udf(t5.stringu1)) +-- !query schema +struct +-- !query output +1000 + + +-- !query +select udf(a.f1), udf(b.f1), udf(t.thousand), udf(t.tenthous) from + tenk1 t, + (select udf(udf(sum(udf(f1))+1)) as f1 from int4_tbl i4a) a, + (select udf(sum(udf(f1))) as f1 from int4_tbl i4b) b +where b.f1 = udf(t.thousand) and udf(a.f1) = udf(b.f1) and udf((udf(a.f1)+udf(b.f1)+999)) = udf(udf(t.tenthous)) +-- !query schema +struct +-- !query output + + + +-- !query +select * from + j1_tbl full join + (select * from j2_tbl order by udf(udf(j2_tbl.i)) desc, udf(j2_tbl.k) asc) j2_tbl + on udf(j1_tbl.i) = udf(j2_tbl.i) and udf(j1_tbl.i) = udf(j2_tbl.k) +-- !query schema +struct +-- !query output +0 NULL zero NULL NULL +1 4 one NULL NULL +2 3 two 2 2 +3 2 three NULL NULL +4 1 four NULL NULL +5 0 five NULL NULL +6 6 six NULL NULL +7 7 seven NULL NULL +8 8 eight NULL NULL +NULL 0 zero NULL NULL +NULL NULL NULL 0 NULL +NULL NULL NULL 1 -1 +NULL NULL NULL 2 4 +NULL NULL NULL 3 -3 +NULL NULL NULL 5 -5 +NULL NULL NULL 5 -5 +NULL NULL NULL NULL 0 +NULL NULL NULL NULL NULL +NULL NULL null NULL NULL + + +-- !query +select udf(count(*)) from + (select * from tenk1 x order by udf(x.thousand), udf(udf(x.twothousand)), x.fivethous) x + left join + (select * from tenk1 y order by udf(y.unique2)) y + on udf(x.thousand) = y.unique2 and x.twothousand = udf(y.hundred) and x.fivethous = y.unique2 +-- !query schema +struct +-- !query output +10000 + + +-- !query +DROP TABLE t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE t3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE J1_TBL +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE J2_TBL +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view tt1 as select * from + (values (1, 11), (2, NULL)) + as v(tt1_id, joincol) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view tt2 as select * from + (values (21, 11), (22, 11)) + as v(tt2_id, joincol) +-- !query schema +struct<> +-- !query output + + + +-- !query +select tt1.*, tt2.* from tt1 left join tt2 on udf(udf(tt1.joincol)) = udf(tt2.joincol) +-- !query schema +struct +-- !query output +1 11 21 11 +1 11 22 11 +2 NULL NULL NULL + + +-- !query +select tt1.*, tt2.* from tt2 right join tt1 on udf(udf(tt1.joincol)) = udf(udf(tt2.joincol)) +-- !query schema +struct +-- !query output +1 11 21 11 +1 11 22 11 +2 NULL NULL NULL + + +-- !query +select udf(count(*)) from tenk1 a, tenk1 b + where udf(a.hundred) = b.thousand and udf(udf((b.fivethous % 10)) < 10) +-- !query schema +struct +-- !query output +100000 + + +-- !query +DROP TABLE IF EXISTS tt3 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tt3(f1 int, f2 string) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO tt3 SELECT x.id, repeat('xyzzy', 100) FROM range(1,10001) x +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS tt4 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE tt4(f1 int) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO tt4 VALUES (0),(1),(9999) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(udf(a.f1)) as f1 +FROM tt4 a +LEFT JOIN ( + SELECT b.f1 + FROM tt3 b LEFT JOIN tt3 c ON udf(b.f1) = udf(c.f1) + WHERE udf(c.f1) IS NULL +) AS d ON udf(a.f1) = d.f1 +WHERE udf(udf(d.f1)) IS NULL +-- !query schema +struct +-- !query output +0 +1 +9999 + + +-- !query +create or replace temporary view tt5 as select * from + (values (1, 10), (1, 11)) + as v(f1, f2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view tt6 as select * from + (values (1, 9), (1, 2), (2, 9)) + as v(f1, f2) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from tt5,tt6 where udf(tt5.f1) = udf(tt6.f1) and udf(tt5.f1) = udf(udf(tt5.f2) - udf(tt6.f2)) +-- !query schema +struct +-- !query output +1 10 1 9 + + +-- !query +create or replace temporary view xx as select * from + (values (1), (2), (3)) + as v(pkxx) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view yy as select * from + (values (101, 1), (201, 2), (301, NULL)) + as v(pkyy, pkxx) +-- !query schema +struct<> +-- !query output + + + +-- !query +select udf(udf(yy.pkyy)) as yy_pkyy, udf(yy.pkxx) as yy_pkxx, udf(yya.pkyy) as yya_pkyy, + udf(xxa.pkxx) as xxa_pkxx, udf(xxb.pkxx) as xxb_pkxx +from yy + left join (SELECT * FROM yy where pkyy = 101) as yya ON udf(yy.pkyy) = udf(yya.pkyy) + left join xx xxa on udf(yya.pkxx) = udf(udf(xxa.pkxx)) + left join xx xxb on udf(udf(coalesce (xxa.pkxx, 1))) = udf(xxb.pkxx) +-- !query schema +struct +-- !query output +101 1 101 1 1 +201 2 NULL NULL 1 +301 NULL NULL NULL 1 + + +-- !query +create or replace temporary view zt1 as select * from + (values (53)) + as v(f1) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view zt2 as select * from + (values (53)) + as v(f2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view zt3(f3 int) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from + zt2 left join zt3 on (udf(f2) = udf(udf(f3))) + left join zt1 on (udf(udf(f3)) = udf(f1)) +where udf(f2) = 53 +-- !query schema +struct +-- !query output +53 NULL NULL + + +-- !query +create temp view zv1 as select *,'dummy' AS junk from zt1 +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from + zt2 left join zt3 on (f2 = udf(f3)) + left join zv1 on (udf(f3) = f1) +where udf(udf(f2)) = 53 +-- !query schema +struct +-- !query output +53 NULL NULL NULL + + +-- !query +select udf(a.unique2), udf(a.ten), udf(b.tenthous), udf(b.unique2), udf(b.hundred) +from tenk1 a left join tenk1 b on a.unique2 = udf(b.tenthous) +where udf(a.unique1) = 42 and + ((udf(b.unique2) is null and udf(a.ten) = 2) or udf(udf(b.hundred)) = udf(udf(3))) +-- !query schema +struct +-- !query output + + + +-- !query +create or replace temporary view a (i integer) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view b (x integer, y integer) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from a left join b on udf(i) = x and i = udf(y) and udf(x) = udf(i) +-- !query schema +struct +-- !query output + + + +-- !query +select udf(t1.q2), udf(count(t2.q1, t2.q2)) +from int8_tbl t1 left join int8_tbl t2 on (udf(udf(t1.q2)) = t2.q1) +group by udf(t1.q2) order by 1 +-- !query schema +struct +-- !query output +-4567890123456789 0 +123 2 +456 0 +4567890123456789 6 + + +-- !query +select udf(udf(t1.q2)), udf(count(t2.q1, t2.q2)) +from int8_tbl t1 left join (select * from int8_tbl) t2 on (udf(udf(t1.q2)) = udf(t2.q1)) +group by udf(udf(t1.q2)) order by 1 +-- !query schema +struct +-- !query output +-4567890123456789 0 +123 2 +456 0 +4567890123456789 6 + + +-- !query +select udf(t1.q2) as q2, udf(udf(count(t2.q1, t2.q2))) +from int8_tbl t1 left join + (select udf(q1) as q1, case when q2=1 then 1 else q2 end as q2 from int8_tbl) t2 + on (udf(t1.q2) = udf(t2.q1)) +group by t1.q2 order by 1 +-- !query schema +struct +-- !query output +-4567890123456789 0 +123 2 +456 0 +4567890123456789 6 + + +-- !query +create or replace temporary view a as select * from + (values ('p'), ('q')) + as v(code) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view b as select * from + (values ('p', 1), ('p', 2)) + as v(a, num) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view c as select * from + (values ('A', 'p'), ('B', 'q'), ('C', null)) + as v(name, a) +-- !query schema +struct<> +-- !query output + + + +-- !query +select udf(c.name), udf(ss.code), udf(ss.b_cnt), udf(ss.const) +from c left join + (select a.code, coalesce(b_grp.cnt, 0) as b_cnt, -1 as const + from a left join + (select udf(count(1)) as cnt, b.a as a from b group by b.a) as b_grp + on udf(a.code) = udf(udf(b_grp.a)) + ) as ss + on (udf(udf(c.a)) = udf(ss.code)) +order by c.name +-- !query schema +struct +-- !query output +A p 2 -1 +B q 0 -1 +C NULL NULL NULL + + +-- !query +SELECT * FROM +( SELECT 1 as key1 ) sub1 +LEFT JOIN +( SELECT sub3.key3, sub4.value2, COALESCE(sub4.value2, 66) as value3 FROM + ( SELECT 1 as key3 ) sub3 + LEFT JOIN + ( SELECT udf(sub5.key5) as key5, udf(udf(COALESCE(sub6.value1, 1))) as value2 FROM + ( SELECT 1 as key5 ) sub5 + LEFT JOIN + ( SELECT 2 as key6, 42 as value1 ) sub6 + ON sub5.key5 = udf(sub6.key6) + ) sub4 + ON udf(sub4.key5) = sub3.key3 +) sub2 +ON udf(udf(sub1.key1)) = udf(udf(sub2.key3)) +-- !query schema +struct +-- !query output +1 1 1 1 + + +-- !query +SELECT * FROM +( SELECT 1 as key1 ) sub1 +LEFT JOIN +( SELECT udf(sub3.key3) as key3, udf(value2), udf(COALESCE(value2, 66)) as value3 FROM + ( SELECT 1 as key3 ) sub3 + LEFT JOIN + ( SELECT sub5.key5, COALESCE(sub6.value1, 1) as value2 FROM + ( SELECT 1 as key5 ) sub5 + LEFT JOIN + ( SELECT 2 as key6, 42 as value1 ) sub6 + ON udf(udf(sub5.key5)) = sub6.key6 + ) sub4 + ON sub4.key5 = sub3.key3 +) sub2 +ON sub1.key1 = udf(udf(sub2.key3)) +-- !query schema +struct +-- !query output +1 1 1 1 + + +-- !query +SELECT udf(qq), udf(udf(unique1)) + FROM + ( SELECT udf(COALESCE(q1, 0)) AS qq FROM int8_tbl a ) AS ss1 + FULL OUTER JOIN + ( SELECT udf(udf(COALESCE(q2, -1))) AS qq FROM int8_tbl b ) AS ss2 + USING (qq) + INNER JOIN tenk1 c ON udf(qq) = udf(unique2) +-- !query schema +struct +-- !query output +123 4596 +123 4596 +456 7318 + + +-- !query +create or replace temporary view nt1 as select * from + (values(1,true,true), (2,true,false), (3,false,false)) + as v(id, a1, a2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view nt2 as select * from + (values(1,1,true,true), (2,2,true,false), (3,3,false,false)) + as v(id, nt1_id, b1, b2) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view nt3 as select * from + (values(1,1,true), (2,2,false), (3,3,true)) + as v(id, nt2_id, c1) +-- !query schema +struct<> +-- !query output + + + +-- !query +select udf(nt3.id) +from nt3 as nt3 + left join + (select nt2.*, (udf(nt2.b1) and udf(ss1.a3)) AS b3 + from nt2 as nt2 + left join + (select nt1.*, (udf(nt1.id) is not null) as a3 from nt1) as ss1 + on ss1.id = udf(udf(nt2.nt1_id)) + ) as ss2 + on udf(ss2.id) = nt3.nt2_id +where udf(nt3.id) = 1 and udf(ss2.b3) +-- !query schema +struct +-- !query output +1 + + +-- !query +select * from int4_tbl a full join int4_tbl b on true +-- !query schema +struct +-- !query output +-123456 -123456 +-123456 -2147483647 +-123456 0 +-123456 123456 +-123456 2147483647 +-2147483647 -123456 +-2147483647 -2147483647 +-2147483647 0 +-2147483647 123456 +-2147483647 2147483647 +0 -123456 +0 -2147483647 +0 0 +0 123456 +0 2147483647 +123456 -123456 +123456 -2147483647 +123456 0 +123456 123456 +123456 2147483647 +2147483647 -123456 +2147483647 -2147483647 +2147483647 0 +2147483647 123456 +2147483647 2147483647 + + +-- !query +select * from int4_tbl a full join int4_tbl b on false +-- !query schema +struct +-- !query output +-123456 NULL +-2147483647 NULL +0 NULL +123456 NULL +2147483647 NULL +NULL -123456 +NULL -2147483647 +NULL 0 +NULL 123456 +NULL 2147483647 + + +-- !query +select udf(count(*)) from + tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2) + left join tenk1 c on udf(a.unique2) = udf(b.unique1) and udf(c.thousand) = udf(udf(a.thousand)) + join int4_tbl on udf(b.thousand) = f1 +-- !query schema +struct +-- !query output +10 + + +-- !query +select udf(b.unique1) from + tenk1 a join tenk1 b on udf(a.unique1) = udf(b.unique2) + left join tenk1 c on udf(b.unique1) = 42 and c.thousand = udf(a.thousand) + join int4_tbl i1 on udf(b.thousand) = udf(udf(f1)) + right join int4_tbl i2 on udf(udf(i2.f1)) = udf(b.tenthous) + order by udf(1) +-- !query schema +struct +-- !query output +NULL +NULL +0 +NULL +NULL + + +-- !query +select * from +( + select udf(unique1), udf(q1), udf(udf(coalesce(unique1, -1)) + udf(q1)) as fault + from int8_tbl left join tenk1 on (udf(q2) = udf(unique2)) +) ss +where udf(fault) = udf(122) +order by udf(fault) +-- !query schema +struct +-- !query output +NULL 123 122 + + +-- !query +select udf(q1), udf(unique2), udf(thousand), udf(hundred) + from int8_tbl a left join tenk1 b on udf(q1) = udf(unique2) + where udf(coalesce(thousand,123)) = udf(q1) and udf(q1) = udf(udf(coalesce(hundred,123))) +-- !query schema +struct +-- !query output + + + +-- !query +select udf(f1), udf(unique2), case when udf(udf(unique2)) is null then udf(f1) else 0 end + from int4_tbl a left join tenk1 b on udf(f1) = udf(udf(unique2)) + where (case when udf(unique2) is null then udf(f1) else 0 end) = 0 +-- !query schema +struct +-- !query output +0 0 0 + + +-- !query +select udf(a.unique1), udf(b.unique1), udf(c.unique1), udf(coalesce(b.twothousand, a.twothousand)) + from tenk1 a left join tenk1 b on udf(b.thousand) = a.unique1 left join tenk1 c on udf(c.unique2) = udf(coalesce(b.twothousand, a.twothousand)) + where a.unique2 < udf(10) and udf(udf(coalesce(b.twothousand, a.twothousand))) = udf(44) +-- !query schema +struct +-- !query output + + + +-- !query +select * from + text_tbl t1 + inner join int8_tbl i8 + on udf(i8.q2) = udf(udf(456)) + right join text_tbl t2 + on udf(t1.f1) = udf(udf('doh!')) + left join int4_tbl i4 + on udf(udf(i8.q1)) = i4.f1 +-- !query schema +struct +-- !query output +doh! 123 456 doh! NULL +doh! 123 456 hi de ho neighbor NULL + + +-- !query +select * from + (select udf(udf(1)) as id) as xx + left join + (tenk1 as a1 full join (select udf(1) as id) as yy on (udf(a1.unique1) = udf(yy.id))) + on (xx.id = udf(udf(coalesce(yy.id)))) +-- !query schema +struct +-- !query output +1 1 2838 1 1 1 1 1 1 1 1 1 2 3 BAAAAA EFEAAA OOOOxx 1 + + +-- !query +select udf(a.q2), udf(b.q1) + from int8_tbl a left join int8_tbl b on udf(a.q2) = coalesce(b.q1, 1) + where udf(udf(coalesce(b.q1, 1)) > 0) +-- !query schema +struct +-- !query output +-4567890123456789 NULL +123 123 +123 123 +456 NULL +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query +create or replace temporary view parent as select * from + (values (1, 10), (2, 20), (3, 30)) + as v(k, pd) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view child as select * from + (values (1, 100), (4, 400)) + as v(k, cd) +-- !query schema +struct<> +-- !query output + + + +-- !query +select p.* from parent p left join child c on (udf(p.k) = udf(c.k)) +-- !query schema +struct +-- !query output +1 10 +2 20 +3 30 + + +-- !query +select p.*, linked from parent p + left join (select c.*, udf(udf(true)) as linked from child c) as ss + on (udf(p.k) = udf(udf(ss.k))) +-- !query schema +struct +-- !query output +1 10 true +2 20 NULL +3 30 NULL + + +-- !query +select p.* from + parent p left join child c on (udf(p.k) = c.k) + where p.k = udf(1) and udf(udf(p.k)) = udf(udf(2)) +-- !query schema +struct +-- !query output + + + +-- !query +select p.* from + (parent p left join child c on (udf(p.k) = c.k)) join parent x on p.k = udf(x.k) + where udf(p.k) = udf(1) and udf(udf(p.k)) = udf(udf(2)) +-- !query schema +struct +-- !query output + + + +-- !query +create or replace temporary view a as select * from + (values (0), (1)) + as v(id) +-- !query schema +struct<> +-- !query output + + + +-- !query +create or replace temporary view b as select * from + (values (0, 0), (1, NULL)) + as v(id, a_id) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(udf(a.id)) IS NULL OR udf(a.id) > 0) +-- !query schema +struct +-- !query output +1 NULL NULL + + +-- !query +SELECT b.* FROM b LEFT JOIN a ON (udf(b.a_id) = udf(a.id)) WHERE (udf(a.id) IS NULL OR udf(udf(a.id)) > 0) +-- !query schema +struct +-- !query output +1 NULL + + +-- !query +create or replace temporary view innertab as select * from + (values (123L, 42L)) + as v(id, dat1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM + (SELECT udf(1) AS x) ss1 + LEFT JOIN + (SELECT udf(q1), udf(q2), udf(COALESCE(dat1, q1)) AS y + FROM int8_tbl LEFT JOIN innertab ON udf(udf(q2)) = id) ss2 + ON true +-- !query schema +struct +-- !query output +1 123 456 123 +1 123 4567890123456789 123 +1 4567890123456789 -4567890123456789 4567890123456789 +1 4567890123456789 123 42 +1 4567890123456789 4567890123456789 4567890123456789 + + +-- !query +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(f1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`f1`", + "referenceNames" : "[`j`.`f1`, `j`.`f1`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 87, + "stopIndex" : 88, + "fragment" : "f1" + } ] +} + + +-- !query +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y) j on udf(q1) = udf(y.f1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`y`.`f1`", + "proposal" : "`j`.`f1`, `j`.`f1`, `x`.`q1`, `x`.`q2`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 87, + "stopIndex" : 90, + "fragment" : "y.f1" + } ] +} + + +-- !query +select * from + int8_tbl x join (int4_tbl x cross join int4_tbl y(ff)) j on udf(q1) = udf(udf(f1)) +-- !query schema +struct +-- !query output + + + +-- !query +select udf(t1.uunique1) from + tenk1 t1 join tenk2 t2 on t1.two = udf(t2.two) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t1`.`uunique1`", + "proposal" : "`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`four`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 22, + "fragment" : "t1.uunique1" + } ] +} + + +-- !query +select udf(udf(t2.uunique1)) from + tenk1 t1 join tenk2 t2 on udf(t1.two) = t2.two +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`t2`.`uunique1`", + "proposal" : "`t2`.`unique1`, `t1`.`unique1`, `t2`.`unique2`, `t1`.`unique2`, `t2`.`four`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 16, + "stopIndex" : 26, + "fragment" : "t2.uunique1" + } ] +} + + +-- !query +select udf(uunique1) from + tenk1 t1 join tenk2 t2 on udf(t1.two) = udf(t2.two) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`uunique1`", + "proposal" : "`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`ten`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 12, + "stopIndex" : 19, + "fragment" : "uunique1" + } ] +} + + +-- !query +select unique2, x.* +from tenk1 a, lateral (select * from int4_tbl b where f1 = a.unique1) x +-- !query schema +struct +-- !query output +9998 0 + + +-- !query +select unique2, x.* +from int4_tbl x, lateral (select unique2 from tenk1 where f1 = unique1) ss +-- !query schema +struct +-- !query output +9998 0 + + +-- !query +select unique2, x.* +from int4_tbl x left join lateral (select unique1, unique2 from tenk1 where f1 = unique1) ss on true +-- !query schema +struct +-- !query output +9998 0 +NULL -123456 +NULL -2147483647 +NULL 123456 +NULL 2147483647 + + +-- !query +select * from (select f1/2 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1, + lateral (select x) ss2(y) +-- !query schema +struct +-- !query output +0.0 0 0.0 + + +-- !query +select * from ((select f1/2 as x from int4_tbl) ss1 join int4_tbl i4 on x = f1) j, + lateral (select x) ss2(y) +-- !query schema +struct +-- !query output +0.0 0 0.0 + + +-- !query +select * from + int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1, + lateral (select x.q1,y.q1,y.q2) v(xq1,yq1,yq2) +-- !query schema +struct +-- !query output +123 456 NULL NULL 123 NULL NULL +123 4567890123456789 4567890123456789 -4567890123456789 123 4567890123456789 -4567890123456789 +123 4567890123456789 4567890123456789 123 123 4567890123456789 123 +123 4567890123456789 4567890123456789 4567890123456789 123 4567890123456789 4567890123456789 +4567890123456789 -4567890123456789 NULL NULL 4567890123456789 NULL NULL +4567890123456789 123 123 456 4567890123456789 123 456 +4567890123456789 123 123 4567890123456789 4567890123456789 123 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 -4567890123456789 4567890123456789 4567890123456789 -4567890123456789 +4567890123456789 4567890123456789 4567890123456789 123 4567890123456789 4567890123456789 123 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 + + +-- !query +select x.* from + int8_tbl x left join (select q1,coalesce(q2,0) q2 from int8_tbl) y on x.q2 = y.q1, + lateral (select x.q1,y.q1,y.q2) v(xq1,yq1,yq2) +-- !query schema +struct +-- !query output +123 456 +123 4567890123456789 +123 4567890123456789 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 123 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query +select * from + int8_tbl a left join + lateral (select *, a.q2 as x from int8_tbl b) ss on a.q2 = ss.q1 +-- !query schema +struct +-- !query output +123 456 NULL NULL NULL +123 4567890123456789 4567890123456789 -4567890123456789 4567890123456789 +123 4567890123456789 4567890123456789 123 4567890123456789 +123 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 -4567890123456789 NULL NULL NULL +4567890123456789 123 123 456 123 +4567890123456789 123 123 4567890123456789 123 +4567890123456789 4567890123456789 4567890123456789 -4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 123 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 + + +-- !query +select * from + int8_tbl a left join + lateral (select *, coalesce(a.q2, 42) as x from int8_tbl b) ss on a.q2 = ss.q1 +-- !query schema +struct +-- !query output +123 456 NULL NULL NULL +123 4567890123456789 4567890123456789 -4567890123456789 4567890123456789 +123 4567890123456789 4567890123456789 123 4567890123456789 +123 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 -4567890123456789 NULL NULL NULL +4567890123456789 123 123 456 123 +4567890123456789 123 123 4567890123456789 123 +4567890123456789 4567890123456789 4567890123456789 -4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 123 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 + + +-- !query +select * from int4_tbl i left join + lateral (select * from int2_tbl j where i.f1 = j.f1) k on true +-- !query schema +struct +-- !query output +-123456 NULL +-2147483647 NULL +0 0 +123456 NULL +2147483647 NULL + + +-- !query +select * from + int8_tbl a left join lateral + (select b.q1 as bq1, c.q1 as cq1, least(a.q1,b.q1,c.q1) from + int8_tbl b cross join int8_tbl c) ss + on a.q2 = ss.bq1 +-- !query schema +struct +-- !query output +123 456 NULL NULL NULL +123 4567890123456789 4567890123456789 123 123 +123 4567890123456789 4567890123456789 123 123 +123 4567890123456789 4567890123456789 123 123 +123 4567890123456789 4567890123456789 123 123 +123 4567890123456789 4567890123456789 123 123 +123 4567890123456789 4567890123456789 123 123 +123 4567890123456789 4567890123456789 4567890123456789 123 +123 4567890123456789 4567890123456789 4567890123456789 123 +123 4567890123456789 4567890123456789 4567890123456789 123 +123 4567890123456789 4567890123456789 4567890123456789 123 +123 4567890123456789 4567890123456789 4567890123456789 123 +123 4567890123456789 4567890123456789 4567890123456789 123 +123 4567890123456789 4567890123456789 4567890123456789 123 +123 4567890123456789 4567890123456789 4567890123456789 123 +123 4567890123456789 4567890123456789 4567890123456789 123 +4567890123456789 -4567890123456789 NULL NULL NULL +4567890123456789 123 123 123 123 +4567890123456789 123 123 123 123 +4567890123456789 123 123 123 123 +4567890123456789 123 123 123 123 +4567890123456789 123 123 4567890123456789 123 +4567890123456789 123 123 4567890123456789 123 +4567890123456789 123 123 4567890123456789 123 +4567890123456789 123 123 4567890123456789 123 +4567890123456789 123 123 4567890123456789 123 +4567890123456789 123 123 4567890123456789 123 +4567890123456789 4567890123456789 4567890123456789 123 123 +4567890123456789 4567890123456789 4567890123456789 123 123 +4567890123456789 4567890123456789 4567890123456789 123 123 +4567890123456789 4567890123456789 4567890123456789 123 123 +4567890123456789 4567890123456789 4567890123456789 123 123 +4567890123456789 4567890123456789 4567890123456789 123 123 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 4567890123456789 + + +-- !query +select udf(udf(f1,g)) from int4_tbl a, (select udf(udf(f1)) as g) ss +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`f1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 56, + "stopIndex" : 57, + "fragment" : "f1" + } ] +} + + +-- !query +select udf(f1,g) from int4_tbl a, (select a.f1 as g) ss +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`.`f1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 43, + "stopIndex" : 46, + "fragment" : "a.f1" + } ] +} + + +-- !query +select udf(udf(f1,g)) from int4_tbl a cross join (select udf(f1) as g) ss +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`f1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 62, + "stopIndex" : 63, + "fragment" : "f1" + } ] +} + + +-- !query +select udf(f1,g) from int4_tbl a cross join (select udf(udf(a.f1)) as g) ss +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`.`f1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 61, + "stopIndex" : 64, + "fragment" : "a.f1" + } ] +} + + +-- !query +CREATE TABLE j1 (id1 int, id2 int) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE j2 (id1 int, id2 int) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO j1 values(1,1),(1,2) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO j2 values(1,1) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO j2 values(1,2) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from j1 +inner join j2 on udf(j1.id1) = udf(j2.id1) and udf(udf(j1.id2)) = udf(j2.id2) +where udf(j1.id1) % 1000 = 1 and udf(udf(j2.id1) % 1000) = 1 +-- !query schema +struct +-- !query output +1 1 1 1 +1 2 1 2 + + +-- !query +drop table j1 +-- !query schema +struct<> +-- !query output + + + +-- !query +drop table j2 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out new file mode 100644 index 000000000000..a220e6487f81 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out @@ -0,0 +1,222 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE test_having (a int, b int, c string, d string) USING parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (0, 1, 'XXXX', 'A') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (1, 2, 'AAAA', 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (2, 2, 'AAAA', 'c') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (3, 3, 'BBBB', 'D') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (4, 3, 'BBBB', 'e') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (5, 3, 'bbbb', 'F') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (6, 4, 'cccc', 'g') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (7, 4, 'cccc', 'h') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (8, 4, 'CCCC', 'I') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_having VALUES (9, 4, 'CCCC', 'j') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(b), udf(c) FROM test_having + GROUP BY b, c HAVING udf(count(*)) = 1 ORDER BY udf(b), udf(c) +-- !query schema +struct +-- !query output +1 XXXX +3 bbbb + + +-- !query +SELECT udf(b), udf(c) FROM test_having + GROUP BY b, c HAVING udf(b) = 3 ORDER BY udf(b), udf(c) +-- !query schema +struct +-- !query output +3 BBBB +3 bbbb + + +-- !query +SELECT udf(c), max(udf(a)) FROM test_having + GROUP BY c HAVING udf(count(*)) > 2 OR udf(min(a)) = udf(max(a)) + ORDER BY c +-- !query schema +struct +-- !query output +XXXX 0 +bbbb 5 + + +-- !query +SELECT udf(udf(min(udf(a)))), udf(udf(max(udf(a)))) FROM test_having HAVING udf(udf(min(udf(a)))) = udf(udf(max(udf(a)))) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(min(udf(a))), udf(udf(max(a))) FROM test_having HAVING udf(min(a)) < udf(max(udf(a))) +-- !query schema +struct +-- !query output +0 9 + + +-- !query +SELECT udf(a) FROM test_having HAVING udf(min(a)) < udf(max(a)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 63, + "fragment" : "SELECT udf(a) FROM test_having HAVING udf(min(a)) < udf(max(a))" + } ] +} + + +-- !query +SELECT 1 AS one FROM test_having HAVING udf(a) > 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`", + "proposal" : "`one`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 45, + "fragment" : "a" + } ] +} + + +-- !query +SELECT 1 AS one FROM test_having HAVING udf(udf(1) > udf(2)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 1 AS one FROM test_having HAVING udf(udf(1) < udf(2)) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT 1 AS one FROM test_having WHERE 1/udf(a) = 1 HAVING 1 < 2 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 40, + "stopIndex" : 47, + "fragment" : "1/udf(a)" + } ] +} + + +-- !query +DROP TABLE test_having +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out new file mode 100755 index 000000000000..a3a7cee4eaa7 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out @@ -0,0 +1,510 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TABLE test_missing_target (a int, b int, c string, d string) using parquet +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (0, 1, 'XXXX', 'A') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (1, 2, 'ABAB', 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (2, 2, 'ABAB', 'c') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (3, 3, 'BBBB', 'D') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (4, 3, 'BBBB', 'e') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (5, 3, 'bbbb', 'F') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (6, 4, 'cccc', 'g') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (7, 4, 'cccc', 'h') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (8, 4, 'CCCC', 'I') +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO test_missing_target VALUES (9, 4, 'CCCC', 'j') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY +udf(test_missing_target.c) +ORDER BY udf(c) +-- !query schema +struct +-- !query output +ABAB 2 +BBBB 2 +CCCC 2 +XXXX 1 +bbbb 1 +cccc 2 + + +-- !query +SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(test_missing_target.c) +ORDER BY udf(c) +-- !query schema +struct +-- !query output +2 +2 +2 +1 +1 +2 + + +-- !query +SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(a) ORDER BY udf(b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`b`", + "proposal" : "`udf(count(1))`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 76, + "stopIndex" : 76, + "fragment" : "b" + } ] +} + + +-- !query +SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(b) ORDER BY udf(b) +-- !query schema +struct +-- !query output +1 +2 +3 +4 + + +-- !query +SELECT udf(test_missing_target.b), udf(count(*)) + FROM test_missing_target GROUP BY udf(b) ORDER BY udf(b) +-- !query schema +struct +-- !query output +1 1 +2 2 +3 3 +4 4 + + +-- !query +SELECT udf(c) FROM test_missing_target ORDER BY udf(a) +-- !query schema +struct +-- !query output +XXXX +ABAB +ABAB +BBBB +BBBB +bbbb +cccc +cccc +CCCC +CCCC + + +-- !query +SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(b) ORDER BY udf(b) desc +-- !query schema +struct +-- !query output +4 +3 +2 +1 + + +-- !query +SELECT udf(count(*)) FROM test_missing_target ORDER BY udf(1) desc +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY 1 ORDER BY 1 +-- !query schema +struct +-- !query output +ABAB 2 +BBBB 2 +CCCC 2 +XXXX 1 +bbbb 1 +cccc 2 + + +-- !query +SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE", + "sqlState" : "42805", + "messageParameters" : { + "index" : "3", + "size" : "2" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 64, + "stopIndex" : 64, + "fragment" : "3" + } ] +} + + +-- !query +SELECT udf(count(*)) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY udf(b) ORDER BY udf(b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`b`", + "referenceNames" : "[`x`.`b`, `y`.`b`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 113, + "stopIndex" : 113, + "fragment" : "b" + } ] +} + + +-- !query +SELECT udf(a), udf(a) FROM test_missing_target + ORDER BY udf(a) +-- !query schema +struct +-- !query output +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 + + +-- !query +SELECT udf(udf(a)/2), udf(udf(a)/2) FROM test_missing_target + ORDER BY udf(udf(a)/2) +-- !query schema +struct +-- !query output +0.0 0.0 +0.5 0.5 +1.0 1.0 +1.5 1.5 +2.0 2.0 +2.5 2.5 +3.0 3.0 +3.5 3.5 +4.0 4.0 +4.5 4.5 + + +-- !query +SELECT udf(a/2), udf(a/2) FROM test_missing_target + GROUP BY udf(a/2) ORDER BY udf(a/2) +-- !query schema +struct +-- !query output +0.0 0.0 +0.5 0.5 +1.0 1.0 +1.5 1.5 +2.0 2.0 +2.5 2.5 +3.0 3.0 +3.5 3.5 +4.0 4.0 +4.5 4.5 + + +-- !query +SELECT udf(x.b), udf(count(*)) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY udf(x.b) ORDER BY udf(x.b) +-- !query schema +struct +-- !query output +1 1 +2 2 +3 3 +4 4 + + +-- !query +SELECT udf(count(*)) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY udf(x.b) ORDER BY udf(x.b) +-- !query schema +struct +-- !query output +1 +2 +3 +4 + + +-- !query +SELECT udf(a%2), udf(count(udf(b))) FROM test_missing_target +GROUP BY udf(test_missing_target.a%2) +ORDER BY udf(test_missing_target.a%2) +-- !query schema +struct +-- !query output +0 5 +1 5 + + +-- !query +SELECT udf(count(c)) FROM test_missing_target +GROUP BY udf(lower(test_missing_target.c)) +ORDER BY udf(lower(test_missing_target.c)) +-- !query schema +struct +-- !query output +2 +3 +4 +1 + + +-- !query +SELECT udf(count(udf(a))) FROM test_missing_target GROUP BY udf(a) ORDER BY udf(b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`b`", + "proposal" : "`udf(count(udf(a)))`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 81, + "stopIndex" : 81, + "fragment" : "b" + } ] +} + + +-- !query +SELECT udf(count(b)) FROM test_missing_target GROUP BY udf(b/2) ORDER BY udf(b/2) +-- !query schema +struct +-- !query output +1 +2 +3 +4 + + +-- !query +SELECT udf(lower(test_missing_target.c)), udf(count(udf(c))) + FROM test_missing_target GROUP BY udf(lower(c)) ORDER BY udf(lower(c)) +-- !query schema +struct +-- !query output +abab 2 +bbbb 3 +cccc 4 +xxxx 1 + + +-- !query +SELECT udf(a) FROM test_missing_target ORDER BY udf(upper(udf(d))) +-- !query schema +struct +-- !query output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +SELECT udf(count(b)) FROM test_missing_target + GROUP BY udf((b + 1) / 2) ORDER BY udf((b + 1) / 2) desc +-- !query schema +struct +-- !query output +4 +3 +2 +1 + + +-- !query +SELECT udf(count(udf(x.a))) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY udf(b/2) ORDER BY udf(b/2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`b`", + "referenceNames" : "[`x`.`b`, `y`.`b`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 120, + "stopIndex" : 120, + "fragment" : "b" + } ] +} + + +-- !query +SELECT udf(x.b/2), udf(count(udf(x.b))) FROM test_missing_target x, +test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY udf(x.b/2) ORDER BY udf(x.b/2) +-- !query schema +struct +-- !query output +0.5 1 +1.0 2 +1.5 3 +2.0 4 + + +-- !query +SELECT udf(count(udf(b))) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY udf(x.b/2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "AMBIGUOUS_REFERENCE", + "sqlState" : "42704", + "messageParameters" : { + "name" : "`b`", + "referenceNames" : "[`x`.`b`, `y`.`b`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 22, + "stopIndex" : 22, + "fragment" : "b" + } ] +} + + +-- !query +DROP TABLE test_missing_target +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-count.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-count.sql.out new file mode 100644 index 000000000000..ce8d5a93a35b --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-count.sql.out @@ -0,0 +1,52 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + udf(count(*)), udf(count(1)), udf(count(null)), udf(count(a)), udf(count(b)), udf(count(a + b)), udf(count((a, b))) +FROM testData +-- !query schema +struct +-- !query output +7 7 0 5 5 4 7 + + +-- !query +SELECT + udf(count(DISTINCT 1)), + udf(count(DISTINCT null)), + udf(count(DISTINCT a)), + udf(count(DISTINCT b)), + udf(count(DISTINCT (a + b))), + udf(count(DISTINCT (a, b))) +FROM testData +-- !query schema +struct +-- !query output +1 0 2 2 2 6 + + +-- !query +SELECT udf(count(a, b)), udf(count(b, a)), udf(count(testData.*, testData.*)) FROM testData +-- !query schema +struct +-- !query output +4 4 4 + + +-- !query +SELECT + udf(count(DISTINCT a, b)), udf(count(DISTINCT b, a)), udf(count(DISTINCT *)), udf(count(DISTINCT testData.*, testData.*)) +FROM testData +-- !query schema +struct +-- !query output +3 3 3 3 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-cross-join.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-cross-join.sql.out new file mode 100644 index 000000000000..edfd359a7f46 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-cross-join.sql.out @@ -0,0 +1,137 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM nt1 cross join nt2 +-- !query schema +struct +-- !query output +one 1 one 1 +one 1 one 5 +one 1 two 22 +three 3 one 1 +three 3 one 5 +three 3 two 22 +two 2 one 1 +two 2 one 5 +two 2 two 22 + + +-- !query +SELECT * FROM nt1 cross join nt2 where udf(nt1.k) = udf(nt2.k) +-- !query schema +struct +-- !query output +one 1 one 1 +one 1 one 5 +two 2 two 22 + + +-- !query +SELECT * FROM nt1 cross join nt2 on (udf(nt1.k) = udf(nt2.k)) +-- !query schema +struct +-- !query output +one 1 one 1 +one 1 one 5 +two 2 two 22 + + +-- !query +SELECT * FROM nt1 cross join nt2 where udf(nt1.v1) = "1" and udf(nt2.v2) = "22" +-- !query schema +struct +-- !query output +one 1 two 22 + + +-- !query +SELECT udf(a.key), udf(b.key) FROM +(SELECT udf(k) key FROM nt1 WHERE v1 < 2) a +CROSS JOIN +(SELECT udf(k) key FROM nt2 WHERE v2 = 22) b +-- !query schema +struct +-- !query output +one two + + +-- !query +create temporary view A(a, va) as select * from nt1 +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view B(b, vb) as select * from nt1 +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view C(c, vc) as select * from nt1 +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view D(d, vd) as select * from nt1 +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from ((A join B on (udf(a) = udf(b))) cross join C) join D on (udf(a) = udf(d)) +-- !query schema +struct +-- !query output +one 1 one 1 one 1 one 1 +one 1 one 1 three 3 one 1 +one 1 one 1 two 2 one 1 +three 3 three 3 one 1 three 3 +three 3 three 3 three 3 three 3 +three 3 three 3 two 2 three 3 +two 2 two 2 one 1 two 2 +two 2 two 2 three 3 two 2 +two 2 two 2 two 2 two 2 + + +-- !query +SELECT * FROM nt1 CROSS JOIN nt2 ON (udf(nt1.k) > udf(nt2.k)) +-- !query schema +struct +-- !query output +three 3 one 1 +three 3 one 5 +two 2 one 1 +two 2 one 5 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-except-all.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-except-all.sql.out new file mode 100644 index 000000000000..ad1b0aabcdec --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-except-all.sql.out @@ -0,0 +1,377 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (0), (1), (2), (2), (2), (2), (3), (null), (null) AS tab1(c1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1), (2), (2), (3), (5), (5), (null) AS tab2(c1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW tab3 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (2, 3), + (2, 2) + AS tab3(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW tab4 AS SELECT * FROM VALUES + (1, 2), + (2, 3), + (2, 2), + (2, 2), + (2, 20) + AS tab4(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 +-- !query schema +struct +-- !query output +0 +2 +2 +NULL + + +-- !query +SELECT udf(c1) FROM tab1 +MINUS ALL +SELECT udf(c1) FROM tab2 +-- !query schema +struct +-- !query output +0 +2 +2 +NULL + + +-- !query +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 WHERE udf(c1) IS NOT NULL +-- !query schema +struct +-- !query output +0 +2 +2 +NULL +NULL + + +-- !query +SELECT udf(c1) FROM tab1 WHERE udf(c1) > 5 +EXCEPT ALL +SELECT udf(c1) FROM tab2 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 WHERE udf(c1 > udf(6)) +-- !query schema +struct +-- !query output +0 +1 +2 +2 +2 +2 +3 +NULL +NULL + + +-- !query +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT CAST(udf(1) AS BIGINT) +-- !query schema +struct +-- !query output +0 +2 +2 +2 +2 +3 +NULL +NULL + + +-- !query +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT array(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"ARRAY\"", + "dataType2" : "\"INT\"", + "hint" : "", + "operator" : "EXCEPT ALL", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 51, + "fragment" : "SELECT udf(c1) FROM tab1\nEXCEPT ALL\nSELECT array(1)" + } ] +} + + +-- !query +SELECT udf(k), v FROM tab3 +EXCEPT ALL +SELECT k, udf(v) FROM tab4 +-- !query schema +struct +-- !query output +1 2 +1 3 + + +-- !query +SELECT k, udf(v) FROM tab4 +EXCEPT ALL +SELECT udf(k), v FROM tab3 +-- !query schema +struct +-- !query output +2 2 +2 20 + + +-- !query +SELECT udf(k), udf(v) FROM tab4 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab3 +INTERSECT DISTINCT +SELECT udf(k), udf(v) FROM tab4 +-- !query schema +struct +-- !query output +2 2 +2 20 + + +-- !query +SELECT udf(k), v FROM tab4 +EXCEPT ALL +SELECT k, udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), udf(v) FROM tab4 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT k, udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab4 +UNION ALL +SELECT udf(k), v FROM tab3 +EXCEPT DISTINCT +SELECT k, udf(v) FROM tab4 +-- !query schema +struct +-- !query output +1 3 + + +-- !query +SELECT k FROM tab3 +EXCEPT ALL +SELECT k, v FROM tab4 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NUM_COLUMNS_MISMATCH", + "sqlState" : "42826", + "messageParameters" : { + "firstNumColumns" : "1", + "invalidNumColumns" : "2", + "invalidOrdinalNum" : "second", + "operator" : "EXCEPT ALL" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 51, + "fragment" : "SELECT k FROM tab3\nEXCEPT ALL\nSELECT k, v FROM tab4" + } ] +} + + +-- !query +SELECT udf(k), udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab4 +UNION +SELECT udf(k), udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), udf(v) FROM tab4 +-- !query schema +struct +-- !query output +1 3 + + +-- !query +SELECT udf(k), udf(v) FROM tab3 +MINUS ALL +SELECT k, udf(v) FROM tab4 +UNION +SELECT udf(k), udf(v) FROM tab3 +MINUS DISTINCT +SELECT k, udf(v) FROM tab4 +-- !query schema +struct +-- !query output +1 3 + + +-- !query +SELECT k, udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), v FROM tab4 +EXCEPT DISTINCT +SELECT k, udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), v FROM tab4 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM (SELECT tab3.k, + udf(tab4.v) + FROM tab3 + JOIN tab4 + ON udf(tab3.k) = tab4.k) +EXCEPT ALL +SELECT * +FROM (SELECT udf(tab3.k), + tab4.v + FROM tab3 + JOIN tab4 + ON tab3.k = udf(tab4.k)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM (SELECT udf(udf(tab3.k)), + udf(tab4.v) + FROM tab3 + JOIN tab4 + ON udf(udf(tab3.k)) = udf(tab4.k)) +EXCEPT ALL +SELECT * +FROM (SELECT udf(tab4.v) AS k, + udf(udf(tab3.k)) AS v + FROM tab3 + JOIN tab4 + ON udf(tab3.k) = udf(tab4.k)) +-- !query schema +struct +-- !query output +1 2 +1 2 +1 2 +2 20 +2 20 +2 3 +2 3 + + +-- !query +SELECT udf(v) FROM tab3 GROUP BY v +EXCEPT ALL +SELECT udf(k) FROM tab4 GROUP BY k +-- !query schema +struct +-- !query output +3 + + +-- !query +DROP VIEW IF EXISTS tab1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS tab2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS tab3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS tab4 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-except.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-except.sql.out new file mode 100644 index 000000000000..14ecf98c7a83 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-except.sql.out @@ -0,0 +1,102 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view t1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", NULL) + as t1(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view t2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5), + ("one", NULL), + (NULL, 5) + as t2(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(k), udf(v) FROM t1 EXCEPT SELECT udf(k), udf(v) FROM t2 +-- !query schema +struct +-- !query output +three 3 +two 2 + + +-- !query +SELECT * FROM t1 EXCEPT SELECT * FROM t1 where udf(v) <> 1 and v <> udf(2) +-- !query schema +struct +-- !query output +one 1 +one NULL +two 2 + + +-- !query +SELECT * FROM t1 where udf(v) <> 1 and v <> udf(22) EXCEPT SELECT * FROM t1 where udf(v) <> 2 and v >= udf(3) +-- !query schema +struct +-- !query output +two 2 + + +-- !query +SELECT t1.* FROM t1, t2 where t1.k = t2.k +EXCEPT +SELECT t1.* FROM t1, t2 where t1.k = t2.k and t1.k != udf('one') +-- !query schema +struct +-- !query output +one 1 +one NULL + + +-- !query +SELECT * FROM t2 where v >= udf(1) and udf(v) <> 22 EXCEPT SELECT * FROM t1 +-- !query schema +struct +-- !query output +NULL 5 +one 5 + + +-- !query +SELECT (SELECT min(udf(k)) FROM t2 WHERE t2.k = t1.k) min_t2 FROM t1 +MINUS +SELECT (SELECT udf(min(k)) FROM t2) abs_min_t2 FROM t1 WHERE t1.k = udf('one') +-- !query schema +struct +-- !query output +NULL +two + + +-- !query +SELECT t1.k +FROM t1 +WHERE t1.v <= (SELECT udf(max(udf(t2.v))) + FROM t2 + WHERE udf(t2.k) = udf(t1.k)) +MINUS +SELECT t1.k +FROM t1 +WHERE udf(t1.v) >= (SELECT min(udf(t2.v)) + FROM t2 + WHERE t2.k = t1.k) +-- !query schema +struct +-- !query output +two diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-group-analytics.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-group-analytics.sql.out new file mode 100644 index 000000000000..9beee9972ab7 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-group-analytics.sql.out @@ -0,0 +1,423 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(a + b), b, udf(SUM(a - b)) FROM testData GROUP BY udf(a + b), b WITH CUBE +-- !query schema +struct +-- !query output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL 1 3 +NULL 2 0 +NULL NULL 3 + + +-- !query +SELECT udf(a), udf(b), SUM(b) FROM testData GROUP BY udf(a), b WITH CUBE +-- !query schema +struct +-- !query output +1 1 1 +1 2 2 +1 NULL 3 +2 1 1 +2 2 2 +2 NULL 3 +3 1 1 +3 2 2 +3 NULL 3 +NULL 1 3 +NULL 2 6 +NULL NULL 9 + + +-- !query +SELECT udf(a + b), b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP +-- !query schema +struct +-- !query output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL NULL 3 + + +-- !query +SELECT udf(a), b, udf(SUM(b)) FROM testData GROUP BY udf(a), b WITH ROLLUP +-- !query schema +struct +-- !query output +1 1 1 +1 2 2 +1 NULL 3 +2 1 1 +2 2 2 +2 NULL 3 +3 1 1 +3 2 2 +3 NULL 3 +NULL NULL 9 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW courseSales AS SELECT * FROM VALUES +("dotNET", 2012, 10000), ("Java", 2012, 20000), ("dotNET", 2012, 5000), ("dotNET", 2013, 48000), ("Java", 2013, 30000) +AS courseSales(course, year, earnings) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY udf(course), year +-- !query schema +struct +-- !query output +NULL NULL 113000 +Java NULL 50000 +Java 2012 20000 +Java 2013 30000 +dotNET NULL 63000 +dotNET 2012 15000 +dotNET 2013 48000 + + +-- !query +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, udf(year) +-- !query schema +struct +-- !query output +NULL NULL 113000 +NULL 2012 35000 +NULL 2013 78000 +Java NULL 50000 +Java 2012 20000 +Java 2013 30000 +dotNET NULL 63000 +dotNET 2012 15000 +dotNET 2013 48000 + + +-- !query +SELECT course, udf(year), SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year) +-- !query schema +struct +-- !query output +Java NULL 50000 +NULL 2012 35000 +NULL 2013 78000 +dotNET NULL 63000 + + +-- !query +SELECT course, year, udf(SUM(earnings)) FROM courseSales GROUP BY course, year GROUPING SETS(course) +-- !query schema +struct +-- !query output +Java NULL 50000 +dotNET NULL 63000 + + +-- !query +SELECT udf(course), year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year) +-- !query schema +struct +-- !query output +NULL 2012 35000 +NULL 2013 78000 + + +-- !query +SELECT course, udf(SUM(earnings)) AS sum FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, udf(sum) +-- !query schema +struct +-- !query output +NULL 113000 +Java 20000 +Java 30000 +Java 50000 +dotNET 5000 +dotNET 10000 +dotNET 48000 +dotNET 63000 + + +-- !query +SELECT course, SUM(earnings) AS sum, GROUPING_ID(course, earnings) FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY udf(course), sum +-- !query schema +struct +-- !query output +NULL 113000 3 +Java 20000 0 +Java 30000 0 +Java 50000 1 +dotNET 5000 0 +dotNET 10000 0 +dotNET 48000 0 +dotNET 63000 1 + + +-- !query +SELECT udf(course), udf(year), GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales +GROUP BY CUBE(course, year) +-- !query schema +struct +-- !query output +Java 2012 0 0 0 +Java 2013 0 0 0 +Java NULL 0 1 1 +NULL 2012 1 0 2 +NULL 2013 1 0 2 +NULL NULL 1 1 3 +dotNET 2012 0 0 0 +dotNET 2013 0 0 0 +dotNET NULL 0 1 1 + + +-- !query +SELECT course, udf(year), GROUPING(course) FROM courseSales GROUP BY course, udf(year) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", + "sqlState" : "42K0E", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 42, + "fragment" : "GROUPING(course)" + } ] +} + + +-- !query +SELECT course, udf(year), GROUPING_ID(course, year) FROM courseSales GROUP BY udf(course), year +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", + "sqlState" : "42K0E", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 27, + "stopIndex" : 51, + "fragment" : "GROUPING_ID(course, year)" + } ] +} + + +-- !query +SELECT course, year, grouping__id FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, udf(year) +-- !query schema +struct +-- !query output +Java 2012 0 +Java 2013 0 +dotNET 2012 0 +dotNET 2013 0 +Java NULL 1 +dotNET NULL 1 +NULL 2012 2 +NULL 2013 2 +NULL NULL 3 + + +-- !query +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) +HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0 ORDER BY course, udf(year) +-- !query schema +struct +-- !query output +NULL NULL +Java NULL +dotNET NULL + + +-- !query +SELECT course, udf(year) FROM courseSales GROUP BY udf(course), year HAVING GROUPING(course) > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", + "sqlState" : "42K0E" +} + + +-- !query +SELECT course, udf(udf(year)) FROM courseSales GROUP BY course, year HAVING GROUPING_ID(course) > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", + "sqlState" : "42K0E" +} + + +-- !query +SELECT udf(course), year FROM courseSales GROUP BY CUBE(course, year) HAVING grouping__id > 0 +-- !query schema +struct +-- !query output +Java NULL +NULL 2012 +NULL 2013 +NULL NULL +dotNET NULL + + +-- !query +SELECT course, year, GROUPING(course), GROUPING(year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, udf(year) +-- !query schema +struct +-- !query output +Java 2012 0 0 +Java 2013 0 0 +dotNET 2012 0 0 +dotNET 2013 0 0 +Java NULL 0 1 +dotNET NULL 0 1 +NULL 2012 1 0 +NULL 2013 1 0 +NULL NULL 1 1 + + +-- !query +SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, udf(year) +-- !query schema +struct +-- !query output +Java 2012 0 +Java 2013 0 +dotNET 2012 0 +dotNET 2013 0 +Java NULL 1 +dotNET NULL 1 +NULL 2012 2 +NULL 2013 2 +NULL NULL 3 + + +-- !query +SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GROUPING(course) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", + "sqlState" : "42K0E" +} + + +-- !query +SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GROUPING_ID(course) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION", + "sqlState" : "42K0E" +} + + +-- !query +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, udf(course), year +-- !query schema +struct +-- !query output +Java 2012 +Java 2013 +dotNET 2012 +dotNET 2013 +Java NULL +dotNET NULL +NULL 2012 +NULL 2013 +NULL NULL + + +-- !query +SELECT udf(a + b) AS k1, udf(b) AS k2, SUM(a - b) FROM testData GROUP BY CUBE(k1, k2) +-- !query schema +struct +-- !query output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL 1 3 +NULL 2 0 +NULL NULL 3 + + +-- !query +SELECT udf(udf(a + b)) AS k, b, SUM(a - b) FROM testData GROUP BY ROLLUP(k, b) +-- !query schema +struct +-- !query output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL NULL 3 + + +-- !query +SELECT udf(a + b), udf(udf(b)) AS k, SUM(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k) +-- !query schema +struct +-- !query output +NULL 1 3 +NULL 2 0 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-group-by.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-group-by.sql.out new file mode 100644 index 000000000000..6a70c8b96841 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-group-by.sql.out @@ -0,0 +1,671 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(a), udf(COUNT(b)) FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 42, + "fragment" : "SELECT udf(a), udf(COUNT(b)) FROM testData" + } ] +} + + +-- !query +SELECT COUNT(udf(a)), udf(COUNT(b)) FROM testData +-- !query schema +struct +-- !query output +7 7 + + +-- !query +SELECT udf(a), COUNT(udf(b)) FROM testData GROUP BY a +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT udf(a), udf(COUNT(udf(b))) FROM testData GROUP BY b +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT COUNT(udf(a)), COUNT(udf(b)) FROM testData GROUP BY udf(a) +-- !query schema +struct +-- !query output +0 1 +2 2 +2 2 +3 2 + + +-- !query +SELECT 'foo', COUNT(udf(a)) FROM testData GROUP BY 1 +-- !query schema +struct +-- !query output +foo 7 + + +-- !query +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY udf(1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT 'foo', MAX(STRUCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1) +-- !query schema +struct> +-- !query output + + + +-- !query +SELECT udf(a + b), udf(COUNT(b)) FROM testData GROUP BY a + b +-- !query schema +struct +-- !query output +2 1 +3 2 +4 2 +5 1 +NULL 1 + + +-- !query +SELECT udf(a + 2), udf(COUNT(b)) FROM testData GROUP BY a + 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"a\"", + "expressionAnyValue" : "\"any_value(a)\"" + } +} + + +-- !query +SELECT udf(a + 1) + 1, udf(COUNT(b)) FROM testData GROUP BY udf(a + 1) +-- !query schema +struct<(udf((a + 1)) + 1):int,udf(count(b)):bigint> +-- !query output +3 2 +4 2 +5 2 +NULL 1 + + +-- !query +SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a)) +FROM testData +-- !query schema +struct +-- !query output +-0.2723801058145729 -1.5069204152249134 1 3 2.142857142857143 0.8095238095238094 0.8997354108424372 15 7 + + +-- !query +SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY udf(a) +-- !query schema +struct +-- !query output +1 1 + + +-- !query +SELECT udf(a) AS k, COUNT(udf(b)) FROM testData GROUP BY k +-- !query schema +struct +-- !query output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query +SELECT a AS k, udf(COUNT(b)) FROM testData GROUP BY k HAVING k > 1 +-- !query schema +struct +-- !query output +2 2 +3 2 + + +-- !query +SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "GROUP_BY_AGGREGATE", + "sqlState" : "42903", + "messageParameters" : { + "sqlExpr" : "CAST(udf(cast(count(b) as string)) AS BIGINT)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "udf(COUNT(b))" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY udf(a) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_AGGREGATION", + "sqlState" : "42803", + "messageParameters" : { + "expression" : "\"k\"", + "expressionAnyValue" : "\"any_value(k)\"" + } +} + + +-- !query +set spark.sql.groupByAliases=false +-- !query schema +struct +-- !query output +spark.sql.groupByAliases false + + +-- !query +SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`k`", + "proposal" : "`a`, `b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 58, + "stopIndex" : 58, + "fragment" : "k" + } ] +} + + +-- !query +SELECT udf(a), COUNT(udf(1)) FROM testData WHERE false GROUP BY udf(a) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(COUNT(1)) FROM testData WHERE false +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT 1 FROM (SELECT udf(COUNT(1)) FROM testData WHERE false) t +-- !query schema +struct<1:int> +-- !query output +1 + + +-- !query +SELECT 1 from ( + SELECT 1 AS z, + udf(MIN(a.x)) + FROM (select 1 as x) a + WHERE false +) b +where b.z != b.z +-- !query schema +struct<1:int> +-- !query output + + + +-- !query +SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) +-- !query schema +struct +-- !query output +1.0 1.0 3 + + +-- !query +SELECT udf(1) FROM range(10) HAVING true +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT udf(udf(1)) FROM range(10) HAVING MAX(id) > 0 +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT udf(id) FROM range(10) HAVING id > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_GROUP_BY", + "sqlState" : "42803", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 43, + "fragment" : "SELECT udf(id) FROM range(10) HAVING id > 0" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0 +-- !query schema +struct +-- !query output +NULL NULL NULL + + +-- !query +SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4 +-- !query schema +struct +-- !query output +NULL NULL NULL + + +-- !query +SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5 +-- !query schema +struct +-- !query output +false true true + + +-- !query +SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k) +-- !query schema +struct +-- !query output +1 false true true +2 true true true +3 false false false +4 NULL NULL NULL +5 false true true + + +-- !query +SELECT udf(k), every(v) FROM test_agg GROUP BY k HAVING every(v) = false +-- !query schema +struct +-- !query output +1 false +3 false +5 false + + +-- !query +SELECT udf(k), udf(every(v)) FROM test_agg GROUP BY udf(k) HAVING every(v) IS NULL +-- !query schema +struct +-- !query output +4 NULL + + +-- !query +SELECT udf(k), + udf(Every(v)) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY udf(k) +-- !query schema +struct +-- !query output +2 true + + +-- !query +SELECT udf(udf(k)), + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY udf(udf(k)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT every(udf(1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"udf(1)\"", + "inputType" : "\"INT\"", + "paramIndex" : "first", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"every(udf(1))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "every(udf(1))" + } ] +} + + +-- !query +SELECT some(udf(1S)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"udf(1)\"", + "inputType" : "\"SMALLINT\"", + "paramIndex" : "first", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"some(udf(1))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 20, + "fragment" : "some(udf(1S))" + } ] +} + + +-- !query +SELECT any(udf(1L)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"udf(1)\"", + "inputType" : "\"BIGINT\"", + "paramIndex" : "first", + "requiredType" : "\"BOOLEAN\"", + "sqlExpr" : "\"any(udf(1))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "any(udf(1L))" + } ] +} + + +-- !query +SELECT udf(every("true")) +-- !query schema +struct +-- !query output +true + + +-- !query +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true false +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true false + + +-- !query +SELECT k, udf(udf(v)), some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT udf(udf(k)), v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query schema +struct +-- !query output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query +SELECT udf(count(*)) FROM test_agg HAVING count(*) > 1L +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT k, udf(max(v)) FROM test_agg GROUP BY k HAVING max(v) = true +-- !query schema +struct +-- !query output +1 true +2 true +5 true + + +-- !query +SELECT * FROM (SELECT udf(COUNT(*)) AS cnt FROM test_agg) WHERE cnt > 1L +-- !query schema +struct +-- !query output +10 + + +-- !query +SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(count(1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 54, + "fragment" : "SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L" + } ] +} + + +-- !query +SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"((count(1) + 1) > 1)\"", + "expressionList" : "count(1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 59, + "fragment" : "SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L" + } ] +} + + +-- !query +SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INVALID_WHERE_CONDITION", + "sqlState" : "42903", + "messageParameters" : { + "condition" : "\"(((k = 1) OR (k = 2)) OR (((count(1) + 1) > 1) OR (max(k) > 1)))\"", + "expressionList" : "count(1), max(test_agg.k)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 91, + "fragment" : "SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-having.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-having.sql.out new file mode 100644 index 000000000000..147c0e24cd99 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-having.sql.out @@ -0,0 +1,46 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view hav as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", 5) + as hav(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(k) AS k, udf(sum(v)) FROM hav GROUP BY k HAVING udf(sum(v)) > 2 +-- !query schema +struct +-- !query output +one 6 +three 3 + + +-- !query +SELECT udf(count(udf(k))) FROM hav GROUP BY v + 1 HAVING v + 1 = udf(2) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT udf(MIN(t.v)) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(udf(COUNT(udf(1))) > 0) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT udf(a + b) FROM VALUES (1L, 2), (3L, 4) AS T(a, b) GROUP BY a + b HAVING a + b > udf(1) +-- !query schema +struct +-- !query output +3 +7 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-inline-table.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-inline-table.sql.out new file mode 100644 index 000000000000..3e84ec09c215 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-inline-table.sql.out @@ -0,0 +1,233 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select udf(col1), udf(col2) from values ("one", 1) +-- !query schema +struct +-- !query output +one 1 + + +-- !query +select udf(col1), udf(udf(col2)) from values ("one", 1) as data +-- !query schema +struct +-- !query output +one 1 + + +-- !query +select udf(a), b from values ("one", 1) as data(a, b) +-- !query schema +struct +-- !query output +one 1 + + +-- !query +select udf(a) from values 1, 2, 3 as data(a) +-- !query schema +struct +-- !query output +1 +2 +3 + + +-- !query +select udf(a), b from values ("one", 1), ("two", 2), ("three", null) as data(a, b) +-- !query schema +struct +-- !query output +one 1 +three NULL +two 2 + + +-- !query +select udf(a), b from values ("one", null), ("two", null) as data(a, b) +-- !query schema +struct +-- !query output +one NULL +two NULL + + +-- !query +select udf(a), b from values ("one", 1), ("two", 2L) as data(a, b) +-- !query schema +struct +-- !query output +one 1 +two 2 + + +-- !query +select udf(udf(a)), udf(b) from values ("one", 1 + 0), ("two", 1 + 3L) as data(a, b) +-- !query schema +struct +-- !query output +one 1 +two 4 + + +-- !query +select udf(a), b from values ("one", array(0, 1)), ("two", array(2, 3)) as data(a, b) +-- !query schema +struct> +-- !query output +one [0,1] +two [2,3] + + +-- !query +select udf(a), b from values ("one", 2.0), ("two", 3.0D) as data(a, b) +-- !query schema +struct +-- !query output +one 2.0 +two 3.0 + + +-- !query +select udf(a), b from values ("one", rand(5)), ("two", 3.0D) as data(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_INLINE_TABLE.CANNOT_EVALUATE_EXPRESSION_IN_INLINE_TABLE", + "sqlState" : "42000", + "messageParameters" : { + "expr" : "\"rand(5)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 38, + "stopIndex" : 44, + "fragment" : "rand(5)" + } ] +} + + +-- !query +select udf(a), udf(b) from values ("one", 2.0), ("two") as data(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INLINE_TABLE.NUM_COLUMNS_MISMATCH", + "sqlState" : "42000", + "messageParameters" : { + "actualNumCols" : "1", + "expectedNumCols" : "2", + "rowIndex" : "1" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 69, + "fragment" : "values (\"one\", 2.0), (\"two\") as data(a, b)" + } ] +} + + +-- !query +select udf(a), udf(b) from values ("one", array(0, 1)), ("two", struct(1, 2)) as data(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_INLINE_TABLE.INCOMPATIBLE_TYPES_IN_INLINE_TABLE", + "sqlState" : "42000", + "messageParameters" : { + "colName" : "`b`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 91, + "fragment" : "values (\"one\", array(0, 1)), (\"two\", struct(1, 2)) as data(a, b)" + } ] +} + + +-- !query +select udf(a), udf(b) from values ("one"), ("two") as data(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_INLINE_TABLE.NUM_COLUMNS_MISMATCH", + "sqlState" : "42000", + "messageParameters" : { + "actualNumCols" : "1", + "expectedNumCols" : "2", + "rowIndex" : "0" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 28, + "stopIndex" : 64, + "fragment" : "values (\"one\"), (\"two\") as data(a, b)" + } ] +} + + +-- !query +select udf(a), udf(b) from values ("one", random_not_exist_func(1)), ("two", 2) as data(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNRESOLVED_ROUTINE", + "sqlState" : "42883", + "messageParameters" : { + "routineName" : "`random_not_exist_func`", + "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 43, + "stopIndex" : 66, + "fragment" : "random_not_exist_func(1)" + } ] +} + + +-- !query +select udf(a), udf(b) from values ("one", count(1)), ("two", 2) as data(a, b) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_INLINE_TABLE.CANNOT_EVALUATE_EXPRESSION_IN_INLINE_TABLE", + "sqlState" : "42000", + "messageParameters" : { + "expr" : "\"count(1)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 43, + "stopIndex" : 50, + "fragment" : "count(1)" + } ] +} + + +-- !query +select udf(a), b from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991-12-06 01:00:00.0'), timestamp('1991-12-06 12:00:00.0'))) as data(a, b) +-- !query schema +struct> +-- !query output +1991-12-06 00:00:00 [1991-12-06 01:00:00,1991-12-06 12:00:00] diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-inner-join.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-inner-join.sql.out new file mode 100644 index 000000000000..2ac134fa6661 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-inner-join.sql.out @@ -0,0 +1,64 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW ta AS +SELECT udf(a) AS a, udf('a') AS tag FROM t1 +UNION ALL +SELECT udf(a) AS a, udf('b') AS tag FROM t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW tb AS +SELECT udf(a) AS a, udf('a') AS tag FROM t3 +UNION ALL +SELECT udf(a) AS a, udf('b') AS tag FROM t4 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag +-- !query schema +struct +-- !query output +1 a +1 a +1 b +1 b diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-intersect-all.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-intersect-all.sql.out new file mode 100644 index 000000000000..240469b0cab6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-intersect-all.sql.out @@ -0,0 +1,338 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (1, 3), + (2, 3), + (null, null), + (null, null) + AS tab1(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (2, 3), + (3, 4), + (null, null), + (null, null) + AS tab2(k, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT k, udf(v) FROM tab2 +-- !query schema +struct +-- !query output +1 2 +1 2 +2 3 +NULL NULL +NULL NULL + + +-- !query +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab1 WHERE udf(k) = 1 +-- !query schema +struct +-- !query output +1 2 +1 2 +1 3 +1 3 + + +-- !query +SELECT udf(k), udf(v) FROM tab1 WHERE k > udf(2) +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab2 WHERE udf(udf(k)) > 3 +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT CAST(udf(1) AS BIGINT), CAST(udf(2) AS BIGINT) +-- !query schema +struct +-- !query output +1 2 + + +-- !query +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT array(1), udf(2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "INCOMPATIBLE_COLUMN_TYPE", + "sqlState" : "42825", + "messageParameters" : { + "columnOrdinalNumber" : "first", + "dataType1" : "\"ARRAY\"", + "dataType2" : "\"INT\"", + "hint" : "", + "operator" : "INTERSECT ALL", + "tableOrdinalNumber" : "second" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 64, + "fragment" : "SELECT k, udf(v) FROM tab1\nINTERSECT ALL\nSELECT array(1), udf(2)" + } ] +} + + +-- !query +SELECT udf(k) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NUM_COLUMNS_MISMATCH", + "sqlState" : "42826", + "messageParameters" : { + "firstNumColumns" : "1", + "invalidNumColumns" : "2", + "invalidOrdinalNum" : "second", + "operator" : "INTERSECT ALL" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 69, + "fragment" : "SELECT udf(k) FROM tab1\nINTERSECT ALL\nSELECT udf(k), udf(v) FROM tab2" + } ] +} + + +-- !query +SELECT udf(k), v FROM tab2 +INTERSECT ALL +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2 +-- !query schema +struct +-- !query output +1 2 +1 2 +2 3 +NULL NULL +NULL NULL + + +-- !query +SELECT udf(k), v FROM tab1 +EXCEPT +SELECT k, udf(v) FROM tab2 +UNION ALL +SELECT k, udf(udf(v)) FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab2 +-- !query schema +struct +-- !query output +1 2 +1 2 +1 3 +2 3 +NULL NULL +NULL NULL + + +-- !query +SELECT udf(k), udf(v) FROM tab1 +EXCEPT +SELECT udf(k), v FROM tab2 +EXCEPT +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(udf(v)) FROM tab2 +-- !query schema +struct +-- !query output +1 3 + + +-- !query +( + ( + ( + SELECT udf(k), v FROM tab1 + EXCEPT + SELECT k, udf(v) FROM tab2 + ) + EXCEPT + SELECT udf(k), udf(v) FROM tab1 + ) + INTERSECT ALL + SELECT udf(k), udf(v) FROM tab2 +) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(udf(tab1.k)) = tab2.k) +INTERSECT ALL +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(tab1.k) = udf(udf(tab2.k))) +-- !query schema +struct +-- !query output +1 2 +1 2 +1 2 +1 2 +1 2 +1 2 +1 2 +1 2 +2 3 + + +-- !query +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(tab1.k) = udf(tab2.k)) +INTERSECT ALL +SELECT * +FROM (SELECT udf(tab2.v) AS k, + udf(tab1.k) AS v + FROM tab1 + JOIN tab2 + ON tab1.k = udf(tab2.k)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(v) FROM tab1 GROUP BY v +INTERSECT ALL +SELECT udf(udf(k)) FROM tab2 GROUP BY k +-- !query schema +struct +-- !query output +2 +3 +NULL + + +-- !query +SET spark.sql.legacy.setopsPrecedence.enabled= true +-- !query schema +struct +-- !query output +spark.sql.legacy.setopsPrecedence.enabled true + + +-- !query +SELECT udf(k), v FROM tab1 +EXCEPT +SELECT k, udf(v) FROM tab2 +UNION ALL +SELECT udf(k), udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(udf(k)), udf(v) FROM tab2 +-- !query schema +struct +-- !query output +1 2 +1 2 +2 3 +NULL NULL +NULL NULL + + +-- !query +SELECT k, udf(v) FROM tab1 +EXCEPT +SELECT udf(k), v FROM tab2 +UNION ALL +SELECT udf(k), udf(v) FROM tab1 +INTERSECT +SELECT udf(k), udf(udf(v)) FROM tab2 +-- !query schema +struct +-- !query output +1 2 +2 3 +NULL NULL + + +-- !query +SET spark.sql.legacy.setopsPrecedence.enabled = false +-- !query schema +struct +-- !query output +spark.sql.legacy.setopsPrecedence.enabled false + + +-- !query +DROP VIEW IF EXISTS tab1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS tab2 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-join-empty-relation.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-join-empty-relation.sql.out new file mode 100644 index 000000000000..5dbfb54b4b16 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-join-empty-relation.sql.out @@ -0,0 +1,191 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW empty_table as SELECT a FROM t2 WHERE false +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(t1.a), udf(empty_table.a) FROM t1 INNER JOIN empty_table ON (udf(t1.a) = udf(udf(empty_table.a))) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(t1.a), udf(udf(empty_table.a)) FROM t1 CROSS JOIN empty_table ON (udf(udf(t1.a)) = udf(empty_table.a)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(udf(t1.a)), empty_table.a FROM t1 LEFT OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) +-- !query schema +struct +-- !query output +1 NULL + + +-- !query +SELECT udf(t1.a), udf(empty_table.a) FROM t1 RIGHT OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(t1.a), empty_table.a FROM t1 FULL OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) +-- !query schema +struct +-- !query output +1 NULL + + +-- !query +SELECT udf(udf(t1.a)) FROM t1 LEFT SEMI JOIN empty_table ON (udf(t1.a) = udf(udf(empty_table.a))) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(t1.a) FROM t1 LEFT ANTI JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT udf(empty_table.a), udf(t1.a) FROM empty_table INNER JOIN t1 ON (udf(udf(empty_table.a)) = udf(t1.a)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(empty_table.a), udf(udf(t1.a)) FROM empty_table CROSS JOIN t1 ON (udf(empty_table.a) = udf(udf(t1.a))) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(udf(empty_table.a)), udf(t1.a) FROM empty_table LEFT OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT empty_table.a, udf(t1.a) FROM empty_table RIGHT OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) +-- !query schema +struct +-- !query output +NULL 1 + + +-- !query +SELECT empty_table.a, udf(udf(t1.a)) FROM empty_table FULL OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) +-- !query schema +struct +-- !query output +NULL 1 + + +-- !query +SELECT udf(udf(empty_table.a)) FROM empty_table LEFT SEMI JOIN t1 ON (udf(empty_table.a) = udf(udf(t1.a))) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT empty_table.a FROM empty_table LEFT ANTI JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(empty_table.a) FROM empty_table INNER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(udf(empty_table2.a))) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(udf(empty_table.a)) FROM empty_table CROSS JOIN empty_table AS empty_table2 ON (udf(udf(empty_table.a)) = udf(empty_table2.a)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(empty_table.a) FROM empty_table LEFT OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(udf(empty_table.a)) FROM empty_table RIGHT OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(udf(empty_table2.a))) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(empty_table.a) FROM empty_table FULL OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(udf(empty_table.a)) FROM empty_table LEFT SEMI JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT udf(empty_table.a) FROM empty_table LEFT ANTI JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)) +-- !query schema +struct +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-natural-join.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-natural-join.sql.out new file mode 100644 index 000000000000..ca3dae070e77 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-natural-join.sql.out @@ -0,0 +1,61 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM nt1 natural join nt2 where udf(k) = "one" +-- !query schema +struct +-- !query output +one 1 1 +one 1 5 + + +-- !query +SELECT * FROM nt1 natural left join nt2 where k <> udf("") order by v1, v2 +-- !query schema +struct +-- !query output +one 1 1 +one 1 5 +two 2 22 +three 3 NULL + + +-- !query +SELECT * FROM nt1 natural right join nt2 where udf(k) <> udf("") order by v1, v2 +-- !query schema +struct +-- !query output +one 1 1 +one 1 5 +two 2 22 + + +-- !query +SELECT udf(count(*)) FROM nt1 natural full outer join nt2 +-- !query schema +struct +-- !query output +4 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-outer-join.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-outer-join.sql.out new file mode 100644 index 000000000000..2771d144cde8 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-outer-join.sql.out @@ -0,0 +1,85 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(-234), (145), (367), (975), (298) +as t1(int_col1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES +(-769, -244), (-800, -409), (940, 86), (-507, 304), (-367, 158) +as t2(int_col0, int_col1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT + (udf(SUM(udf(COALESCE(t1.int_col1, t2.int_col0))))), + (udf(COALESCE(t1.int_col1, t2.int_col0)) * 2) +FROM t1 +RIGHT JOIN t2 + ON udf(t2.int_col0) = udf(t1.int_col1) +GROUP BY udf(GREATEST(COALESCE(udf(t2.int_col1), 109), COALESCE(t1.int_col1, udf(-449)))), + COALESCE(t1.int_col1, t2.int_col0) +HAVING (udf(SUM(COALESCE(udf(t1.int_col1), udf(t2.int_col0))))) + > (udf(COALESCE(t1.int_col1, t2.int_col0)) * 2) +-- !query schema +struct +-- !query output +-367 -734 +-507 -1014 +-769 -1538 +-800 -1600 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (97) as t1(int_col1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (0) as t2(int_col1) +-- !query schema +struct<> +-- !query output + + + +-- !query +set spark.sql.crossJoin.enabled = true +-- !query schema +struct +-- !query output +spark.sql.crossJoin.enabled true + + +-- !query +SELECT * +FROM ( +SELECT + udf(COALESCE(udf(t2.int_col1), udf(t1.int_col1))) AS int_col + FROM t1 + LEFT JOIN t2 ON false +) t where (udf(t.int_col)) is not null +-- !query schema +struct +-- !query output +97 + + +-- !query +set spark.sql.crossJoin.enabled = false +-- !query schema +struct +-- !query output +spark.sql.crossJoin.enabled false diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-pivot.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-pivot.sql.out new file mode 100644 index 000000000000..50376fb86a50 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-pivot.sql.out @@ -0,0 +1,531 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view courseSales as select * from values + ("dotNET", 2012, 10000), + ("Java", 2012, 20000), + ("dotNET", 2012, 5000), + ("dotNET", 2013, 48000), + ("Java", 2013, 30000) + as courseSales(course, year, earnings) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view years as select * from values + (2012, 1), + (2013, 2) + as years(y, s) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view yearsWithComplexTypes as select * from values + (2012, array(1, 1), map('1', 1), struct(1, 'a')), + (2013, array(2, 2), map('2', 2), struct(2, 'b')) + as yearsWithComplexTypes(y, a, m, s) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM ( + SELECT udf(year), course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct +-- !query output +2012 15000 20000 +2013 48000 30000 + + +-- !query +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (2012, 2013) +) +-- !query schema +struct +-- !query output +Java 20000 30000 +dotNET 15000 48000 + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)), udf(avg(earnings)) + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct +-- !query output +2012 15000 7500.0 20000 20000.0 +2013 48000 48000.0 30000 30000.0 + + +-- !query +SELECT * FROM ( + SELECT udf(course) as course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct +-- !query output +63000 50000 + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(udf(earnings))), udf(min(year)) + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct +-- !query output +63000 2012 50000 2012 + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, udf(s) as s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR s IN (1, 2) +) +-- !query schema +struct +-- !query output +Java 2012 20000 NULL +Java 2013 NULL 30000 +dotNET 2012 15000 NULL +dotNET 2013 NULL 48000 + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)), udf(min(s)) + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct +-- !query output +2012 15000 1 20000 1 +2013 48000 2 30000 2 + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings * s)) + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct +-- !query output +2012 15000 20000 +2013 96000 60000 + + +-- !query +SELECT 2012_s, 2013_s, 2012_a, 2013_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + udf(sum(e)) s, udf(avg(e)) a + FOR y IN (2012, 2013) +) +-- !query schema +struct<2012_s:bigint,2013_s:bigint,2012_a:double,2013_a:double,c:string> +-- !query output +15000 48000 7500.0 48000.0 dotNET +20000 30000 20000.0 30000.0 Java + + +-- !query +SELECT firstYear_s, secondYear_s, firstYear_a, secondYear_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + udf(sum(e)) s, udf(avg(e)) a + FOR y IN (2012 as firstYear, 2013 secondYear) +) +-- !query schema +struct +-- !query output +15000 48000 7500.0 48000.0 dotNET +20000 30000 20000.0 30000.0 Java + + +-- !query +SELECT * FROM courseSales +PIVOT ( + udf(abs(earnings)) + FOR year IN (2012, 2013) +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1006", + "messageParameters" : { + "sql" : "coursesales.earnings" + } +} + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)), year + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1006", + "messageParameters" : { + "sql" : "__auto_generated_subquery_name.year" + } +} + + +-- !query +SELECT * FROM ( + SELECT course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR year IN (2012, 2013) +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`year`", + "proposal" : "`course`, `earnings`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 62, + "stopIndex" : 118, + "fragment" : "PIVOT (\n udf(sum(earnings))\n FOR year IN (2012, 2013)\n)" + } ] +} + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(ceil(udf(sum(earnings)))), avg(earnings) + 1 as a1 + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct +-- !query output +2012 15000 7501.0 20000 20001.0 +2013 48000 48001.0 30000 30001.0 + + +-- !query +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(udf(avg(earnings))) + FOR course IN ('dotNET', 'Java') +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "NESTED_AGGREGATE_FUNCTION", + "sqlState" : "42607", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 86, + "stopIndex" : 98, + "fragment" : "avg(earnings)" + } ] +} + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, year) IN (('dotNET', 2012), ('Java', 2013)) +) +-- !query schema +struct +-- !query output +1 15000 NULL +2 NULL 30000 + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, s) IN (('dotNET', 2) as c1, ('Java', 1) as c2) +) +-- !query schema +struct +-- !query output +2012 NULL 20000 +2013 48000 NULL + + +-- !query +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, year) IN ('dotNET', 'Java') +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PIVOT_VALUE_DATA_TYPE_MISMATCH", + "sqlState" : "42K09", + "messageParameters" : { + "pivotType" : "struct", + "value" : "dotNET", + "valueType" : "string" + } +} + + +-- !query +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (s, 2013) +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`s`", + "proposal" : "`year`, `course`, `earnings`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 71, + "stopIndex" : 71, + "fragment" : "s" + } ] +} + + +-- !query +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (course, 2013) +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NON_LITERAL_PIVOT_VALUES", + "sqlState" : "42K08", + "messageParameters" : { + "expression" : "\"course\"" + } +} + + +-- !query +SELECT * FROM ( + SELECT earnings, year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR a IN (array(1, 1), array(2, 2)) +) +-- !query schema +struct +-- !query output +2012 35000 NULL +2013 NULL 78000 + + +-- !query +SELECT * FROM ( + SELECT course, earnings, udf(year) as year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2))) +) +-- !query schema +struct +-- !query output +2012 15000 NULL +2013 NULL 30000 + + +-- !query +SELECT * FROM ( + SELECT earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR s IN ((1, 'a'), (2, 'b')) +) +-- !query schema +struct +-- !query output +2012 35000 NULL +2013 NULL 78000 + + +-- !query +SELECT * FROM ( + SELECT course, earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b'))) +) +-- !query schema +struct +-- !query output +2012 15000 NULL +2013 NULL 30000 + + +-- !query +SELECT * FROM ( + SELECT earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR m IN (map('1', 1), map('2', 2)) +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPARABLE_PIVOT_COLUMN", + "sqlState" : "42818", + "messageParameters" : { + "columnName" : "`m`" + } +} + + +-- !query +SELECT * FROM ( + SELECT course, earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, m) IN (('dotNET', map('1', 1)), ('Java', map('2', 2))) +) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPARABLE_PIVOT_COLUMN", + "sqlState" : "42818", + "messageParameters" : { + "columnName" : "`named_struct('course', __auto_generated_subquery_name`.`course, 'm', __auto_generated_subquery_name`.`m)`" + } +} + + +-- !query +SELECT * FROM ( + SELECT course, earnings, udf("a") as a, udf("z") as z, udf("b") as b, udf("y") as y, + udf("c") as c, udf("x") as x, udf("d") as d, udf("w") as w + FROM courseSales +) +PIVOT ( + udf(sum(Earnings)) + FOR Course IN ('dotNET', 'Java') +) +-- !query schema +struct +-- !query output +a z b y c x d w 63000 50000 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-special-values.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-special-values.sql.out new file mode 100644 index 000000000000..73c0a0642dad --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-special-values.sql.out @@ -0,0 +1,59 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT udf(x) FROM (VALUES (1), (2), (NULL)) v(x) +-- !query schema +struct +-- !query output +1 +2 +NULL + + +-- !query +SELECT udf(x) FROM (VALUES ('A'), ('B'), (NULL)) v(x) +-- !query schema +struct +-- !query output +A +B +NULL + + +-- !query +SELECT udf(x) FROM (VALUES ('NaN'), ('1'), ('2')) v(x) +-- !query schema +struct +-- !query output +1 +2 +NaN + + +-- !query +SELECT udf(x) FROM (VALUES ('Infinity'), ('1'), ('2')) v(x) +-- !query schema +struct +-- !query output +1 +2 +Infinity + + +-- !query +SELECT udf(x) FROM (VALUES ('-Infinity'), ('1'), ('2')) v(x) +-- !query schema +struct +-- !query output +-Infinity +1 +2 + + +-- !query +SELECT udf(x) FROM (VALUES 0.00000001, 0.00000002, 0.00000003) v(x) +-- !query schema +struct +-- !query output +0.00000001 +0.00000002 +0.00000003 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-udaf.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-udaf.sql.out new file mode 100644 index 000000000000..ad8b42d9a5db --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-udaf.sql.out @@ -0,0 +1,97 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(1), (2), (3), (4) +as t1(int_col1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION myDoubleAvg AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT default.myDoubleAvg(udf(int_col1)) as my_avg, udf(default.myDoubleAvg(udf(int_col1))) as my_avg2, udf(default.myDoubleAvg(int_col1)) as my_avg3 from t1 +-- !query schema +struct +-- !query output +102.5 102.5 102.5 + + +-- !query +SELECT default.myDoubleAvg(udf(int_col1), udf(3)) as my_avg from t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`spark_catalog`.`default`.`mydoubleavg`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 49, + "fragment" : "default.myDoubleAvg(udf(int_col1), udf(3))" + } ] +} + + +-- !query +CREATE FUNCTION udaf1 AS 'test.non.existent.udaf' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT default.udaf1(udf(int_col1)) as udaf1, udf(default.udaf1(udf(int_col1))) as udaf2, udf(default.udaf1(int_col1)) as udaf3 from t1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CANNOT_LOAD_FUNCTION_CLASS", + "sqlState" : "46103", + "messageParameters" : { + "className" : "test.non.existent.udaf", + "functionName" : "`spark_catalog`.`default`.`udaf1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 35, + "fragment" : "default.udaf1(udf(int_col1))" + } ] +} + + +-- !query +DROP FUNCTION myDoubleAvg +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP FUNCTION udaf1 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-union.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-union.sql.out new file mode 100644 index 000000000000..6aa2148fa1e6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-union.sql.out @@ -0,0 +1,207 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS VALUES (1, 'a'), (2, 'b') tbl(c1, c2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t2 AS VALUES (1.0, 1), (2.0, 4) tbl(c1, c2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(c1) as c1, udf(c2) as c2 +FROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 + UNION ALL + SELECT udf(c1) as c1, udf(c2) as c2 FROM t1) +-- !query schema +struct +-- !query output +1 a +1 a +2 b +2 b + + +-- !query +SELECT udf(c1) as c1, udf(c2) as c2 +FROM (SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 WHERE c2 = 'a' + UNION ALL + SELECT udf(c1) as c1, udf(c2) as c2 FROM t2 + UNION ALL + SELECT udf(c1) as c1, udf(c2) as c2 FROM t2) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 45, + "stopIndex" : 172, + "fragment" : "SELECT udf(c1) as c1, udf(c2) as c2 FROM t1 WHERE c2 = 'a'\n UNION ALL\n SELECT udf(c1) as c1, udf(c2) as c2 FROM t2" + } ] +} + + +-- !query +SELECT udf(udf(a)) as a +FROM (SELECT udf(0) a, udf(0) b + UNION ALL + SELECT udf(SUM(1)) a, udf(CAST(0 AS BIGINT)) b + UNION ALL SELECT udf(0) a, udf(0) b) T +-- !query schema +struct +-- !query output +0 +0 +1 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW p1 AS VALUES 1 T(col) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW p2 AS VALUES 1 T(col) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW p3 AS VALUES 1 T(col) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(1) AS x, + udf(col) as col +FROM (SELECT udf(col) AS col + FROM (SELECT udf(p1.col) AS col + FROM p1 CROSS JOIN p2 + UNION ALL + SELECT udf(col) + FROM p3) T1) T2 +-- !query schema +struct +-- !query output +1 1 +1 1 + + +-- !query +SELECT map(1, 2), udf('str') as str +UNION ALL +SELECT map(1, 2, 3, NULL), udf(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'str'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 79, + "fragment" : "SELECT map(1, 2), udf('str') as str\nUNION ALL\nSELECT map(1, 2, 3, NULL), udf(1)" + } ] +} + + +-- !query +SELECT array(1, 2), udf('str') as str +UNION ALL +SELECT array(1, 2, 3, NULL), udf(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'str'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 83, + "fragment" : "SELECT array(1, 2), udf('str') as str\nUNION ALL\nSELECT array(1, 2, 3, NULL), udf(1)" + } ] +} + + +-- !query +DROP VIEW IF EXISTS t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS p1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS p2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS p3 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-window.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-window.sql.out new file mode 100644 index 000000000000..40e24e7b4e87 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udf/udf-window.sql.out @@ -0,0 +1,489 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), +(null, null, null, null, null, null), +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) +AS testData(val, val_long, val_double, val_date, val_timestamp, cate) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT udf(val), cate, count(val) OVER(PARTITION BY cate ORDER BY udf(val) ROWS CURRENT ROW) FROM testData +ORDER BY cate, udf(val) +-- !query schema +struct +-- !query output +NULL NULL 0 +3 NULL 1 +NULL a 0 +1 a 1 +1 a 1 +2 a 1 +1 b 1 +2 b 1 +3 b 1 + + +-- !query +SELECT udf(val), cate, sum(val) OVER(PARTITION BY cate ORDER BY udf(val) +ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) +-- !query schema +struct +-- !query output +NULL NULL 3 +3 NULL 3 +NULL a 1 +1 a 2 +1 a 4 +2 a 4 +1 b 3 +2 b 6 +3 b 6 + + +-- !query +SELECT val_long, udf(cate), sum(val_long) OVER(PARTITION BY cate ORDER BY udf(val_long) +ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY udf(cate), val_long +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "expectedType" : "\"INT\"", + "exprType" : "\"BIGINT\"", + "location" : "upper", + "sqlExpr" : "\"ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 47, + "stopIndex" : 138, + "fragment" : "(PARTITION BY cate ORDER BY udf(val_long)\nROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING)" + } ] +} + + +-- !query +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY val RANGE 1 PRECEDING) FROM testData +ORDER BY cate, udf(val) +-- !query schema +struct +-- !query output +NULL NULL 0 +3 NULL 1 +NULL a 0 +1 a 2 +1 a 2 +2 a 3 +1 b 1 +2 b 2 +3 b 2 + + +-- !query +SELECT val, udf(cate), sum(val) OVER(PARTITION BY udf(cate) ORDER BY val +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY udf(cate), val +-- !query schema +struct +-- !query output +NULL NULL NULL +3 NULL 3 +NULL a NULL +1 a 4 +1 a 4 +2 a 2 +1 b 3 +2 b 5 +3 b 3 + + +-- !query +SELECT val_long, udf(cate), sum(val_long) OVER(PARTITION BY udf(cate) ORDER BY val_long +RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY udf(cate), val_long +-- !query schema +struct +-- !query output +NULL NULL NULL +1 NULL 1 +1 a 4 +1 a 4 +2 a 2147483652 +2147483650 a 2147483650 +NULL b NULL +3 b 2147483653 +2147483650 b 2147483650 + + +-- !query +SELECT val_double, udf(cate), sum(val_double) OVER(PARTITION BY udf(cate) ORDER BY val_double +RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING) FROM testData ORDER BY udf(cate), val_double +-- !query schema +struct +-- !query output +NULL NULL NULL +1.0 NULL 1.0 +1.0 a 4.5 +1.0 a 4.5 +2.5 a 2.5 +100.001 a 100.001 +1.0 b 4.3 +3.3 b 3.3 +100.001 b 100.001 + + +-- !query +SELECT val_date, udf(cate), max(val_date) OVER(PARTITION BY udf(cate) ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING) FROM testData ORDER BY udf(cate), val_date +-- !query schema +struct +-- !query output +NULL NULL NULL +2017-08-01 NULL 2017-08-01 +2017-08-01 a 2017-08-02 +2017-08-01 a 2017-08-02 +2017-08-02 a 2017-08-02 +2020-12-31 a 2020-12-31 +2017-08-01 b 2017-08-03 +2017-08-03 b 2017-08-03 +2020-12-31 b 2020-12-31 + + +-- !query +SELECT val_timestamp, udf(cate), avg(val_timestamp) OVER(PARTITION BY udf(cate) ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData +ORDER BY udf(cate), val_timestamp +-- !query schema +struct +-- !query output +NULL NULL NULL +2017-07-31 17:00:00 NULL 1.5015456E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-08-05 23:13:20 a 1.502E9 +2020-12-30 16:00:00 a 1.6093728E9 +2017-07-31 17:00:00 b 1.5022728E9 +2017-08-17 13:00:00 b 1.503E9 +2020-12-30 16:00:00 b 1.6093728E9 + + +-- !query +SELECT val, udf(cate), sum(val) OVER(PARTITION BY cate ORDER BY val DESC +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL NULL +3 NULL 3 +NULL a NULL +1 a 2 +1 a 2 +2 a 4 +1 b 1 +2 b 3 +3 b 5 + + +-- !query +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) +ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_INVALID_BOUND", + "sqlState" : "42K09", + "messageParameters" : { + "lower" : "\"UNBOUNDED FOLLOWING\"", + "sqlExpr" : "\"ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING\"", + "upper" : "\"1\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 111, + "fragment" : "(PARTITION BY udf(cate)\nROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_WITHOUT_ORDER", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"(PARTITION BY udf(cate) RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 104, + "fragment" : "(PARTITION BY udf(cate)\nRANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY udf(val), cate +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_MULTI_ORDER", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpec" : "cast(udf(cast(val#x as string)) as int) ASC NULLS FIRST,cate#x ASC NULLS FIRST", + "sqlExpr" : "\"(PARTITION BY udf(cate) ORDER BY udf(val) ASC NULLS FIRST, cate ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 128, + "fragment" : "(PARTITION BY udf(cate) ORDER BY udf(val), cate\nRANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY current_timestamp +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, udf(val) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpecType" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(PARTITION BY udf(cate) ORDER BY current_timestamp() ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"", + "valueBoundaryType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 131, + "fragment" : "(PARTITION BY udf(cate) ORDER BY current_timestamp\nRANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY val +RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY udf(cate), val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_WRONG_COMPARISON", + "sqlState" : "42K09", + "messageParameters" : { + "comparison" : "less than or equal", + "sqlExpr" : "\"RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 39, + "stopIndex" : 117, + "fragment" : "(PARTITION BY udf(cate) ORDER BY val\nRANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING)" + } ] +} + + +-- !query +SELECT udf(val), cate, count(val) OVER(PARTITION BY udf(cate) ORDER BY udf(val) +RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val(val) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Frame bound value must be a literal." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 111, + "stopIndex" : 132, + "fragment" : "current_date PRECEDING" + } ] +} + + +-- !query +SELECT udf(val), cate, +max(udf(val)) OVER w AS max, +min(udf(val)) OVER w AS min, +min(udf(val)) OVER w AS min, +count(udf(val)) OVER w AS count, +sum(udf(val)) OVER w AS sum, +avg(udf(val)) OVER w AS avg, +stddev(udf(val)) OVER w AS stddev, +first_value(udf(val)) OVER w AS first_value, +first_value(udf(val), true) OVER w AS first_value_ignore_null, +first_value(udf(val), false) OVER w AS first_value_contain_null, +any_value(udf(val)) OVER w AS any_value, +any_value(udf(val), true) OVER w AS any_value_ignore_null, +any_value(udf(val), false) OVER w AS any_value_contain_null, +last_value(udf(val)) OVER w AS last_value, +last_value(udf(val), true) OVER w AS last_value_ignore_null, +last_value(udf(val), false) OVER w AS last_value_contain_null, +rank() OVER w AS rank, +dense_rank() OVER w AS dense_rank, +cume_dist() OVER w AS cume_dist, +percent_rank() OVER w AS percent_rank, +ntile(2) OVER w AS ntile, +row_number() OVER w AS row_number, +var_pop(udf(val)) OVER w AS var_pop, +var_samp(udf(val)) OVER w AS var_samp, +approx_count_distinct(udf(val)) OVER w AS approx_count_distinct, +covar_pop(udf(val), udf(val_long)) OVER w AS covar_pop, +corr(udf(val), udf(val_long)) OVER w AS corr, +stddev_samp(udf(val)) OVER w AS stddev_samp, +stddev_pop(udf(val)) OVER w AS stddev_pop, +collect_list(udf(val)) OVER w AS collect_list, +collect_set(udf(val)) OVER w AS collect_set, +skewness(udf(val_double)) OVER w AS skewness, +kurtosis(udf(val_double)) OVER w AS kurtosis +FROM testData +WINDOW w AS (PARTITION BY udf(cate) ORDER BY udf(val)) +ORDER BY cate, udf(val) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1126, + "stopIndex" : 1161, + "fragment" : "corr(udf(val), udf(val_long)) OVER w" + } ] +} + + +-- !query +SELECT udf(val), cate, avg(null) OVER(PARTITION BY cate ORDER BY val) FROM testData ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL NULL +3 NULL NULL +NULL a NULL +1 a NULL +1 a NULL +2 a NULL +1 b NULL +2 b NULL +3 b NULL + + +-- !query +SELECT udf(val), cate, row_number() OVER(PARTITION BY cate) FROM testData ORDER BY cate, udf(val) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1037", + "messageParameters" : { + "wf" : "row_number()" + } +} + + +-- !query +SELECT udf(val), cate, sum(val) OVER(), avg(val) OVER() FROM testData ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL 13 1.8571428571428572 +3 NULL 13 1.8571428571428572 +NULL a 13 1.8571428571428572 +1 a 13 1.8571428571428572 +1 a 13 1.8571428571428572 +2 a 13 1.8571428571428572 +1 b 13 1.8571428571428572 +2 b 13 1.8571428571428572 +3 b 13 1.8571428571428572 + + +-- !query +SELECT udf(val), cate, +first_value(false) OVER w AS first_value, +first_value(true, true) OVER w AS first_value_ignore_null, +first_value(false, false) OVER w AS first_value_contain_null, +any_value(false) OVER w AS any_value, +any_value(true, true) OVER w AS any_value_ignore_null, +any_value(false, false) OVER w AS any_value_contain_null, +last_value(false) OVER w AS last_value, +last_value(true, true) OVER w AS last_value_ignore_null, +last_value(false, false) OVER w AS last_value_contain_null +FROM testData +WINDOW w AS () +ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL false true false false true false false true false +3 NULL false true false false true false false true false +NULL a false true false false true false false true false +1 a false true false false true false false true false +1 a false true false false true false false true false +2 a false true false false true false false true false +1 b false true false false true false false true false +2 b false true false false true false false true false +3 b false true false false true false false true false + + +-- !query +SELECT udf(cate), sum(val) OVER (w) +FROM testData +WHERE val is not null +WINDOW w AS (PARTITION BY cate ORDER BY val) +-- !query schema +struct +-- !query output +NULL 3 +a 2 +a 2 +a 4 +b 1 +b 3 +b 6 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udtf/udtf.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udtf/udtf.sql.out new file mode 100644 index 000000000000..b20516151146 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/udtf/udtf.sql.out @@ -0,0 +1,1134 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +DROP VIEW IF EXISTS t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS VALUES (0, 1), (1, 2) t(c1, c2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t2 AS VALUES (0, 1), (1, 2), (1, 3) t(partition_col, input) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM udtf(1, 2) +-- !query schema +struct +-- !query output +1 -1 +2 1 + + +-- !query +SELECT * FROM udtf(-1, 0) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM udtf(0, -1) +-- !query schema +struct +-- !query output + + + +-- !query +SELECT * FROM udtf(0, 0) +-- !query schema +struct +-- !query output +0 0 + + +-- !query +SELECT a, b FROM udtf(1, 2) t(a, b) +-- !query schema +struct +-- !query output +1 -1 +2 1 + + +-- !query +SELECT * FROM t1, LATERAL udtf(c1, c2) +-- !query schema +struct +-- !query output +1 2 1 -1 +1 2 2 1 + + +-- !query +SELECT * FROM t1 LEFT JOIN LATERAL udtf(c1, c2) +-- !query schema +struct +-- !query output +1 2 1 -1 +1 2 2 1 + + +-- !query +SELECT * FROM udtf(1, 2) t(c1, c2), LATERAL udtf(c1, c2) +-- !query schema +struct +-- !query output +2 1 1 -1 +2 1 2 1 + + +-- !query +SELECT * FROM udtf(cast(rand(0) AS int) + 1, 1) +-- !query schema +struct +-- !query output +1 0 +1 0 + + +-- !query +SELECT * FROM UDTFCountSumLast(TABLE(t2) WITH SINGLE PARTITION) +-- !query schema +struct +-- !query output +3 6 3 + + +-- !query +SELECT * FROM UDTFCountSumLast(TABLE(t2) PARTITION BY partition_col ORDER BY input) +-- !query schema +struct +-- !query output +1 1 1 +2 5 3 + + +-- !query +SELECT * FROM UDTFCountSumLast(TABLE(t2) PARTITION BY partition_col ORDER BY input DESC) +-- !query schema +struct +-- !query output +1 1 1 +2 5 2 + + +-- !query +SELECT * FROM + VALUES (0), (1) AS t(col) + JOIN LATERAL + UDTFCountSumLast(TABLE(t2) PARTITION BY partition_col ORDER BY input DESC) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.NON_DETERMINISTIC_LATERAL_SUBQUERIES", + "sqlState" : "0A000", + "messageParameters" : { + "treeNode" : "LateralJoin lateral-subquery#x [], Inner\n: +- Project [count#x, total#x, last#x]\n: +- LateralJoin lateral-subquery#x [c#x], Inner\n: : +- SubqueryAlias __auto_generated_subquery_name_1\n: : +- Generate UDTFCountSumLast(outer(c#x))#x, false, [count#x, total#x, last#x]\n: : +- OneRowRelation\n: +- SubqueryAlias __auto_generated_subquery_name_0\n: +- Project [named_struct(partition_col, partition_col#x, input, input#x, partition_by_0, partition_by_0#x) AS c#x]\n: +- Sort [partition_by_0#x ASC NULLS FIRST, input#x DESC NULLS LAST], false\n: +- RepartitionByExpression [partition_by_0#x]\n: +- Project [partition_col#x, input#x, partition_col#x AS partition_by_0#x]\n: +- SubqueryAlias t2\n: +- View (`t2`, [partition_col#x, input#x])\n: +- Project [cast(partition_col#x as int) AS partition_col#x, cast(input#x as int) AS input#x]\n: +- SubqueryAlias t\n: +- LocalRelation [partition_col#x, input#x]\n+- SubqueryAlias t\n +- LocalRelation [col#x]\n" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 49, + "stopIndex" : 139, + "fragment" : "JOIN LATERAL\n UDTFCountSumLast(TABLE(t2) PARTITION BY partition_col ORDER BY input DESC)" + } ] +} + + +-- !query +SELECT * FROM UDTFWithSinglePartition(0, TABLE(t2)) +-- !query schema +struct +-- !query output +3 6 3 + + +-- !query +SELECT * FROM UDTFWithSinglePartition(1, TABLE(t2)) +-- !query schema +struct +-- !query output +3 6 3 + + +-- !query +SELECT * FROM UDTFWithSinglePartition(0, TABLE(t2) WITH SINGLE PARTITION) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_REQUIRED_METADATA_INCOMPATIBLE_WITH_CALL", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "UDTFWithSinglePartition", + "invalidFunctionCallProperty" : "specified the WITH SINGLE PARTITION or PARTITION BY clause; please remove these clauses and retry the query again.", + "requestedMetadata" : "specified its own required partitioning of the input table" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 73, + "fragment" : "UDTFWithSinglePartition(0, TABLE(t2) WITH SINGLE PARTITION)" + } ] +} + + +-- !query +SELECT * FROM UDTFWithSinglePartition(0, TABLE(t2) PARTITION BY partition_col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_REQUIRED_METADATA_INCOMPATIBLE_WITH_CALL", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "UDTFWithSinglePartition", + "invalidFunctionCallProperty" : "specified the WITH SINGLE PARTITION or PARTITION BY clause; please remove these clauses and retry the query again.", + "requestedMetadata" : "specified its own required partitioning of the input table" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 78, + "fragment" : "UDTFWithSinglePartition(0, TABLE(t2) PARTITION BY partition_col)" + } ] +} + + +-- !query +SELECT * FROM + VALUES (0), (1) AS t(col) + JOIN LATERAL + UDTFWithSinglePartition(0, TABLE(t2) PARTITION BY partition_col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_REQUIRED_METADATA_INCOMPATIBLE_WITH_CALL", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "UDTFWithSinglePartition", + "invalidFunctionCallProperty" : "specified the WITH SINGLE PARTITION or PARTITION BY clause; please remove these clauses and retry the query again.", + "requestedMetadata" : "specified its own required partitioning of the input table" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 66, + "stopIndex" : 129, + "fragment" : "UDTFWithSinglePartition(0, TABLE(t2) PARTITION BY partition_col)" + } ] +} + + +-- !query +SELECT * FROM UDTFPartitionByOrderBy(TABLE(t2)) +-- !query schema +struct +-- !query output +0 1 1 1 +1 2 5 3 + + +-- !query +SELECT * FROM UDTFPartitionByOrderBy(TABLE(t2) WITH SINGLE PARTITION) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_REQUIRED_METADATA_INCOMPATIBLE_WITH_CALL", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "UDTFPartitionByOrderBy", + "invalidFunctionCallProperty" : "specified the WITH SINGLE PARTITION or PARTITION BY clause; please remove these clauses and retry the query again.", + "requestedMetadata" : "specified its own required partitioning of the input table" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 69, + "fragment" : "UDTFPartitionByOrderBy(TABLE(t2) WITH SINGLE PARTITION)" + } ] +} + + +-- !query +SELECT * FROM UDTFPartitionByOrderBy(TABLE(t2) PARTITION BY partition_col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_REQUIRED_METADATA_INCOMPATIBLE_WITH_CALL", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "UDTFPartitionByOrderBy", + "invalidFunctionCallProperty" : "specified the WITH SINGLE PARTITION or PARTITION BY clause; please remove these clauses and retry the query again.", + "requestedMetadata" : "specified its own required partitioning of the input table" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 74, + "fragment" : "UDTFPartitionByOrderBy(TABLE(t2) PARTITION BY partition_col)" + } ] +} + + +-- !query +SELECT * FROM + VALUES (0), (1) AS t(col) + JOIN LATERAL + UDTFPartitionByOrderBy(TABLE(t2) PARTITION BY partition_col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_REQUIRED_METADATA_INCOMPATIBLE_WITH_CALL", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "UDTFPartitionByOrderBy", + "invalidFunctionCallProperty" : "specified the WITH SINGLE PARTITION or PARTITION BY clause; please remove these clauses and retry the query again.", + "requestedMetadata" : "specified its own required partitioning of the input table" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 66, + "stopIndex" : 125, + "fragment" : "UDTFPartitionByOrderBy(TABLE(t2) PARTITION BY partition_col)" + } ] +} + + +-- !query +SELECT * FROM UDTFPartitionByOrderByComplexExpr(TABLE(t2)) +-- !query schema +struct +-- !query output +0 1 1 1 +1 2 5 3 + + +-- !query +SELECT * FROM UDTFPartitionByOrderBySelectExpr(TABLE(t2)) +-- !query schema +struct +-- !query output +0 1 1 1 +1 2 5 3 + + +-- !query +SELECT * FROM UDTFPartitionByOrderBySelectComplexExpr(TABLE(t2)) +-- !query schema +struct +-- !query output +0 1 2 2 +1 2 7 4 + + +-- !query +SELECT * FROM UDTFPartitionByOrderBySelectExprOnlyPartitionColumn(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.api.python.PythonException +ValueError: 'input' is not in list + +During handling of the above exception, another exception occurred: + +pyspark.errors.exceptions.base.PySparkValueError: input + + +-- !query +SELECT * FROM UDTFInvalidSelectExprParseError(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`unparsable`", + "proposal" : "`t2`.`input`, `partition_by_0`, `t2`.`partition_col`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 10, + "fragment" : "unparsable" + } ] +} + + +-- !query +SELECT * FROM UDTFInvalidSelectExprStringValue(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_FAILED_TO_ANALYZE_IN_PYTHON", + "sqlState" : "38000", + "messageParameters" : { + "msg" : "Failed to evaluate the user-defined table function 'UDTFInvalidSelectExprStringValue' because the static 'analyze' method returned an 'AnalyzeResult' object with the 'select' field set to a value besides a list or tuple of 'SelectedColumn' objects. Please update the table function and then try the query again." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 57, + "fragment" : "UDTFInvalidSelectExprStringValue(TABLE(t2))" + } ] +} + + +-- !query +SELECT * FROM UDTFInvalidComplexSelectExprMissingAlias(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UDTF_INVALID_REQUESTED_SELECTED_EXPRESSION_FROM_ANALYZE_METHOD_REQUIRES_ALIAS", + "sqlState" : "42802", + "messageParameters" : { + "expression" : "(input + 1)" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 65, + "fragment" : "UDTFInvalidComplexSelectExprMissingAlias(TABLE(t2))" + } ] +} + + +-- !query +SELECT * FROM UDTFInvalidOrderByAscKeyword(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UDTF_INVALID_ALIAS_IN_REQUESTED_ORDERING_STRING_FROM_ANALYZE_METHOD", + "sqlState" : "42802", + "messageParameters" : { + "aliasName" : "ASC" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 53, + "fragment" : "UDTFInvalidOrderByAscKeyword(TABLE(t2))" + } ] +} + + +-- !query +SELECT * FROM UDTFInvalidOrderByStringList(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_FAILED_TO_ANALYZE_IN_PYTHON", + "sqlState" : "38000", + "messageParameters" : { + "msg" : "Failed to evaluate the user-defined table function 'UDTFInvalidOrderByStringList' because the static 'analyze' method returned an 'AnalyzeResult' object with the 'orderBy' field set to a value besides a list or tuple of 'OrderingColumn' objects. Please update the table function and then try the query again." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 53, + "fragment" : "UDTFInvalidOrderByStringList(TABLE(t2))" + } ] +} + + +-- !query +SELECT * FROM UDTFInvalidPartitionByAndWithSinglePartition(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_REQUIRED_METADATA_INVALID", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "UDTFInvalidPartitionByAndWithSinglePartition", + "reason" : "the 'with_single_partition' field cannot be assigned to true if the 'partition_by' list is non-empty" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 69, + "fragment" : "UDTFInvalidPartitionByAndWithSinglePartition(TABLE(t2))" + } ] +} + + +-- !query +SELECT * FROM UDTFInvalidPartitionByAndWithSinglePartition(TABLE(t2) WITH SINGLE PARTITION) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_REQUIRED_METADATA_INVALID", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "UDTFInvalidPartitionByAndWithSinglePartition", + "reason" : "the 'with_single_partition' field cannot be assigned to true if the 'partition_by' list is non-empty" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 91, + "fragment" : "UDTFInvalidPartitionByAndWithSinglePartition(TABLE(t2) WITH SINGLE PARTITION)" + } ] +} + + +-- !query +SELECT * FROM UDTFInvalidPartitionByAndWithSinglePartition(TABLE(t2) PARTITION BY partition_col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_REQUIRED_METADATA_INVALID", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "UDTFInvalidPartitionByAndWithSinglePartition", + "reason" : "the 'with_single_partition' field cannot be assigned to true if the 'partition_by' list is non-empty" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 96, + "fragment" : "UDTFInvalidPartitionByAndWithSinglePartition(TABLE(t2) PARTITION BY partition_col)" + } ] +} + + +-- !query +SELECT * FROM + VALUES (0), (1) AS t(col) + JOIN LATERAL + UDTFInvalidPartitionByAndWithSinglePartition(TABLE(t2) PARTITION BY partition_col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_REQUIRED_METADATA_INVALID", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "UDTFInvalidPartitionByAndWithSinglePartition", + "reason" : "the 'with_single_partition' field cannot be assigned to true if the 'partition_by' list is non-empty" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 66, + "stopIndex" : 147, + "fragment" : "UDTFInvalidPartitionByAndWithSinglePartition(TABLE(t2) PARTITION BY partition_col)" + } ] +} + + +-- !query +SELECT * FROM UDTFInvalidOrderByWithoutPartitionBy(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_REQUIRED_METADATA_INVALID", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "UDTFInvalidOrderByWithoutPartitionBy", + "reason" : "the 'order_by' field cannot be non-empty unless the 'with_single_partition' field is set to true or the 'partition_by' list is non-empty" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 61, + "fragment" : "UDTFInvalidOrderByWithoutPartitionBy(TABLE(t2))" + } ] +} + + +-- !query +SELECT * FROM UDTFInvalidOrderByWithoutPartitionBy(TABLE(t2) WITH SINGLE PARTITION) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_REQUIRED_METADATA_INVALID", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "UDTFInvalidOrderByWithoutPartitionBy", + "reason" : "the 'order_by' field cannot be non-empty unless the 'with_single_partition' field is set to true or the 'partition_by' list is non-empty" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 83, + "fragment" : "UDTFInvalidOrderByWithoutPartitionBy(TABLE(t2) WITH SINGLE PARTITION)" + } ] +} + + +-- !query +SELECT * FROM UDTFInvalidOrderByWithoutPartitionBy(TABLE(t2) PARTITION BY partition_col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_REQUIRED_METADATA_INVALID", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "UDTFInvalidOrderByWithoutPartitionBy", + "reason" : "the 'order_by' field cannot be non-empty unless the 'with_single_partition' field is set to true or the 'partition_by' list is non-empty" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 88, + "fragment" : "UDTFInvalidOrderByWithoutPartitionBy(TABLE(t2) PARTITION BY partition_col)" + } ] +} + + +-- !query +SELECT * FROM + VALUES (0), (1) AS t(col) + JOIN LATERAL + UDTFInvalidOrderByWithoutPartitionBy(TABLE(t2) PARTITION BY partition_col) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_REQUIRED_METADATA_INVALID", + "sqlState" : "22023", + "messageParameters" : { + "functionName" : "UDTFInvalidOrderByWithoutPartitionBy", + "reason" : "the 'order_by' field cannot be non-empty unless the 'with_single_partition' field is set to true or the 'partition_by' list is non-empty" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 66, + "stopIndex" : 139, + "fragment" : "UDTFInvalidOrderByWithoutPartitionBy(TABLE(t2) PARTITION BY partition_col)" + } ] +} + + +-- !query +SELECT * FROM InvalidEvalReturnsNoneToNonNullableColumnScalarType(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.api.python.PythonException +pyspark.errors.exceptions.base.PySparkRuntimeError: [UDTF_EXEC_ERROR] User defined table function encountered an error in the 'eval' or 'terminate' method: Column 0 within a returned row had a value of None, either directly or within array/struct/map subfields, but the corresponding column type was declared as non-nullable; please update the UDTF to return a non-None value at this location or otherwise declare the column type as nullable. + + +-- !query +SELECT * FROM InvalidEvalReturnsNoneToNonNullableColumnArrayType(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.api.python.PythonException +pyspark.errors.exceptions.base.PySparkRuntimeError: [UDTF_EXEC_ERROR] User defined table function encountered an error in the 'eval' or 'terminate' method: Column 0 within a returned row had a value of None, either directly or within array/struct/map subfields, but the corresponding column type was declared as non-nullable; please update the UDTF to return a non-None value at this location or otherwise declare the column type as nullable. + + +-- !query +SELECT * FROM InvalidEvalReturnsNoneToNonNullableColumnArrayElementType(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.api.python.PythonException +pyspark.errors.exceptions.base.PySparkRuntimeError: [UDTF_EXEC_ERROR] User defined table function encountered an error in the 'eval' or 'terminate' method: Column 0 within a returned row had a value of None, either directly or within array/struct/map subfields, but the corresponding column type was declared as non-nullable; please update the UDTF to return a non-None value at this location or otherwise declare the column type as nullable. + + +-- !query +SELECT * FROM InvalidEvalReturnsNoneToNonNullableColumnStructType(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.api.python.PythonException +pyspark.errors.exceptions.base.PySparkRuntimeError: [UDTF_EXEC_ERROR] User defined table function encountered an error in the 'eval' or 'terminate' method: Column 0 within a returned row had a value of None, either directly or within array/struct/map subfields, but the corresponding column type was declared as non-nullable; please update the UDTF to return a non-None value at this location or otherwise declare the column type as nullable. + + +-- !query +SELECT * FROM InvalidEvalReturnsNoneToNonNullableColumnMapType(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.api.python.PythonException +pyspark.errors.exceptions.base.PySparkRuntimeError: [UDTF_EXEC_ERROR] User defined table function encountered an error in the 'eval' or 'terminate' method: Column 0 within a returned row had a value of None, either directly or within array/struct/map subfields, but the corresponding column type was declared as non-nullable; please update the UDTF to return a non-None value at this location or otherwise declare the column type as nullable. + + +-- !query +SELECT * FROM InvalidTerminateReturnsNoneToNonNullableColumnScalarType(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.api.python.PythonException +pyspark.errors.exceptions.base.PySparkRuntimeError: [UDTF_EXEC_ERROR] User defined table function encountered an error in the 'eval' or 'terminate' method: Column 0 within a returned row had a value of None, either directly or within array/struct/map subfields, but the corresponding column type was declared as non-nullable; please update the UDTF to return a non-None value at this location or otherwise declare the column type as nullable. + + +-- !query +SELECT * FROM InvalidTerminateReturnsNoneToNonNullableColumnArrayType(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.api.python.PythonException +pyspark.errors.exceptions.base.PySparkRuntimeError: [UDTF_EXEC_ERROR] User defined table function encountered an error in the 'eval' or 'terminate' method: Column 0 within a returned row had a value of None, either directly or within array/struct/map subfields, but the corresponding column type was declared as non-nullable; please update the UDTF to return a non-None value at this location or otherwise declare the column type as nullable. + + +-- !query +SELECT * FROM InvalidTerminateReturnsNoneToNonNullableColumnArrayElementType(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.api.python.PythonException +pyspark.errors.exceptions.base.PySparkRuntimeError: [UDTF_EXEC_ERROR] User defined table function encountered an error in the 'eval' or 'terminate' method: Column 0 within a returned row had a value of None, either directly or within array/struct/map subfields, but the corresponding column type was declared as non-nullable; please update the UDTF to return a non-None value at this location or otherwise declare the column type as nullable. + + +-- !query +SELECT * FROM InvalidTerminateReturnsNoneToNonNullableColumnStructType(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.api.python.PythonException +pyspark.errors.exceptions.base.PySparkRuntimeError: [UDTF_EXEC_ERROR] User defined table function encountered an error in the 'eval' or 'terminate' method: Column 0 within a returned row had a value of None, either directly or within array/struct/map subfields, but the corresponding column type was declared as non-nullable; please update the UDTF to return a non-None value at this location or otherwise declare the column type as nullable. + + +-- !query +SELECT * FROM InvalidTerminateReturnsNoneToNonNullableColumnMapType(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.api.python.PythonException +pyspark.errors.exceptions.base.PySparkRuntimeError: [UDTF_EXEC_ERROR] User defined table function encountered an error in the 'eval' or 'terminate' method: Column 0 within a returned row had a value of None, either directly or within array/struct/map subfields, but the corresponding column type was declared as non-nullable; please update the UDTF to return a non-None value at this location or otherwise declare the column type as nullable. + + +-- !query +SELECT * FROM UDTFForwardStateFromAnalyzeWithKwargs() +-- !query schema +struct<> +-- !query output +org.apache.spark.api.python.PythonException +pyspark.errors.exceptions.base.PySparkRuntimeError: [UDTF_EVAL_METHOD_ARGUMENTS_DO_NOT_MATCH_SIGNATURE] Failed to evaluate the user-defined table function 'UDTFForwardStateFromAnalyzeWithKwargs' because the function arguments did not match the expected signature of the 'eval' method (missing a required argument: 'argument'). Please update the query so that this table function call provides arguments matching the expected signature, or else update the table function so that its 'eval' method accepts the provided arguments, and then try the query again. + + +-- !query +SELECT * FROM UDTFForwardStateFromAnalyzeWithKwargs(1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_FAILED_TO_ANALYZE_IN_PYTHON", + "sqlState" : "38000", + "messageParameters" : { + "msg" : "Failed to evaluate the user-defined table function 'UDTFForwardStateFromAnalyzeWithKwargs' because the function arguments did not match the expected signature of the static 'analyze' method (too many positional arguments). Please update the query so that this table function call provides arguments matching the expected signature, or else update the table function so that its static 'analyze' method accepts the provided arguments, and then try the query again." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 57, + "fragment" : "UDTFForwardStateFromAnalyzeWithKwargs(1, 2)" + } ] +} + + +-- !query +SELECT * FROM UDTFForwardStateFromAnalyzeWithKwargs(invalid => 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.api.python.PythonException +pyspark.errors.exceptions.base.PySparkRuntimeError: [UDTF_EVAL_METHOD_ARGUMENTS_DO_NOT_MATCH_SIGNATURE] Failed to evaluate the user-defined table function 'UDTFForwardStateFromAnalyzeWithKwargs' because the function arguments did not match the expected signature of the 'eval' method (missing a required argument: 'argument'). Please update the query so that this table function call provides arguments matching the expected signature, or else update the table function so that its 'eval' method accepts the provided arguments, and then try the query again. + + +-- !query +SELECT * FROM UDTFForwardStateFromAnalyzeWithKwargs(argument => 1, argument => 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.DOUBLE_NAMED_ARGUMENT_REFERENCE", + "sqlState" : "4274K", + "messageParameters" : { + "parameterName" : "`argument`", + "routineName" : "`UDTFForwardStateFromAnalyzeWithKwargs`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 81, + "fragment" : "UDTFForwardStateFromAnalyzeWithKwargs(argument => 1, argument => 2)" + } ] +} + + +-- !query +SELECT * FROM InvalidAnalyzeMethodWithSinglePartitionNoInputTable(argument => 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_FAILED_TO_ANALYZE_IN_PYTHON", + "sqlState" : "38000", + "messageParameters" : { + "msg" : "Failed to evaluate the user-defined table function 'InvalidAnalyzeMethodWithSinglePartitionNoInputTable' because the static 'analyze' method returned an 'AnalyzeResult' object with the 'withSinglePartition' field set to 'true', but the function call did not provide any table argument. Please update the query so that it provides a table argument, or else update the table function so that its 'analyze' method returns an 'AnalyzeResult' object with the 'withSinglePartition' field set to 'false', and then try the query again." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 80, + "fragment" : "InvalidAnalyzeMethodWithSinglePartitionNoInputTable(argument => 1)" + } ] +} + + +-- !query +SELECT * FROM InvalidAnalyzeMethodWithPartitionByNoInputTable(argument => 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_FAILED_TO_ANALYZE_IN_PYTHON", + "sqlState" : "38000", + "messageParameters" : { + "msg" : "Failed to evaluate the user-defined table function 'InvalidAnalyzeMethodWithPartitionByNoInputTable' because the static 'analyze' method returned an 'AnalyzeResult' object with the 'partitionBy' list set to non-empty, but the function call did not provide any table argument. Please update the query so that it provides a table argument, or else update the table function so that its 'analyze' method returns an 'AnalyzeResult' object with the 'partitionBy' list set to empty, and then try the query again." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 76, + "fragment" : "InvalidAnalyzeMethodWithPartitionByNoInputTable(argument => 1)" + } ] +} + + +-- !query +SELECT * FROM InvalidAnalyzeMethodReturnsNonStructTypeSchema(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_FAILED_TO_ANALYZE_IN_PYTHON", + "sqlState" : "38000", + "messageParameters" : { + "msg" : "Failed to evaluate the user-defined table function 'InvalidAnalyzeMethodReturnsNonStructTypeSchema' because the static 'analyze' method expects a result of type pyspark.sql.udtf.AnalyzeResult with a 'schema' field comprising a StructType, but the 'schema' field had the wrong type: " + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 71, + "fragment" : "InvalidAnalyzeMethodReturnsNonStructTypeSchema(TABLE(t2))" + } ] +} + + +-- !query +SELECT * FROM InvalidAnalyzeMethodWithPartitionByListOfStrings(argument => TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_FAILED_TO_ANALYZE_IN_PYTHON", + "sqlState" : "38000", + "messageParameters" : { + "msg" : "Failed to evaluate the user-defined table function 'InvalidAnalyzeMethodWithPartitionByListOfStrings' because the static 'analyze' method returned an 'AnalyzeResult' object with the 'partitionBy' field set to a value besides a list or tuple of 'PartitioningColumn' objects. Please update the table function and then try the query again." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 85, + "fragment" : "InvalidAnalyzeMethodWithPartitionByListOfStrings(argument => TABLE(t2))" + } ] +} + + +-- !query +SELECT * FROM InvalidForwardStateFromAnalyzeTooManyInitArgs(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.api.python.PythonException +pyspark.errors.exceptions.base.PySparkRuntimeError: [UDTF_CONSTRUCTOR_INVALID_IMPLEMENTS_ANALYZE_METHOD] Failed to evaluate the user-defined table function 'InvalidForwardStateFromAnalyzeTooManyInitArgs' because its constructor is invalid: the function implements the 'analyze' method, but its constructor has more than two arguments (including the 'self' reference). Please update the table function so that its constructor accepts exactly one 'self' argument, or one 'self' argument plus another argument for the result of the 'analyze' method, and try the query again. + + +-- !query +SELECT * FROM InvalidNotForwardStateFromAnalyzeTooManyInitArgs(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.api.python.PythonException +pyspark.errors.exceptions.base.PySparkRuntimeError: [UDTF_CONSTRUCTOR_INVALID_NO_ANALYZE_METHOD] Failed to evaluate the user-defined table function 'InvalidNotForwardStateFromAnalyzeTooManyInitArgs' because its constructor is invalid: the function does not implement the 'analyze' method, and its constructor has more than one argument (including the 'self' reference). Please update the table function so that its constructor accepts exactly one 'self' argument, and try the query again. + + +-- !query +SELECT * FROM UDTFWithSinglePartition(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_FAILED_TO_ANALYZE_IN_PYTHON", + "sqlState" : "38000", + "messageParameters" : { + "msg" : "Failed to evaluate the user-defined table function 'UDTFWithSinglePartition' because the function arguments did not match the expected signature of the static 'analyze' method (missing a required argument: 'input_table'). Please update the query so that this table function call provides arguments matching the expected signature, or else update the table function so that its static 'analyze' method accepts the provided arguments, and then try the query again." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 40, + "fragment" : "UDTFWithSinglePartition(1)" + } ] +} + + +-- !query +SELECT * FROM UDTFWithSinglePartition(1, 2, 3) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_FAILED_TO_ANALYZE_IN_PYTHON", + "sqlState" : "38000", + "messageParameters" : { + "msg" : "Failed to evaluate the user-defined table function 'UDTFWithSinglePartition' because the function arguments did not match the expected signature of the static 'analyze' method (too many positional arguments). Please update the query so that this table function call provides arguments matching the expected signature, or else update the table function so that its static 'analyze' method accepts the provided arguments, and then try the query again." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 46, + "fragment" : "UDTFWithSinglePartition(1, 2, 3)" + } ] +} + + +-- !query +SELECT * FROM UDTFWithSinglePartition(1, invalid_arg_name => 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_FAILED_TO_ANALYZE_IN_PYTHON", + "sqlState" : "38000", + "messageParameters" : { + "msg" : "Failed to evaluate the user-defined table function 'UDTFWithSinglePartition' because the function arguments did not match the expected signature of the static 'analyze' method (missing a required argument: 'input_table'). Please update the query so that this table function call provides arguments matching the expected signature, or else update the table function so that its static 'analyze' method accepts the provided arguments, and then try the query again." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 63, + "fragment" : "UDTFWithSinglePartition(1, invalid_arg_name => 2)" + } ] +} + + +-- !query +SELECT * FROM UDTFWithSinglePartition(1, initial_count => 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "TABLE_VALUED_FUNCTION_FAILED_TO_ANALYZE_IN_PYTHON", + "sqlState" : "38000", + "messageParameters" : { + "msg" : "Failed to evaluate the user-defined table function 'UDTFWithSinglePartition' because the function arguments did not match the expected signature of the static 'analyze' method (multiple values for argument 'initial_count'). Please update the query so that this table function call provides arguments matching the expected signature, or else update the table function so that its static 'analyze' method accepts the provided arguments, and then try the query again." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 60, + "fragment" : "UDTFWithSinglePartition(1, initial_count => 2)" + } ] +} + + +-- !query +SELECT * FROM UDTFWithSinglePartition(initial_count => 1, initial_count => 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.DOUBLE_NAMED_ARGUMENT_REFERENCE", + "sqlState" : "4274K", + "messageParameters" : { + "parameterName" : "`initial_count`", + "routineName" : "`UDTFWithSinglePartition`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 77, + "fragment" : "UDTFWithSinglePartition(initial_count => 1, initial_count => 2)" + } ] +} + + +-- !query +SELECT * FROM UDTFInvalidPartitionByOrderByParseError(TABLE(t2)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`unparsable`", + "proposal" : "`input`, `partition_col`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 10, + "fragment" : "unparsable" + } ] +} + + +-- !query +SELECT * FROM UDTFPartitionByIndexingBug( + TABLE( + SELECT + 5 AS unused_col, + 'hi' AS partition_col, + 1.0 AS double_col + + UNION ALL + + SELECT + 4 AS unused_col, + 'hi' AS partition_col, + 1.0 AS double_col + ) +) +-- !query schema +struct +-- !query output +NULL 1.0 +NULL 1.0 +NULL 1.0 +NULL 1.0 +NULL 1.0 + + +-- !query +SELECT * FROM + InvalidEvalReturnsNoneToNonNullableColumnScalarType(TABLE(SELECT 1 AS X), unresolved_column) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`unresolved_column`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 93, + "stopIndex" : 109, + "fragment" : "unresolved_column" + } ] +} + + +-- !query +DROP VIEW t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW t2 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/union.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/union.sql.out new file mode 100644 index 000000000000..ce4d3421ae83 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/union.sql.out @@ -0,0 +1,248 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW t1 AS VALUES (1, 'a'), (2, 'b') tbl(c1, c2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t2 AS VALUES (1.0, 1), (2.0, 4) tbl(c1, c2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * +FROM (SELECT * FROM t1 + UNION ALL + SELECT * FROM t1) +-- !query schema +struct +-- !query output +1 a +1 a +2 b +2 b + + +-- !query +SELECT * +FROM (SELECT * FROM t1 where c1 = 1 + UNION ALL + SELECT * FROM t2 + UNION ALL + SELECT * FROM t2) +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 18, + "stopIndex" : 89, + "fragment" : "SELECT * FROM t1 where c1 = 1\n UNION ALL\n SELECT * FROM t2" + } ] +} + + +-- !query +SELECT a +FROM (SELECT 0 a, 0 b + UNION ALL + SELECT SUM(1) a, CAST(0 AS BIGINT) b + UNION ALL SELECT 0 a, 0 b) T +-- !query schema +struct +-- !query output +0 +0 +1 + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW p1 AS VALUES 1 T(col) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW p2 AS VALUES 1 T(col) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW p3 AS VALUES 1 T(col) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT 1 AS x, + col +FROM (SELECT col AS col + FROM (SELECT p1.col AS col + FROM p1 CROSS JOIN p2 + UNION ALL + SELECT col + FROM p3) T1) T2 +-- !query schema +struct +-- !query output +1 1 +1 1 + + +-- !query +SELECT map(1, 2), 'str' +UNION ALL +SELECT map(1, 2, 3, NULL), 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'str'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 62, + "fragment" : "SELECT map(1, 2), 'str'\nUNION ALL\nSELECT map(1, 2, 3, NULL), 1" + } ] +} + + +-- !query +SELECT array(1, 2), 'str' +UNION ALL +SELECT array(1, 2, 3, NULL), 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'str'", + "sourceType" : "\"STRING\"", + "targetType" : "\"BIGINT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 66, + "fragment" : "SELECT array(1, 2), 'str'\nUNION ALL\nSELECT array(1, 2, 3, NULL), 1" + } ] +} + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW t3 AS VALUES (decimal(1)) tbl(v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT t.v FROM ( + SELECT v FROM t3 + UNION ALL + SELECT v + v AS v FROM t3 +) t +-- !query schema +struct +-- !query output +1 +2 + + +-- !query +SELECT SUM(t.v) FROM ( + SELECT v FROM t3 + UNION + SELECT v + v AS v FROM t3 +) t +-- !query schema +struct +-- !query output +3 + + +-- !query +DROP VIEW IF EXISTS t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS t3 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS p1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS p2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS p3 +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/unpivot.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/unpivot.sql.out new file mode 100644 index 000000000000..fe2db8ca1170 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/unpivot.sql.out @@ -0,0 +1,113 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view courseEarnings as select * from values + ("dotNET", 15000, 48000, 22500), + ("Java", 20000, 30000, NULL) + as courseEarnings(course, `2012`, `2013`, `2014`) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM courseEarnings +UNPIVOT ( + earningsYear FOR year IN (`2012`, `2013`, `2014`) +) +-- !query schema +struct +-- !query output +Java 2012 20000 +Java 2013 30000 +dotNET 2012 15000 +dotNET 2013 48000 +dotNET 2014 22500 + + +-- !query +SELECT * FROM courseEarnings +UNPIVOT INCLUDE NULLS ( + earningsYear FOR year IN (`2012`, `2013`, `2014`) +) +-- !query schema +struct +-- !query output +Java 2012 20000 +Java 2013 30000 +Java 2014 NULL +dotNET 2012 15000 +dotNET 2013 48000 +dotNET 2014 22500 + + +-- !query +SELECT * FROM courseEarnings +UNPIVOT ( + earningsYear FOR year IN (`2012` as `twenty-twelve`, `2013` as `twenty-thirteen`, `2014` as `twenty-fourteen`) +) +-- !query schema +struct +-- !query output +Java twenty-thirteen 30000 +Java twenty-twelve 20000 +dotNET twenty-fourteen 22500 +dotNET twenty-thirteen 48000 +dotNET twenty-twelve 15000 + + +-- !query +create temporary view courseEarningsAndSales as select * from values + ("dotNET", 15000, NULL, 48000, 1, 22500, 1), + ("Java", 20000, 1, 30000, 2, NULL, NULL) + as courseEarningsAndSales(course, earnings2012, sales2012, earnings2013, sales2013, earnings2014, sales2014) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM courseEarningsAndSales +UNPIVOT ( + (earnings, sales) FOR year IN ((earnings2012, sales2012), (earnings2013, sales2013), (earnings2014, sales2014)) +) +-- !query schema +struct +-- !query output +Java earnings2012_sales2012 20000 1 +Java earnings2013_sales2013 30000 2 +dotNET earnings2012_sales2012 15000 NULL +dotNET earnings2013_sales2013 48000 1 +dotNET earnings2014_sales2014 22500 1 + + +-- !query +SELECT * FROM courseEarningsAndSales +UNPIVOT INCLUDE NULLS ( + (earnings, sales) FOR year IN ((earnings2012, sales2012), (earnings2013, sales2013), (earnings2014, sales2014)) +) +-- !query schema +struct +-- !query output +Java earnings2012_sales2012 20000 1 +Java earnings2013_sales2013 30000 2 +Java earnings2014_sales2014 NULL NULL +dotNET earnings2012_sales2012 15000 NULL +dotNET earnings2013_sales2013 48000 1 +dotNET earnings2014_sales2014 22500 1 + + +-- !query +SELECT * FROM courseEarningsAndSales +UNPIVOT ( + (earnings, sales) FOR year IN ((earnings2012, sales2012) as `2012`, (earnings2013, sales2013) as `2013`, (earnings2014, sales2014) as `2014`) +) +-- !query schema +struct +-- !query output +Java 2012 20000 1 +Java 2013 30000 2 +dotNET 2012 15000 NULL +dotNET 2013 48000 1 +dotNET 2014 22500 1 diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/url-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/url-functions.sql.out new file mode 100644 index 000000000000..2d1daee8500a --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/url-functions.sql.out @@ -0,0 +1,158 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'HOST') +-- !query schema +struct +-- !query output +spark.apache.org + + +-- !query +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'PATH') +-- !query schema +struct +-- !query output +/path + + +-- !query +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'QUERY') +-- !query schema +struct +-- !query output +query=1 + + +-- !query +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'REF') +-- !query schema +struct +-- !query output +Ref + + +-- !query +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'PROTOCOL') +-- !query schema +struct +-- !query output +http + + +-- !query +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'FILE') +-- !query schema +struct +-- !query output +/path?query=1 + + +-- !query +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'AUTHORITY') +-- !query schema +struct +-- !query output +userinfo@spark.apache.org + + +-- !query +select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'USERINFO') +-- !query schema +struct +-- !query output +userinfo + + +-- !query +select url_encode('https://spark.apache.org') +-- !query schema +struct +-- !query output +https%3A%2F%2Fspark.apache.org + + +-- !query +select url_encode('inva lid://user:pass@host/file\\;param?query\\;p2') +-- !query schema +struct +-- !query output +inva+lid%3A%2F%2Fuser%3Apass%40host%2Ffile%5C%3Bparam%3Fquery%5C%3Bp2 + + +-- !query +select url_encode(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select url_decode('https%3A%2F%2Fspark.apache.org') +-- !query schema +struct +-- !query output +https://spark.apache.org + + +-- !query +select url_decode('http%3A%2F%2spark.apache.org') +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkIllegalArgumentException +{ + "errorClass" : "CANNOT_DECODE_URL", + "sqlState" : "22546", + "messageParameters" : { + "url" : "http%3A%2F%2spark.apache.org" + } +} + + +-- !query +select url_decode('inva lid://user:pass@host/file\\;param?query\\;p2') +-- !query schema +struct +-- !query output +inva lid://user:pass@host/file\;param?query\;p2 + + +-- !query +select url_decode(null) +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_url_decode('https%3A%2F%2Fspark.apache.org') +-- !query schema +struct +-- !query output +https://spark.apache.org + + +-- !query +select try_url_decode('http%3A%2F%2spark.apache.org') +-- !query schema +struct +-- !query output +NULL + + +-- !query +select try_url_decode('inva lid://user:pass@host/file\\;param?query\\;p2') +-- !query schema +struct +-- !query output +inva lid://user:pass@host/file\;param?query\;p2 + + +-- !query +select try_url_decode(null) +-- !query schema +struct +-- !query output +NULL diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/using-join.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/using-join.sql.out new file mode 100644 index 000000000000..1e71191de873 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/using-join.sql.out @@ -0,0 +1,447 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1) +-- !query schema +struct<> +-- !query output + + + +-- !query +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5), + ("four", 4) + as nt2(k, v2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM nt1 left outer join nt2 using (k) +-- !query schema +struct +-- !query output +one 1 1 +one 1 5 +three 3 NULL +two 2 22 + + +-- !query +SELECT k FROM nt1 left outer join nt2 using (k) +-- !query schema +struct +-- !query output +one +one +three +two + + +-- !query +SELECT nt1.*, nt2.* FROM nt1 left outer join nt2 using (k) +-- !query schema +struct +-- !query output +one 1 one 1 +one 1 one 5 +three 3 NULL NULL +two 2 two 22 + + +-- !query +SELECT nt1.k, nt2.k FROM nt1 left outer join nt2 using (k) +-- !query schema +struct +-- !query output +one one +one one +three NULL +two two + + +-- !query +SELECT k FROM (SELECT nt2.k FROM nt1 left outer join nt2 using (k)) +-- !query schema +struct +-- !query output +NULL +one +one +two + + +-- !query +SELECT nt2.k AS key FROM nt1 left outer join nt2 using (k) ORDER BY key +-- !query schema +struct +-- !query output +NULL +one +one +two + + +-- !query +SELECT nt1.k, nt2.k FROM nt1 left outer join nt2 using (k) ORDER BY nt2.k +-- !query schema +struct +-- !query output +three NULL +one one +one one +two two + + +-- !query +SELECT k, nt1.k FROM nt1 left outer join nt2 using (k) +-- !query schema +struct +-- !query output +one one +one one +three three +two two + + +-- !query +SELECT k, nt2.k FROM nt1 left outer join nt2 using (k) +-- !query schema +struct +-- !query output +one one +one one +three NULL +two two + + +-- !query +SELECT * FROM nt1 left semi join nt2 using (k) +-- !query schema +struct +-- !query output +one 1 +two 2 + + +-- !query +SELECT k FROM nt1 left semi join nt2 using (k) +-- !query schema +struct +-- !query output +one +two + + +-- !query +SELECT nt1.* FROM nt1 left semi join nt2 using (k) +-- !query schema +struct +-- !query output +one 1 +two 2 + + +-- !query +SELECT nt1.k FROM nt1 left semi join nt2 using (k) +-- !query schema +struct +-- !query output +one +two + + +-- !query +SELECT k, nt1.k FROM nt1 left semi join nt2 using (k) +-- !query schema +struct +-- !query output +one one +two two + + +-- !query +SELECT * FROM nt1 right outer join nt2 using (k) +-- !query schema +struct +-- !query output +four NULL 4 +one 1 1 +one 1 5 +two 2 22 + + +-- !query +SELECT k FROM nt1 right outer join nt2 using (k) +-- !query schema +struct +-- !query output +four +one +one +two + + +-- !query +SELECT nt1.*, nt2.* FROM nt1 right outer join nt2 using (k) +-- !query schema +struct +-- !query output +NULL NULL four 4 +one 1 one 1 +one 1 one 5 +two 2 two 22 + + +-- !query +SELECT nt1.k, nt2.k FROM nt1 right outer join nt2 using (k) +-- !query schema +struct +-- !query output +NULL four +one one +one one +two two + + +-- !query +SELECT k FROM (SELECT nt1.k FROM nt1 right outer join nt2 using (k)) +-- !query schema +struct +-- !query output +NULL +one +one +two + + +-- !query +SELECT nt1.k AS key FROM nt1 right outer join nt2 using (k) ORDER BY key +-- !query schema +struct +-- !query output +NULL +one +one +two + + +-- !query +SELECT k, nt1.k FROM nt1 right outer join nt2 using (k) +-- !query schema +struct +-- !query output +four NULL +one one +one one +two two + + +-- !query +SELECT k, nt2.k FROM nt1 right outer join nt2 using (k) +-- !query schema +struct +-- !query output +four four +one one +one one +two two + + +-- !query +SELECT * FROM nt1 full outer join nt2 using (k) +-- !query schema +struct +-- !query output +four NULL 4 +one 1 1 +one 1 5 +three 3 NULL +two 2 22 + + +-- !query +SELECT k FROM nt1 full outer join nt2 using (k) +-- !query schema +struct +-- !query output +four +one +one +three +two + + +-- !query +SELECT nt1.*, nt2.* FROM nt1 full outer join nt2 using (k) +-- !query schema +struct +-- !query output +NULL NULL four 4 +one 1 one 1 +one 1 one 5 +three 3 NULL NULL +two 2 two 22 + + +-- !query +SELECT nt1.k, nt2.k FROM nt1 full outer join nt2 using (k) +-- !query schema +struct +-- !query output +NULL four +one one +one one +three NULL +two two + + +-- !query +SELECT k FROM (SELECT nt2.k FROM nt1 full outer join nt2 using (k)) +-- !query schema +struct +-- !query output +NULL +four +one +one +two + + +-- !query +SELECT nt2.k AS key FROM nt1 full outer join nt2 using (k) ORDER BY key +-- !query schema +struct +-- !query output +NULL +four +one +one +two + + +-- !query +SELECT k, nt1.k FROM nt1 full outer join nt2 using (k) +-- !query schema +struct +-- !query output +four NULL +one one +one one +three three +two two + + +-- !query +SELECT k, nt2.k FROM nt1 full outer join nt2 using (k) +-- !query schema +struct +-- !query output +four four +one one +one one +three NULL +two two + + +-- !query +SELECT * FROM nt1 full outer join nt2 using (k) +-- !query schema +struct +-- !query output +four NULL 4 +one 1 1 +one 1 5 +three 3 NULL +two 2 22 + + +-- !query +SELECT k FROM nt1 inner join nt2 using (k) +-- !query schema +struct +-- !query output +one +one +two + + +-- !query +SELECT nt1.*, nt2.* FROM nt1 inner join nt2 using (k) +-- !query schema +struct +-- !query output +one 1 one 1 +one 1 one 5 +two 2 two 22 + + +-- !query +SELECT nt1.k, nt2.k FROM nt1 inner join nt2 using (k) +-- !query schema +struct +-- !query output +one one +one one +two two + + +-- !query +SELECT k FROM (SELECT nt2.k FROM nt1 inner join nt2 using (k)) +-- !query schema +struct +-- !query output +one +one +two + + +-- !query +SELECT nt2.k AS key FROM nt1 inner join nt2 using (k) ORDER BY key +-- !query schema +struct +-- !query output +one +one +two + + +-- !query +SELECT k, nt1.k FROM nt1 inner join nt2 using (k) +-- !query schema +struct +-- !query output +one one +one one +two two + + +-- !query +SELECT k, nt2.k FROM nt1 inner join nt2 using (k) +-- !query schema +struct +-- !query output +one one +one one +two two + + +-- !query +WITH + t1 AS (select key from values ('a') t(key)), + t2 AS (select key from values ('a') t(key)) +SELECT t1.key +FROM t1 FULL OUTER JOIN t2 USING (key) +WHERE t1.key NOT LIKE 'bb.%' +-- !query schema +struct +-- !query output +a diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/variant/named-function-arguments.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/variant/named-function-arguments.sql.out new file mode 100644 index 000000000000..10a439831fbd --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/variant/named-function-arguments.sql.out @@ -0,0 +1,48 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT * FROM variant_explode(input => parse_json('["hello", "world"]')) +-- !query schema +struct +-- !query output +0 NULL "hello" +1 NULL "world" + + +-- !query +SELECT * FROM variant_explode_outer(input => parse_json('{"a": true, "b": 3.14}')) +-- !query schema +struct +-- !query output +0 a true +1 b 3.14 + + +-- !query +SELECT * FROM variant_explode(parse_json('["hello", "world"]')), variant_explode(parse_json('{"a": true, "b": 3.14}')) +-- !query schema +struct +-- !query output +0 NULL "hello" 0 a true +0 NULL "hello" 1 b 3.14 +1 NULL "world" 0 a true +1 NULL "world" 1 b 3.14 + + +-- !query +SELECT * FROM variant_explode(parse_json('{"a": ["hello", "world"], "b": {"x": true, "y": 3.14}}')) AS t, LATERAL variant_explode(t.value) +-- !query schema +struct +-- !query output +0 a ["hello","world"] 0 NULL "hello" +0 a ["hello","world"] 1 NULL "world" +1 b {"x":true,"y":3.14} 0 x true +1 b {"x":true,"y":3.14} 1 y 3.14 + + +-- !query +SELECT num, key, val, 'Spark' FROM variant_explode(parse_json('["hello", "world"]')) AS t(num, key, val) +-- !query schema +struct +-- !query output +0 NULL "hello" Spark +1 NULL "world" Spark diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/view-schema-binding-config.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/view-schema-binding-config.sql.out new file mode 100644 index 000000000000..270f15e0b493 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/view-schema-binding-config.sql.out @@ -0,0 +1,1229 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SET spark.sql.legacy.viewSchemaBindingMode +-- !query schema +struct +-- !query output +spark.sql.legacy.viewSchemaBindingMode true + + +-- !query +SET spark.sql.legacy.viewSchemaBindingMode = false +-- !query schema +struct +-- !query output +spark.sql.legacy.viewSchemaBindingMode false + + +-- !query +CREATE OR REPLACE VIEW v WITH SCHEMA BINDING AS SELECT 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "FEATURE_NOT_ENABLED", + "sqlState" : "56038", + "messageParameters" : { + "configKey" : "spark.sql.legacy.viewSchemaBindingMode", + "configValue" : "true", + "featureName" : "VIEW ... WITH SCHEMA ..." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 44, + "fragment" : "WITH SCHEMA BINDING" + } ] +} + + +-- !query +CREATE OR REPLACE VIEW v WITH SCHEMA COMPENSATION AS SELECT 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "FEATURE_NOT_ENABLED", + "sqlState" : "56038", + "messageParameters" : { + "configKey" : "spark.sql.legacy.viewSchemaBindingMode", + "configValue" : "true", + "featureName" : "VIEW ... WITH SCHEMA ..." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 49, + "fragment" : "WITH SCHEMA COMPENSATION" + } ] +} + + +-- !query +CREATE OR REPLACE VIEW v WITH SCHEMA TYPE EVOLUTION AS SELECT 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "FEATURE_NOT_ENABLED", + "sqlState" : "56038", + "messageParameters" : { + "configKey" : "spark.sql.legacy.viewSchemaBindingMode", + "configValue" : "true", + "featureName" : "VIEW ... WITH SCHEMA ..." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 51, + "fragment" : "WITH SCHEMA TYPE EVOLUTION" + } ] +} + + +-- !query +CREATE OR REPLACE VIEW v WITH SCHEMA EVOLUTION AS SELECT 1 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "FEATURE_NOT_ENABLED", + "sqlState" : "56038", + "messageParameters" : { + "configKey" : "spark.sql.legacy.viewSchemaBindingMode", + "configValue" : "true", + "featureName" : "VIEW ... WITH SCHEMA ..." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 26, + "stopIndex" : 46, + "fragment" : "WITH SCHEMA EVOLUTION" + } ] +} + + +-- !query +CREATE OR REPLACE VIEW v AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT 1 +View Original Text SELECT 1 +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`1`] + + +-- !query +SHOW TABLE EXTENDED LIKE 'v' +-- !query schema +struct +-- !query output +default v false Catalog: spark_catalog +Database: default +Table: v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type: VIEW +View Text: SELECT 1 +View Original Text: SELECT 1 +View Catalog and Namespace: spark_catalog.default +View Query Output Columns: [`1`] +Schema: root + |-- 1: integer (nullable = false) + + +-- !query +SHOW CREATE TABLE v +-- !query schema +struct +-- !query output +CREATE VIEW default.v ( + `1`) +AS SELECT 1 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +1 int + + +-- !query +SHOW TABLE EXTENDED LIKE 'v' +-- !query schema +struct +-- !query output + v true Table: v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type: VIEW +View Text: SELECT 1 +View Catalog and Namespace: spark_catalog.default +View Query Output Columns: [`1`] +Schema: root + |-- 1: integer (nullable = false) + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT NOT NULL) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +SHOW CREATE TABLE v +-- !query schema +struct +-- !query output +CREATE VIEW default.v ( + c1) +AS SELECT * FROM t + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 BIGINT NOT NULL) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +SET spark.sql.legacy.viewSchemaBindingMode = true +-- !query schema +struct +-- !query output +spark.sql.legacy.viewSchemaBindingMode true + + +-- !query +SET spark.sql.legacy.viewSchemaCompensation = false +-- !query schema +struct +-- !query output +spark.sql.legacy.viewSchemaCompensation false + + +-- !query +SET spark.sql.ansi.enabled = false +-- !query schema +struct +-- !query output +spark.sql.ansi.enabled false + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT NOT NULL) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode BINDING +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +SHOW CREATE TABLE v +-- !query schema +struct +-- !query output +CREATE VIEW default.v ( + c1) +WITH SCHEMA BINDING +AS SELECT * FROM t + + +-- !query +DROP TABLE t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 BIGINT NOT NULL) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES (1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CANNOT_UP_CAST_DATATYPE", + "sqlState" : "42846", + "messageParameters" : { + "details" : "The type path of the target object is:\n\nYou can either add an explicit cast to the input data or choose a higher precision type of the field in the target object", + "expression" : "spark_catalog.default.t.c1", + "sourceType" : "\"BIGINT\"", + "targetType" : "\"INT\"" + } +} + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode BINDING +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +SHOW CREATE TABLE v +-- !query schema +struct +-- !query output +CREATE VIEW default.v ( + c1) +WITH SCHEMA BINDING +AS SELECT * FROM t + + +-- !query +SET spark.sql.legacy.viewSchemaCompensation = true +-- !query schema +struct +-- !query output +spark.sql.legacy.viewSchemaCompensation true + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT NOT NULL) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +SHOW CREATE TABLE v +-- !query schema +struct +-- !query output +CREATE VIEW default.v ( + c1) +WITH SCHEMA COMPENSATION +AS SELECT * FROM t + + +-- !query +DROP TABLE t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 BIGINT NOT NULL) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES (1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +SHOW CREATE TABLE v +-- !query schema +struct +-- !query output +CREATE VIEW default.v ( + c1) +WITH SCHEMA COMPENSATION +AS SELECT * FROM t + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 STRING NOT NULL, c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('1', 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +INSERT INTO t VALUES ('a', 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 15, + "fragment" : "v" + } ] +} + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 MAP, c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"c1\"", + "srcType" : "\"MAP\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 15, + "fragment" : "v" + } ] +} + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT, c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES (1, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 2 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int +c2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT NOT NULL) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "c2", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE VIEW spark_catalog.default.v AS SELECT * FROM t", + "viewName" : "`spark_catalog`.`default`.`v`" + } +} + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int +c2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c3 INT NOT NULL, c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "c1", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE VIEW spark_catalog.default.v AS SELECT * FROM t", + "viewName" : "`spark_catalog`.`default`.`v`" + } +} + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int +c2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +SET spark.sql.legacy.viewSchemaBindingMode = false +-- !query schema +struct +-- !query output +spark.sql.legacy.viewSchemaBindingMode false + + +-- !query +SET spark.sql.legacy.viewSchemaCompensation = false +-- !query schema +struct +-- !query output +spark.sql.legacy.viewSchemaCompensation false + + +-- !query +CREATE OR REPLACE VIEW v AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET spark.sql.legacy.viewSchemaBindingMode = true +-- !query schema +struct +-- !query output +spark.sql.legacy.viewSchemaBindingMode true + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT 1 +View Original Text SELECT 1 +View Schema Mode BINDING +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`1`] + + +-- !query +SHOW TABLE EXTENDED LIKE 'v' +-- !query schema +struct +-- !query output +default v false Catalog: spark_catalog +Database: default +Table: v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type: VIEW +View Text: SELECT 1 +View Original Text: SELECT 1 +View Schema Mode: BINDING +View Catalog and Namespace: spark_catalog.default +View Query Output Columns: [`1`] +Schema: root + |-- 1: integer (nullable = false) + + +-- !query +SHOW CREATE TABLE v +-- !query schema +struct +-- !query output +CREATE VIEW default.v ( + `1`) +WITH SCHEMA BINDING +AS SELECT 1 + + +-- !query +SET spark.sql.legacy.viewSchemaCompensation = true +-- !query schema +struct +-- !query output +spark.sql.legacy.viewSchemaCompensation true + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT 1 +View Original Text SELECT 1 +View Schema Mode BINDING +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`1`] + + +-- !query +SHOW TABLE EXTENDED LIKE 'v' +-- !query schema +struct +-- !query output +default v false Catalog: spark_catalog +Database: default +Table: v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type: VIEW +View Text: SELECT 1 +View Original Text: SELECT 1 +View Schema Mode: BINDING +View Catalog and Namespace: spark_catalog.default +View Query Output Columns: [`1`] +Schema: root + |-- 1: integer (nullable = false) + + +-- !query +SHOW CREATE TABLE v +-- !query schema +struct +-- !query output +CREATE VIEW default.v ( + `1`) +WITH SCHEMA BINDING +AS SELECT 1 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +SET spark.sql.legacy.viewSchemaBindingMode = false +-- !query schema +struct +-- !query output +spark.sql.legacy.viewSchemaBindingMode false + + +-- !query +SET spark.sql.legacy.viewSchemaCompensation = false +-- !query schema +struct +-- !query output +spark.sql.legacy.viewSchemaCompensation false + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SET spark.sql.legacy.viewSchemaBindingMode = true +-- !query schema +struct +-- !query output +spark.sql.legacy.viewSchemaBindingMode true + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +1 int + + +-- !query +SHOW TABLE EXTENDED LIKE 'v' +-- !query schema +struct +-- !query output + v true Table: v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type: VIEW +View Text: SELECT 1 +View Schema Mode: BINDING +View Catalog and Namespace: spark_catalog.default +View Query Output Columns: [`1`] +Schema: root + |-- 1: integer (nullable = false) + + +-- !query +SET spark.sql.legacy.viewSchemaCompensation = true +-- !query schema +struct +-- !query output +spark.sql.legacy.viewSchemaCompensation true + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +1 int + + +-- !query +SHOW TABLE EXTENDED LIKE 'v' +-- !query schema +struct +-- !query output + v true Table: v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type: VIEW +View Text: SELECT 1 +View Schema Mode: BINDING +View Catalog and Namespace: spark_catalog.default +View Query Output Columns: [`1`] +Schema: root + |-- 1: integer (nullable = false) + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/view-schema-binding.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/view-schema-binding.sql.out new file mode 100644 index 000000000000..b0311699d1b5 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/view-schema-binding.sql.out @@ -0,0 +1,386 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT NOT NULL) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v WITH SCHEMA BINDING AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode BINDING +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +DROP TABLE t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 BIGINT NOT NULL) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CANNOT_UP_CAST_DATATYPE", + "sqlState" : "42846", + "messageParameters" : { + "details" : "The type path of the target object is:\n\nYou can either add an explicit cast to the input data or choose a higher precision type of the field in the target object", + "expression" : "spark_catalog.default.t.c1", + "sourceType" : "\"BIGINT\"", + "targetType" : "\"INT\"" + } +} + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode BINDING +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT, c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v WITH SCHEMA BINDING AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int +c2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode BINDING +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "c2", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE VIEW spark_catalog.default.v AS SELECT * FROM t", + "viewName" : "`spark_catalog`.`default`.`v`" + } +} + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int +c2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode BINDING +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +SET spark.sql.legacy.viewSchemaCompensation=false +-- !query schema +struct +-- !query output +spark.sql.legacy.viewSchemaCompensation false + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT NOT NULL) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode BINDING +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +ALTER VIEW v WITH SCHEMA BINDING +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode BINDING +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +DROP TABLE t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 BIGINT NOT NULL) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CANNOT_UP_CAST_DATATYPE", + "sqlState" : "42846", + "messageParameters" : { + "details" : "The type path of the target object is:\n\nYou can either add an explicit cast to the input data or choose a higher precision type of the field in the target object", + "expression" : "spark_catalog.default.t.c1", + "sourceType" : "\"BIGINT\"", + "targetType" : "\"INT\"" + } +} + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode BINDING +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/view-schema-compensation.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/view-schema-compensation.sql.out new file mode 100644 index 000000000000..442d1f28744d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/view-schema-compensation.sql.out @@ -0,0 +1,593 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SET spark.sql.ansi.enabled = false +-- !query schema +struct +-- !query output +spark.sql.ansi.enabled false + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT NOT NULL) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v WITH SCHEMA COMPENSATION AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +DROP TABLE t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 BIGINT NOT NULL) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES (1) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 STRING NOT NULL, c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('1', 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +INSERT INTO t VALUES ('a', 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkNumberFormatException +{ + "errorClass" : "CAST_INVALID_INPUT", + "sqlState" : "22018", + "messageParameters" : { + "ansiConfig" : "\"spark.sql.ansi.enabled\"", + "expression" : "'a'", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 15, + "fragment" : "v" + } ] +} + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 MAP, c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.CAST_WITHOUT_SUGGESTION", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"c1\"", + "srcType" : "\"MAP\"", + "targetType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 15, + "fragment" : "v" + } ] +} + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT, c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES (1, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 2 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int +c2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT NOT NULL) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "c2", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE VIEW spark_catalog.default.v AS SELECT * FROM t", + "viewName" : "`spark_catalog`.`default`.`v`" + } +} + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int +c2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c3 INT NOT NULL, c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "c1", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE VIEW spark_catalog.default.v AS SELECT * FROM t", + "viewName" : "`spark_catalog`.`default`.`v`" + } +} + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int +c2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES(1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v WITH SCHEMA BINDING AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode BINDING +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 STRING) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES('1') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "CANNOT_UP_CAST_DATATYPE", + "sqlState" : "42846", + "messageParameters" : { + "details" : "The type path of the target object is:\n\nYou can either add an explicit cast to the input data or choose a higher precision type of the field in the target object", + "expression" : "spark_catalog.default.t.c1", + "sourceType" : "\"STRING\"", + "targetType" : "\"INT\"" + } +} + + +-- !query +ALTER VIEW v WITH SCHEMA COMPENSATION +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/view-schema-evolution.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/view-schema-evolution.sql.out new file mode 100644 index 000000000000..7410e7eaafd6 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/view-schema-evolution.sql.out @@ -0,0 +1,1113 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT NOT NULL, c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES (1, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v WITH SCHEMA EVOLUTION AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 2 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int +c2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c4 STRING NOT NULL, c5 DOUBLE) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('1', 2.0) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 2.0 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c4 string +c5 double + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c4`, `c5`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c4 STRING, c5 DOUBLE, c6 DATE) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('1', 2.0, DATE'2022-01-01') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 2.0 2022-01-01 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c4 string +c5 double +c6 date + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c4`, `c5`, `c6`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT, c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES (1, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v WITH SCHEMA EVOLUTION AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 2 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int +c2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT NOT NULL, c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES (1, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v(a1, a2) WITH SCHEMA EVOLUTION AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 2 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +a1 int +a2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 STRING NOT NULL, c2 DOUBLE) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('1', 2.0) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 2.0 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +a1 string +a2 double + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 STRING, c2 DOUBLE, c3 DATE) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('1', 2.0, DATE'2022-01-01') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 2.0 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +a1 string +a2 double + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT, c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES (1, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v(a1, a2) WITH SCHEMA EVOLUTION AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 2 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +a1 int +a2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "c2", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE VIEW spark_catalog.default.v (a1, a2) AS SELECT * FROM t", + "viewName" : "`spark_catalog`.`default`.`v`" + } +} + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +a1 int +a2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c3 INT, c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "c1", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE VIEW spark_catalog.default.v (a1, a2) AS SELECT * FROM t", + "viewName" : "`spark_catalog`.`default`.`v`" + } +} + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +a1 int +a2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT COMMENT 'c1', c2 INT COMMENT 'c2') USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v(a1, a2) WITH SCHEMA TYPE EVOLUTION AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +a1 int c1 +a2 int c2 + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +CREATE OR REPLACE VIEW v(a1, a2) WITH SCHEMA EVOLUTION AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +a1 int c1 +a2 int c2 + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 BIGINT COMMENT 'c1 6c', c2 STRING COMMENT 'c2 6c') USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +a1 bigint c1 +a2 string c2 + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +CREATE OR REPLACE VIEW v(a1 COMMENT 'a1', a2 COMMENT 'a2') WITH SCHEMA EVOLUTION AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +a1 bigint a1 +a2 string a2 + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 BIGINT COMMENT 'c1 6d', c2 STRING COMMENT 'c2 6d') USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +a1 bigint a1 +a2 string a2 + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +CREATE OR REPLACE VIEW v WITH SCHEMA EVOLUTION AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 bigint c1 6d +c2 string c2 6d + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 BIGINT COMMENT 'c1 6e', c2 STRING COMMENT 'c2 6e') USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 bigint c1 6e +c2 string c2 6e + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t1(c1 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t2(c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v WITH SCHEMA EVOLUTION AS SELECT * FROM t1, t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t2(c1 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "COLUMN_ALREADY_EXISTS", + "sqlState" : "42711", + "messageParameters" : { + "columnName" : "`c1`" + } +} + + +-- !query +DROP TABLE IF EXISTS t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES(1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER VIEW v WITH SCHEMA EVOLUTION +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 STRING, c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 string +c2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/view-schema-type-evolution.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/view-schema-type-evolution.sql.out new file mode 100644 index 000000000000..6254d254b39e --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/view-schema-type-evolution.sql.out @@ -0,0 +1,663 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT NOT NULL, c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES (1, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v WITH SCHEMA TYPE EVOLUTION AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 2 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int +c2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 STRING NOT NULL, c2 DOUBLE) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('1', 2.0) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 2.0 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 string +c2 double + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 STRING, c2 DOUBLE, c3 DATE) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('1', 2.0, DATE'2022-01-01') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 2.0 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 string +c2 double + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT, c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES (1, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v WITH SCHEMA TYPE EVOLUTION AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 2 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int +c2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "c2", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE VIEW spark_catalog.default.v AS SELECT * FROM t", + "viewName" : "`spark_catalog`.`default`.`v`" + } +} + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int +c2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c3 INT, c2 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE", + "sqlState" : "51024", + "messageParameters" : { + "actualCols" : "[]", + "colName" : "c1", + "expectedNum" : "1", + "suggestion" : "CREATE OR REPLACE VIEW spark_catalog.default.v AS SELECT * FROM t", + "viewName" : "`spark_catalog`.`default`.`v`" + } +} + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int +c2 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT COMMENT 'c1', c2 INT COMMENT 'c2') USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v(a1, a2) WITH SCHEMA TYPE EVOLUTION AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +a1 int c1 +a2 int c2 + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 BIGINT COMMENT 'c1 6a', c2 STRING COMMENT 'c2 6a') USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +a1 bigint c1 +a2 string c2 + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +CREATE OR REPLACE VIEW v(a1 COMMENT 'a1', a2 COMMENT 'a2') WITH SCHEMA TYPE EVOLUTION AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +a1 bigint a1 +a2 string a2 + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 BIGINT COMMENT 'c1 6b', c2 STRING COMMENT 'c2 6b') USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output + + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +a1 bigint a1 +a2 string a2 + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`, `c2`] + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 INT) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES(1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v WITH SCHEMA COMPENSATION AS SELECT * FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t(c1 STRING) USING PARQUET +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES('1') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 int + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode COMPENSATION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +ALTER VIEW v WITH SCHEMA TYPE EVOLUTION +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +1 + + +-- !query +DESCRIBE EXTENDED v +-- !query schema +struct +-- !query output +c1 string + +# Detailed Table Information +Catalog spark_catalog +Database default +Table v +Created Time [not included in comparison] +Last Access [not included in comparison] +Created By [not included in comparison] +Type VIEW +View Text SELECT * FROM t +View Original Text SELECT * FROM t +View Schema Mode TYPE EVOLUTION +View Catalog and Namespace spark_catalog.default +View Query Output Columns [`c1`] + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/view-with-default-collation.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/view-with-default-collation.sql.out new file mode 100644 index 000000000000..2078277d97a9 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/view-with-default-collation.sql.out @@ -0,0 +1,967 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v DEFAULT COLLATION UNICODE AS SELECT 'a' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<'a' collate UNICODE:string collate UNICODE> +-- !query output +a + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 'a' AS c1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query schema +struct +-- !query output +1 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('a'), ('A') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v DEFAULT COLLATION SR_AI_CI AS SELECT c1 FROM t WHERE 'ć' = 'č' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COUNT(*) FROM v +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query schema +struct +-- !query output +2 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('ć'), ('č') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT DISTINCT COLLATION(c1) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.sr_AI + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'c' +-- !query schema +struct +-- !query output +2 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('ć'), ('č') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT DISTINCT COLLATION(c1) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.sr_AI + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'c' +-- !query schema +struct +-- !query output +2 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v DEFAULT COLLATION UTF8_LCASE +AS SELECT 'a' AS c1, (SELECT (SELECT CASE 'a' = 'A' WHEN TRUE THEN 'a' ELSE 'b' END) WHERE (SELECT 'b' WHERE 'c' = 'C') = 'B') AS c2, 'c' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) FROM v WHERE c2 = 'a' +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) FROM v WHERE c2 = 'b' +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +a a c + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING, c2 STRING COLLATE UTF8_LCASE) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('a', 'a'), ('A', 'A'), ('b', 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE VIEW v DEFAULT COLLATION sr_ci_ai AS SELECT *, 'ć' AS c3 FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT DISTINCT COLLATION(c1) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.UTF8_BINARY + + +-- !query +SELECT DISTINCT COLLATION(c2) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.UTF8_LCASE + + +-- !query +SELECT DISTINCT COLLATION(c3) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.sr_CI_AI + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) FROM v WHERE c2 = 'a' +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT COUNT(*) FROM v WHERE c3 = 'Č' +-- !query schema +struct +-- !query output +3 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER VIEW v AS SELECT 'a' AS c1, 'b' AS c2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COLLATION(c1) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.UTF8_LCASE + + +-- !query +SELECT COLLATION(c2) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.UTF8_LCASE + + +-- !query +ALTER VIEW v AS SELECT 'c' AS c3 WHERE 'a' = 'A' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COLLATION(c3) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.UTF8_LCASE + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE, c2 STRING, c3 INT) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('a', 'b', 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW v DEFAULT COLLATION sr_AI_CI AS SELECT 'a' AS c1 +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER VIEW v AS + SELECT *, 'c' AS c4, (SELECT (SELECT CASE 'š' = 'S' WHEN TRUE THEN 'd' ELSE 'b' END)) AS c5 + FROM t + WHERE c1 = 'A' AND 'ć' = 'Č' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COLLATION(c4) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.sr_CI_AI + + +-- !query +SELECT COLLATION(c5) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.sr_CI_AI + + +-- !query +SELECT c5 FROM v +-- !query schema +struct +-- !query output +d + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION UNICODE AS SELECT 'a' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT * FROM v +-- !query schema +struct<'a' collate UNICODE:string collate UNICODE> +-- !query output +a + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 'a' AS c1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query schema +struct +-- !query output +1 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('a'), ('A') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION SR_AI_CI AS SELECT c1 FROM t WHERE 'ć' = 'č' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COUNT(*) FROM v +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query schema +struct +-- !query output +2 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('ć'), ('č') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT DISTINCT COLLATION(c1) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.sr_AI + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'c' +-- !query schema +struct +-- !query output +2 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('ć'), ('č') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION UNICODE AS SELECT CAST(c1 AS STRING COLLATE SR_AI) FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT DISTINCT COLLATION(c1) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.sr_AI + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'c' +-- !query schema +struct +-- !query output +2 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION UTF8_LCASE +AS SELECT 'a' AS c1, (SELECT (SELECT CASE 'a' = 'A' WHEN TRUE THEN 'a' ELSE 'b' END) WHERE (SELECT 'b' WHERE 'c' = 'C') = 'B') AS c2, 'c' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) FROM v WHERE c2 = 'a' +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) FROM v WHERE c2 = 'b' +-- !query schema +struct +-- !query output +0 + + +-- !query +SELECT * FROM v +-- !query schema +struct +-- !query output +a a c + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING, c2 STRING COLLATE UTF8_LCASE) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('a', 'a'), ('A', 'A'), ('b', 'b') +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW v DEFAULT COLLATION sr_ci_ai AS SELECT *, 'ć' AS c3 FROM t +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT DISTINCT COLLATION(c1) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.UTF8_BINARY + + +-- !query +SELECT DISTINCT COLLATION(c2) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.UTF8_LCASE + + +-- !query +SELECT DISTINCT COLLATION(c3) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.sr_CI_AI + + +-- !query +SELECT COUNT(*) FROM v WHERE c1 = 'A' +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT COUNT(*) FROM v WHERE c2 = 'a' +-- !query schema +struct +-- !query output +2 + + +-- !query +SELECT COUNT(*) FROM v WHERE c3 = 'Č' +-- !query schema +struct +-- !query output +3 + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION UTF8_LCASE AS SELECT 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER VIEW v AS SELECT 'a' AS c1, 'b' AS c2 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COLLATION(c1) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.UTF8_LCASE + + +-- !query +SELECT COLLATION(c2) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.UTF8_LCASE + + +-- !query +ALTER VIEW v AS SELECT 'c' AS c3 WHERE 'a' = 'A' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COLLATION(c3) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.UTF8_LCASE + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TABLE t (c1 STRING COLLATE UTF8_LCASE, c2 STRING, c3 INT) +-- !query schema +struct<> +-- !query output + + + +-- !query +INSERT INTO t VALUES ('a', 'b', 1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE TEMPORARY VIEW v DEFAULT COLLATION sr_AI_CI AS SELECT 'a' AS c1 +-- !query schema +struct<> +-- !query output + + + +-- !query +ALTER VIEW v AS + SELECT *, 'c' AS c4, (SELECT (SELECT CASE 'š' = 'S' WHEN TRUE THEN 'd' ELSE 'b' END)) AS c5 + FROM t + WHERE c1 = 'A' AND 'ć' = 'Č' +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT COLLATION(c4) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.sr_CI_AI + + +-- !query +SELECT COLLATION(c5) FROM v +-- !query schema +struct +-- !query output +SYSTEM.BUILTIN.sr_CI_AI + + +-- !query +SELECT c5 FROM v +-- !query schema +struct +-- !query output +d + + +-- !query +DROP VIEW IF EXISTS v +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/window.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/window.sql.out new file mode 100644 index 000000000000..ce88fb57f8aa --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/window.sql.out @@ -0,0 +1,1510 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), +(null, null, null, null, null, null), +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) +AS testData(val, val_long, val_double, val_date, val_timestamp, cate) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW basic_pays AS SELECT * FROM VALUES +('Diane Murphy','Accounting',8435), +('Mary Patterson','Accounting',9998), +('Jeff Firrelli','Accounting',8992), +('William Patterson','Accounting',8870), +('Gerard Bondur','Accounting',11472), +('Anthony Bow','Accounting',6627), +('Leslie Jennings','IT',8113), +('Leslie Thompson','IT',5186), +('Julie Firrelli','Sales',9181), +('Steve Patterson','Sales',9441), +('Foon Yue Tseng','Sales',6660), +('George Vanauf','Sales',10563), +('Loui Bondur','SCM',10449), +('Gerard Hernandez','SCM',6949), +('Pamela Castillo','SCM',11303), +('Larry Bott','SCM',11798), +('Barry Jones','SCM',10586) +AS basic_pays(employee_name, department, salary) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE OR REPLACE TEMPORARY VIEW test_ignore_null AS SELECT * FROM VALUES +('a', 0, null), +('a', 1, 'x'), +('b', 2, null), +('c', 3, null), +('a', 4, 'y'), +('b', 5, null), +('a', 6, 'z'), +('a', 7, 'v'), +('a', 8, null) +AS test_ignore_null(content, id, v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val ROWS CURRENT ROW) FROM testData +ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL 0 +3 NULL 1 +NULL a 0 +1 a 1 +1 a 1 +2 a 1 +1 b 1 +2 b 1 +3 b 1 + + +-- !query +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val +ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL 3 +3 NULL 3 +NULL a 1 +1 a 2 +1 a 4 +2 a 4 +1 b 3 +2 b 6 +3 b 6 + + +-- !query +SELECT val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long +ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "expectedType" : "\"INT\"", + "exprType" : "\"BIGINT\"", + "location" : "upper", + "sqlExpr" : "\"ROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 42, + "stopIndex" : 128, + "fragment" : "(PARTITION BY cate ORDER BY val_long\nROWS BETWEEN CURRENT ROW AND 2147483648 FOLLOWING)" + } ] +} + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val RANGE 1 PRECEDING) FROM testData +ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL 0 +3 NULL 1 +NULL a 0 +1 a 2 +1 a 2 +2 a 3 +1 b 1 +2 b 2 +3 b 2 + + +-- !query +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL NULL +3 NULL 3 +NULL a NULL +1 a 4 +1 a 4 +2 a 2 +1 b 3 +2 b 5 +3 b 3 + + +-- !query +SELECT val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long +RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long +-- !query schema +struct +-- !query output +NULL NULL NULL +1 NULL 1 +1 a 4 +1 a 4 +2 a 2147483652 +2147483650 a 2147483650 +NULL b NULL +3 b 2147483653 +2147483650 b 2147483650 + + +-- !query +SELECT val_double, cate, sum(val_double) OVER(PARTITION BY cate ORDER BY val_double +RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING) FROM testData ORDER BY cate, val_double +-- !query schema +struct +-- !query output +NULL NULL NULL +1.0 NULL 1.0 +1.0 a 4.5 +1.0 a 4.5 +2.5 a 2.5 +100.001 a 100.001 +1.0 b 4.3 +3.3 b 3.3 +100.001 b 100.001 + + +-- !query +SELECT val_date, cate, max(val_date) OVER(PARTITION BY cate ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING) FROM testData ORDER BY cate, val_date +-- !query schema +struct +-- !query output +NULL NULL NULL +2017-08-01 NULL 2017-08-01 +2017-08-01 a 2017-08-02 +2017-08-01 a 2017-08-02 +2017-08-02 a 2017-08-02 +2020-12-31 a 2020-12-31 +2017-08-01 b 2017-08-03 +2017-08-03 b 2017-08-03 +2020-12-31 b 2020-12-31 + + +-- !query +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData +ORDER BY cate, val_timestamp +-- !query schema +struct +-- !query output +NULL NULL NULL +2017-07-31 17:00:00 NULL 1.5015456E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-08-05 23:13:20 a 1.502E9 +2020-12-30 16:00:00 a 1.6093728E9 +2017-07-31 17:00:00 b 1.5022728E9 +2017-08-17 13:00:00 b 1.503E9 +2020-12-30 16:00:00 b 1.6093728E9 + + +-- !query +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) +RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData +ORDER BY cate, to_timestamp_ntz(val_timestamp) +-- !query schema +struct +-- !query output +NULL NULL NULL +2017-07-31 17:00:00 NULL 1.5015456E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-08-05 23:13:20 a 1.502E9 +2020-12-30 16:00:00 a 1.6093728E9 +2017-07-31 17:00:00 b 1.5022728E9 +2017-08-17 13:00:00 b 1.503E9 +2020-12-30 16:00:00 b 1.6093728E9 + + +-- !query +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval '1-1' year to month FOLLOWING) FROM testData +ORDER BY cate, val_timestamp +-- !query schema +struct +-- !query output +NULL NULL NULL +2017-07-31 17:00:00 NULL 1.5015456E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-08-05 23:13:20 a 1.502E9 +2020-12-30 16:00:00 a 1.6093728E9 +2017-07-31 17:00:00 b 1.5022728E9 +2017-08-17 13:00:00 b 1.503E9 +2020-12-30 16:00:00 b 1.6093728E9 + + +-- !query +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) +RANGE BETWEEN CURRENT ROW AND interval '1-1' year to month FOLLOWING) FROM testData +ORDER BY cate, to_timestamp_ntz(val_timestamp) +-- !query schema +struct +-- !query output +NULL NULL NULL +2017-07-31 17:00:00 NULL 1.5015456E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-08-05 23:13:20 a 1.502E9 +2020-12-30 16:00:00 a 1.6093728E9 +2017-07-31 17:00:00 b 1.5022728E9 +2017-08-17 13:00:00 b 1.503E9 +2020-12-30 16:00:00 b 1.6093728E9 + + +-- !query +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING) FROM testData +ORDER BY cate, val_timestamp +-- !query schema +struct +-- !query output +NULL NULL NULL +2017-07-31 17:00:00 NULL 1.5015456E9 +2017-07-31 17:00:00 a 1.5015456E9 +2017-07-31 17:00:00 a 1.5015456E9 +2017-08-05 23:13:20 a 1.502E9 +2020-12-30 16:00:00 a 1.6093728E9 +2017-07-31 17:00:00 b 1.5015456E9 +2017-08-17 13:00:00 b 1.503E9 +2020-12-30 16:00:00 b 1.6093728E9 + + +-- !query +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY to_timestamp_ntz(val_timestamp) +RANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING) FROM testData +ORDER BY cate, to_timestamp_ntz(val_timestamp) +-- !query schema +struct +-- !query output +NULL NULL NULL +2017-07-31 17:00:00 NULL 1.5015456E9 +2017-07-31 17:00:00 a 1.5015456E9 +2017-07-31 17:00:00 a 1.5015456E9 +2017-08-05 23:13:20 a 1.502E9 +2020-12-30 16:00:00 a 1.6093728E9 +2017-07-31 17:00:00 b 1.5015456E9 +2017-08-17 13:00:00 b 1.503E9 +2020-12-30 16:00:00 b 1.6093728E9 + + +-- !query +SELECT val_date, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND interval '1-1' year to month FOLLOWING) FROM testData +ORDER BY cate, val_date +-- !query schema +struct +-- !query output +NULL NULL NULL +2017-08-01 NULL 1.5015456E9 +2017-08-01 a 1.5016970666666667E9 +2017-08-01 a 1.5016970666666667E9 +2017-08-02 a 1.502E9 +2020-12-31 a 1.6093728E9 +2017-08-01 b 1.5022728E9 +2017-08-03 b 1.503E9 +2020-12-31 b 1.6093728E9 + + +-- !query +SELECT val_date, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING) FROM testData +ORDER BY cate, val_date +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpecType" : "\"DATE\"", + "sqlExpr" : "\"(PARTITION BY cate ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND INTERVAL '1 02:03:04.001' DAY TO SECOND FOLLOWING)\"", + "valueBoundaryType" : "\"INTERVAL DAY TO SECOND\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 47, + "stopIndex" : 160, + "fragment" : "(PARTITION BY cate ORDER BY val_date\nRANGE BETWEEN CURRENT ROW AND interval '1 2:3:4.001' day to second FOLLOWING)" + } ] +} + + +-- !query +SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val DESC +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL NULL +3 NULL 3 +NULL a NULL +1 a 2 +1 a 2 +2 a 4 +1 b 1 +2 b 3 +3 b 5 + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate +ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_INVALID_BOUND", + "sqlState" : "42K09", + "messageParameters" : { + "lower" : "\"UNBOUNDED FOLLOWING\"", + "sqlExpr" : "\"ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING\"", + "upper" : "\"1\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 101, + "fragment" : "(PARTITION BY cate\nROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_WITHOUT_ORDER", + "sqlState" : "42K09", + "messageParameters" : { + "sqlExpr" : "\"(PARTITION BY cate RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 94, + "fragment" : "(PARTITION BY cate\nRANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val, cate +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_MULTI_ORDER", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpec" : "val#x ASC NULLS FIRST,cate#x ASC NULLS FIRST", + "sqlExpr" : "\"(PARTITION BY cate ORDER BY val ASC NULLS FIRST, cate ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 113, + "fragment" : "(PARTITION BY cate ORDER BY val, cate\nRANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY current_timestamp +RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpecType" : "\"TIMESTAMP\"", + "sqlExpr" : "\"(PARTITION BY cate ORDER BY current_timestamp() ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)\"", + "valueBoundaryType" : "\"INT\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 121, + "fragment" : "(PARTITION BY cate ORDER BY current_timestamp\nRANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)" + } ] +} + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY cate, val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_WRONG_COMPARISON", + "sqlState" : "42K09", + "messageParameters" : { + "comparison" : "less than or equal", + "sqlExpr" : "\"RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 34, + "stopIndex" : 107, + "fragment" : "(PARTITION BY cate ORDER BY val\nRANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING)" + } ] +} + + +-- !query +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val +RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_0064", + "messageParameters" : { + "msg" : "Frame bound value must be a literal." + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 96, + "stopIndex" : 117, + "fragment" : "current_date PRECEDING" + } ] +} + + +-- !query +SELECT val, cate, +max(val) OVER w AS max, +min(val) OVER w AS min, +min(val) OVER w AS min, +count(val) OVER w AS count, +sum(val) OVER w AS sum, +avg(val) OVER w AS avg, +stddev(val) OVER w AS stddev, +first_value(val) OVER w AS first_value, +first_value(val, true) OVER w AS first_value_ignore_null, +first_value(val, false) OVER w AS first_value_contain_null, +any_value(val) OVER w AS any_value, +any_value(val, true) OVER w AS any_value_ignore_null, +any_value(val, false) OVER w AS any_value_contain_null, +last_value(val) OVER w AS last_value, +last_value(val, true) OVER w AS last_value_ignore_null, +last_value(val, false) OVER w AS last_value_contain_null, +rank() OVER w AS rank, +dense_rank() OVER w AS dense_rank, +cume_dist() OVER w AS cume_dist, +percent_rank() OVER w AS percent_rank, +ntile(2) OVER w AS ntile, +row_number() OVER w AS row_number, +var_pop(val) OVER w AS var_pop, +var_samp(val) OVER w AS var_samp, +approx_count_distinct(val) OVER w AS approx_count_distinct, +covar_pop(val, val_long) OVER w AS covar_pop, +corr(val, val_long) OVER w AS corr, +stddev_samp(val) OVER w AS stddev_samp, +stddev_pop(val) OVER w AS stddev_pop, +collect_list(val) OVER w AS collect_list, +collect_set(val) OVER w AS collect_set, +skewness(val_double) OVER w AS skewness, +kurtosis(val_double) OVER w AS kurtosis +FROM testData +WINDOW w AS (PARTITION BY cate ORDER BY val) +ORDER BY cate, val +-- !query schema +struct<> +-- !query output +org.apache.spark.SparkArithmeticException +{ + "errorClass" : "DIVIDE_BY_ZERO", + "sqlState" : "22012", + "messageParameters" : { + "config" : "\"spark.sql.ansi.enabled\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1016, + "stopIndex" : 1041, + "fragment" : "corr(val, val_long) OVER w" + } ] +} + + +-- !query +SELECT val, cate, avg(null) OVER(PARTITION BY cate ORDER BY val) FROM testData ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL NULL +3 NULL NULL +NULL a NULL +1 a NULL +1 a NULL +2 a NULL +1 b NULL +2 b NULL +3 b NULL + + +-- !query +SELECT val, cate, row_number() OVER(PARTITION BY cate) FROM testData ORDER BY cate, val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1037", + "messageParameters" : { + "wf" : "row_number()" + } +} + + +-- !query +SELECT val, cate, sum(val) OVER(), avg(val) OVER() FROM testData ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL 13 1.8571428571428572 +3 NULL 13 1.8571428571428572 +NULL a 13 1.8571428571428572 +1 a 13 1.8571428571428572 +1 a 13 1.8571428571428572 +2 a 13 1.8571428571428572 +1 b 13 1.8571428571428572 +2 b 13 1.8571428571428572 +3 b 13 1.8571428571428572 + + +-- !query +SELECT val, cate, +first_value(false) OVER w AS first_value, +first_value(true, true) OVER w AS first_value_ignore_null, +first_value(false, false) OVER w AS first_value_contain_null, +any_value(false) OVER w AS any_value, +any_value(true, true) OVER w AS any_value_ignore_null, +any_value(false, false) OVER w AS any_value_contain_null, +last_value(false) OVER w AS last_value, +last_value(true, true) OVER w AS last_value_ignore_null, +last_value(false, false) OVER w AS last_value_contain_null +FROM testData +WINDOW w AS () +ORDER BY cate, val +-- !query schema +struct +-- !query output +NULL NULL false true false false true false false true false +3 NULL false true false false true false false true false +NULL a false true false false true false false true false +1 a false true false false true false false true false +1 a false true false false true false false true false +2 a false true false false true false false true false +1 b false true false false true false false true false +2 b false true false false true false false true false +3 b false true false false true false false true false + + +-- !query +SELECT cate, sum(val) OVER (w) +FROM testData +WHERE val is not null +WINDOW w AS (PARTITION BY cate ORDER BY val) +-- !query schema +struct +-- !query output +NULL 3 +a 2 +a 2 +a 4 +b 1 +b 3 +b 6 + + +-- !query +SELECT val, cate, +count(val) FILTER (WHERE val > 1) OVER(PARTITION BY cate) +FROM testData ORDER BY cate, val +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "_LEGACY_ERROR_TEMP_1030" +} + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC) +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Larry Bott Larry Bott NULL +Gerard Bondur 11472 Larry Bott Larry Bott Gerard Bondur +Pamela Castillo 11303 Larry Bott Larry Bott Gerard Bondur +Barry Jones 10586 Larry Bott Larry Bott Gerard Bondur +George Vanauf 10563 Larry Bott Larry Bott Gerard Bondur +Loui Bondur 10449 Larry Bott Larry Bott Gerard Bondur +Mary Patterson 9998 Larry Bott Larry Bott Gerard Bondur +Steve Patterson 9441 Larry Bott Larry Bott Gerard Bondur +Julie Firrelli 9181 Larry Bott Larry Bott Gerard Bondur +Jeff Firrelli 8992 Larry Bott Larry Bott Gerard Bondur +William Patterson 8870 Larry Bott Larry Bott Gerard Bondur +Diane Murphy 8435 Larry Bott Larry Bott Gerard Bondur +Leslie Jennings 8113 Larry Bott Larry Bott Gerard Bondur +Gerard Hernandez 6949 Larry Bott Larry Bott Gerard Bondur +Foon Yue Tseng 6660 Larry Bott Larry Bott Gerard Bondur +Anthony Bow 6627 Larry Bott Larry Bott Gerard Bondur +Leslie Thompson 5186 Larry Bott Larry Bott Gerard Bondur + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Larry Bott Larry Bott NULL +Gerard Bondur 11472 Larry Bott Larry Bott Gerard Bondur +Pamela Castillo 11303 Larry Bott Larry Bott Gerard Bondur +Barry Jones 10586 Larry Bott Larry Bott Gerard Bondur +George Vanauf 10563 Larry Bott Larry Bott Gerard Bondur +Loui Bondur 10449 Larry Bott Larry Bott Gerard Bondur +Mary Patterson 9998 Larry Bott Larry Bott Gerard Bondur +Steve Patterson 9441 Larry Bott Larry Bott Gerard Bondur +Julie Firrelli 9181 Larry Bott Larry Bott Gerard Bondur +Jeff Firrelli 8992 Larry Bott Larry Bott Gerard Bondur +William Patterson 8870 Larry Bott Larry Bott Gerard Bondur +Diane Murphy 8435 Larry Bott Larry Bott Gerard Bondur +Leslie Jennings 8113 Larry Bott Larry Bott Gerard Bondur +Gerard Hernandez 6949 Larry Bott Larry Bott Gerard Bondur +Foon Yue Tseng 6660 Larry Bott Larry Bott Gerard Bondur +Anthony Bow 6627 Larry Bott Larry Bott Gerard Bondur +Leslie Thompson 5186 Larry Bott Larry Bott Gerard Bondur + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Larry Bott Larry Bott NULL +Gerard Bondur 11472 Larry Bott Larry Bott Gerard Bondur +Pamela Castillo 11303 Larry Bott Larry Bott Gerard Bondur +Barry Jones 10586 Larry Bott Larry Bott Gerard Bondur +George Vanauf 10563 Larry Bott Larry Bott Gerard Bondur +Loui Bondur 10449 Larry Bott Larry Bott Gerard Bondur +Mary Patterson 9998 Larry Bott Larry Bott Gerard Bondur +Steve Patterson 9441 Larry Bott Larry Bott Gerard Bondur +Julie Firrelli 9181 Larry Bott Larry Bott Gerard Bondur +Jeff Firrelli 8992 Larry Bott Larry Bott Gerard Bondur +William Patterson 8870 Larry Bott Larry Bott Gerard Bondur +Diane Murphy 8435 Larry Bott Larry Bott Gerard Bondur +Leslie Jennings 8113 Larry Bott Larry Bott Gerard Bondur +Gerard Hernandez 6949 Larry Bott Larry Bott Gerard Bondur +Foon Yue Tseng 6660 Larry Bott Larry Bott Gerard Bondur +Anthony Bow 6627 Larry Bott Larry Bott Gerard Bondur +Leslie Thompson 5186 Larry Bott Larry Bott Gerard Bondur + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary RANGE BETWEEN 2000 PRECEDING AND 1000 FOLLOWING) +ORDER BY salary +-- !query schema +struct +-- !query output +Leslie Thompson 5186 Leslie Thompson Leslie Thompson NULL +Anthony Bow 6627 Leslie Thompson Leslie Thompson Anthony Bow +Foon Yue Tseng 6660 Leslie Thompson Leslie Thompson Anthony Bow +Gerard Hernandez 6949 Leslie Thompson Leslie Thompson Anthony Bow +Leslie Jennings 8113 Anthony Bow Anthony Bow Foon Yue Tseng +Diane Murphy 8435 Anthony Bow Anthony Bow Foon Yue Tseng +William Patterson 8870 Gerard Hernandez Gerard Hernandez Leslie Jennings +Jeff Firrelli 8992 Leslie Jennings Leslie Jennings Diane Murphy +Julie Firrelli 9181 Leslie Jennings Leslie Jennings Diane Murphy +Steve Patterson 9441 Leslie Jennings Leslie Jennings Diane Murphy +Mary Patterson 9998 Leslie Jennings Leslie Jennings Diane Murphy +Loui Bondur 10449 William Patterson William Patterson Jeff Firrelli +George Vanauf 10563 William Patterson William Patterson Jeff Firrelli +Barry Jones 10586 William Patterson William Patterson Jeff Firrelli +Pamela Castillo 11303 Steve Patterson Steve Patterson Mary Patterson +Gerard Bondur 11472 Mary Patterson Mary Patterson Loui Bondur +Larry Bott 11798 Mary Patterson Mary Patterson Loui Bondur + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING) +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Larry Bott Larry Bott Gerard Bondur +Gerard Bondur 11472 Larry Bott Larry Bott Gerard Bondur +Pamela Castillo 11303 Larry Bott Larry Bott Gerard Bondur +Barry Jones 10586 Gerard Bondur Gerard Bondur Pamela Castillo +George Vanauf 10563 Pamela Castillo Pamela Castillo Barry Jones +Loui Bondur 10449 Barry Jones Barry Jones George Vanauf +Mary Patterson 9998 George Vanauf George Vanauf Loui Bondur +Steve Patterson 9441 Loui Bondur Loui Bondur Mary Patterson +Julie Firrelli 9181 Mary Patterson Mary Patterson Steve Patterson +Jeff Firrelli 8992 Steve Patterson Steve Patterson Julie Firrelli +William Patterson 8870 Julie Firrelli Julie Firrelli Jeff Firrelli +Diane Murphy 8435 Jeff Firrelli Jeff Firrelli William Patterson +Leslie Jennings 8113 William Patterson William Patterson Diane Murphy +Gerard Hernandez 6949 Diane Murphy Diane Murphy Leslie Jennings +Foon Yue Tseng 6660 Leslie Jennings Leslie Jennings Gerard Hernandez +Anthony Bow 6627 Gerard Hernandez Gerard Hernandez Foon Yue Tseng +Leslie Thompson 5186 Foon Yue Tseng Foon Yue Tseng Anthony Bow + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Larry Bott Larry Bott Gerard Bondur +Gerard Bondur 11472 Gerard Bondur Gerard Bondur Pamela Castillo +Pamela Castillo 11303 Pamela Castillo Pamela Castillo Barry Jones +Barry Jones 10586 Barry Jones Barry Jones George Vanauf +George Vanauf 10563 George Vanauf George Vanauf Loui Bondur +Loui Bondur 10449 Loui Bondur Loui Bondur Mary Patterson +Mary Patterson 9998 Mary Patterson Mary Patterson Steve Patterson +Steve Patterson 9441 Steve Patterson Steve Patterson Julie Firrelli +Julie Firrelli 9181 Julie Firrelli Julie Firrelli Jeff Firrelli +Jeff Firrelli 8992 Jeff Firrelli Jeff Firrelli William Patterson +William Patterson 8870 William Patterson William Patterson Diane Murphy +Diane Murphy 8435 Diane Murphy Diane Murphy Leslie Jennings +Leslie Jennings 8113 Leslie Jennings Leslie Jennings Gerard Hernandez +Gerard Hernandez 6949 Gerard Hernandez Gerard Hernandez Foon Yue Tseng +Foon Yue Tseng 6660 Foon Yue Tseng Foon Yue Tseng Anthony Bow +Anthony Bow 6627 Anthony Bow Anthony Bow Leslie Thompson +Leslie Thompson 5186 Leslie Thompson Leslie Thompson NULL + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Larry Bott Larry Bott Gerard Bondur +Gerard Bondur 11472 Larry Bott Larry Bott Gerard Bondur +Pamela Castillo 11303 Larry Bott Larry Bott Gerard Bondur +Barry Jones 10586 Larry Bott Larry Bott Gerard Bondur +George Vanauf 10563 Larry Bott Larry Bott Gerard Bondur +Loui Bondur 10449 Larry Bott Larry Bott Gerard Bondur +Mary Patterson 9998 Larry Bott Larry Bott Gerard Bondur +Steve Patterson 9441 Larry Bott Larry Bott Gerard Bondur +Julie Firrelli 9181 Larry Bott Larry Bott Gerard Bondur +Jeff Firrelli 8992 Larry Bott Larry Bott Gerard Bondur +William Patterson 8870 Larry Bott Larry Bott Gerard Bondur +Diane Murphy 8435 Larry Bott Larry Bott Gerard Bondur +Leslie Jennings 8113 Larry Bott Larry Bott Gerard Bondur +Gerard Hernandez 6949 Larry Bott Larry Bott Gerard Bondur +Foon Yue Tseng 6660 Larry Bott Larry Bott Gerard Bondur +Anthony Bow 6627 Larry Bott Larry Bott Gerard Bondur +Leslie Thompson 5186 Larry Bott Larry Bott Gerard Bondur + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Larry Bott Larry Bott Gerard Bondur +Gerard Bondur 11472 Larry Bott Larry Bott Gerard Bondur +Pamela Castillo 11303 Larry Bott Larry Bott Gerard Bondur +Barry Jones 10586 Larry Bott Larry Bott Gerard Bondur +George Vanauf 10563 Larry Bott Larry Bott Gerard Bondur +Loui Bondur 10449 Larry Bott Larry Bott Gerard Bondur +Mary Patterson 9998 Larry Bott Larry Bott Gerard Bondur +Steve Patterson 9441 Larry Bott Larry Bott Gerard Bondur +Julie Firrelli 9181 Larry Bott Larry Bott Gerard Bondur +Jeff Firrelli 8992 Larry Bott Larry Bott Gerard Bondur +William Patterson 8870 Larry Bott Larry Bott Gerard Bondur +Diane Murphy 8435 Larry Bott Larry Bott Gerard Bondur +Leslie Jennings 8113 Larry Bott Larry Bott Gerard Bondur +Gerard Hernandez 6949 Larry Bott Larry Bott Gerard Bondur +Foon Yue Tseng 6660 Larry Bott Larry Bott Gerard Bondur +Anthony Bow 6627 Larry Bott Larry Bott Gerard Bondur +Leslie Thompson 5186 Larry Bott Larry Bott Gerard Bondur + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY salary DESC +-- !query schema +struct +-- !query output +Larry Bott 11798 Larry Bott Larry Bott Gerard Bondur +Gerard Bondur 11472 Larry Bott Larry Bott Gerard Bondur +Pamela Castillo 11303 Larry Bott Larry Bott Gerard Bondur +Barry Jones 10586 Larry Bott Larry Bott Gerard Bondur +George Vanauf 10563 Larry Bott Larry Bott Gerard Bondur +Loui Bondur 10449 Larry Bott Larry Bott Gerard Bondur +Mary Patterson 9998 Larry Bott Larry Bott Gerard Bondur +Steve Patterson 9441 Larry Bott Larry Bott Gerard Bondur +Julie Firrelli 9181 Larry Bott Larry Bott Gerard Bondur +Jeff Firrelli 8992 Larry Bott Larry Bott Gerard Bondur +William Patterson 8870 Larry Bott Larry Bott Gerard Bondur +Diane Murphy 8435 Larry Bott Larry Bott Gerard Bondur +Leslie Jennings 8113 Larry Bott Larry Bott Gerard Bondur +Gerard Hernandez 6949 Larry Bott Larry Bott Gerard Bondur +Foon Yue Tseng 6660 Larry Bott Larry Bott Gerard Bondur +Anthony Bow 6627 Larry Bott Larry Bott Gerard Bondur +Leslie Thompson 5186 Larry Bott Larry Bott Gerard Bondur + + +-- !query +SELECT + employee_name, + department, + salary, + FIRST_VALUE(employee_name) OVER w highest_salary, + NTH_VALUE(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW w AS ( + PARTITION BY department + ORDER BY salary DESC + RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING +) +ORDER BY department +-- !query schema +struct +-- !query output +Gerard Bondur Accounting 11472 Gerard Bondur Mary Patterson +Mary Patterson Accounting 9998 Gerard Bondur Mary Patterson +Jeff Firrelli Accounting 8992 Gerard Bondur Mary Patterson +William Patterson Accounting 8870 Gerard Bondur Mary Patterson +Diane Murphy Accounting 8435 Gerard Bondur Mary Patterson +Anthony Bow Accounting 6627 Gerard Bondur Mary Patterson +Leslie Jennings IT 8113 Leslie Jennings Leslie Thompson +Leslie Thompson IT 5186 Leslie Jennings Leslie Thompson +Larry Bott SCM 11798 Larry Bott Pamela Castillo +Pamela Castillo SCM 11303 Larry Bott Pamela Castillo +Barry Jones SCM 10586 Larry Bott Pamela Castillo +Loui Bondur SCM 10449 Larry Bott Pamela Castillo +Gerard Hernandez SCM 6949 Larry Bott Pamela Castillo +George Vanauf Sales 10563 George Vanauf Steve Patterson +Steve Patterson Sales 9441 George Vanauf Steve Patterson +Julie Firrelli Sales 9181 George Vanauf Steve Patterson +Foon Yue Tseng Sales 6660 George Vanauf Steve Patterson + + +-- !query +SELECT + employee_name, + salary, + first_value(employee_name) OVER w highest_salary, + any_value(employee_name) OVER w highest_salary, + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +WINDOW + w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING), + w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 2 FOLLOWING) +ORDER BY salary DESC +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "INVALID_SQL_SYNTAX.REPETITIVE_WINDOW_DEFINITION", + "sqlState" : "42000", + "messageParameters" : { + "windowName" : "`w`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 226, + "stopIndex" : 394, + "fragment" : "WINDOW\n w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING),\n w AS (ORDER BY salary DESC ROWS BETWEEN UNBOUNDED PRECEDING AND 2 FOLLOWING)" + } ] +} + + +-- !query +SELECT + content, + id, + v, + lead(v, 0) IGNORE NULLS OVER w lead_0, + lead(v, 1) IGNORE NULLS OVER w lead_1, + lead(v, 2) IGNORE NULLS OVER w lead_2, + lead(v, 3) IGNORE NULLS OVER w lead_3, + lag(v, 0) IGNORE NULLS OVER w lag_0, + lag(v, 1) IGNORE NULLS OVER w lag_1, + lag(v, 2) IGNORE NULLS OVER w lag_2, + lag(v, 3) IGNORE NULLS OVER w lag_3, + lag(v, +3) IGNORE NULLS OVER w lag_plus_3, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id) +ORDER BY id +-- !query schema +struct +-- !query output +a 0 NULL NULL x y z NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL +a 1 x x y z v x NULL NULL NULL NULL x NULL NULL x x x +b 2 NULL NULL y z v NULL x NULL NULL NULL x NULL NULL x x x +c 3 NULL NULL y z v NULL x NULL NULL NULL x NULL NULL x x x +a 4 y y z v NULL y x NULL NULL NULL x y NULL x x y +b 5 NULL NULL z v NULL NULL y x NULL NULL x y NULL x x y +a 6 z z v NULL NULL z y x NULL NULL x y z x x z +a 7 v v NULL NULL NULL v z y x x x y z x x v +a 8 NULL NULL NULL NULL NULL NULL v z y y x y z x x v + + +-- !query +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY id +-- !query schema +struct +-- !query output +a 0 NULL NULL NULL NULL NULL NULL NULL +a 1 x x NULL NULL x x x +b 2 NULL x NULL NULL x x x +c 3 NULL x NULL NULL x x x +a 4 y x y NULL x x y +b 5 NULL x y NULL x x y +a 6 z x y z x x z +a 7 v x y z x x v +a 8 NULL x y z x x v + + +-- !query +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) +ORDER BY id +-- !query schema +struct +-- !query output +a 0 NULL NULL NULL NULL NULL NULL NULL +a 1 x x NULL NULL x x x +b 2 NULL x NULL NULL x x x +c 3 NULL x NULL NULL x x x +a 4 y x y NULL x x y +b 5 NULL x y NULL x x y +a 6 z x y z x x z +a 7 v x y z x x v +a 8 NULL x y z x x v + + +-- !query +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN 2 PRECEDING AND 2 FOLLOWING) +ORDER BY id +-- !query schema +struct +-- !query output +a 0 NULL x NULL NULL x x x +a 1 x x NULL NULL x x x +b 2 NULL x y NULL x x y +c 3 NULL x y NULL x x y +a 4 y y z NULL y y z +b 5 NULL y z v y y v +a 6 z y z v y y v +a 7 v z v NULL z z v +a 8 NULL z v NULL z z v + + +-- !query +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN 2 PRECEDING AND 2 FOLLOWING) +ORDER BY id +-- !query schema +struct +-- !query output +a 0 NULL x NULL NULL x x x +a 1 x x NULL NULL x x x +b 2 NULL x y NULL x x y +c 3 NULL x y NULL x x y +a 4 y y z NULL y y z +b 5 NULL y z v y y v +a 6 z y z v y y v +a 7 v z v NULL z z v +a 8 NULL z v NULL z z v + + +-- !query +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING) +ORDER BY id +-- !query schema +struct +-- !query output +a 0 NULL x y z x x v +a 1 x x y z x x v +b 2 NULL y z v y y v +c 3 NULL y z v y y v +a 4 y y z v y y v +b 5 NULL z v NULL z z v +a 6 z z v NULL z z v +a 7 v v NULL NULL v v v +a 8 NULL NULL NULL NULL NULL NULL NULL + + +-- !query +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY id +-- !query schema +struct +-- !query output +a 0 NULL x y z x x v +a 1 x x y z x x v +b 2 NULL x y z x x v +c 3 NULL x y z x x v +a 4 y x y z x x v +b 5 NULL x y z x x v +a 6 z x y z x x v +a 7 v x y z x x v +a 8 NULL x y z x x v + + +-- !query +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) +ORDER BY id +-- !query schema +struct +-- !query output +a 0 NULL x y z x x v +a 1 x x y z x x v +b 2 NULL x y z x x v +c 3 NULL x y z x x v +a 4 y x y z x x v +b 5 NULL x y z x x v +a 6 z x y z x x v +a 7 v x y z x x v +a 8 NULL x y z x x v + + +-- !query +SELECT + content, + id, + v, + nth_value(v, 1) IGNORE NULLS OVER w nth_value_1, + nth_value(v, 2) IGNORE NULLS OVER w nth_value_2, + nth_value(v, 3) IGNORE NULLS OVER w nth_value_3, + first_value(v) IGNORE NULLS OVER w first_value, + any_value(v) IGNORE NULLS OVER w any_value, + last_value(v) IGNORE NULLS OVER w last_value +FROM + test_ignore_null +WINDOW w AS (ORDER BY id ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) +ORDER BY id +-- !query schema +struct +-- !query output +a 0 NULL x NULL NULL x x x +a 1 x x NULL NULL x x x +b 2 NULL x NULL NULL x x x +c 3 NULL x y NULL x x y +a 4 y x y NULL x x y +b 5 NULL x y z x x z +a 6 z x y z x x v +a 7 v x y z x x v +a 8 NULL x y z x x v + + +-- !query +SELECT + nth_value(employee_name, 2) OVER w second_highest_salary +FROM + basic_pays +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_WINDOW_SPECIFICATION", + "sqlState" : "42P20", + "messageParameters" : { + "docroot" : "https://spark.apache.org/docs/latest", + "windowName" : "w" + } +} + + +-- !query +SELECT + SUM(salary) OVER w sum_salary +FROM + basic_pays +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "MISSING_WINDOW_SPECIFICATION", + "sqlState" : "42P20", + "messageParameters" : { + "docroot" : "https://spark.apache.org/docs/latest", + "windowName" : "w" + } +} + + +-- !query +create or replace temp view t1 (p, o) as values (1, 1), (1, 1), (1, 2), (2, 1), (2, 1), (2, 2) +-- !query schema +struct<> +-- !query output + + + +-- !query +select * from (select *, dense_rank() over (partition by p order by o) as rnk from t1) where rnk = 1 +-- !query schema +struct +-- !query output +1 1 1 +1 1 1 +2 1 1 +2 1 1 + + +-- !query +SELECT * FROM (SELECT cate, val, rank() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r = 1 +-- !query schema +struct +-- !query output +NULL NULL 1 +a NULL 1 +b 1 1 + + +-- !query +SELECT * FROM (SELECT cate, val, rank() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r <= 2 +-- !query schema +struct +-- !query output +NULL 3 2 +NULL NULL 1 +a 1 2 +a 1 2 +a NULL 1 +b 1 1 +b 2 2 + + +-- !query +SELECT * FROM (SELECT cate, val, dense_rank() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r = 1 +-- !query schema +struct +-- !query output +NULL NULL 1 +a NULL 1 +b 1 1 + + +-- !query +SELECT * FROM (SELECT cate, val, dense_rank() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r <= 2 +-- !query schema +struct +-- !query output +NULL 3 2 +NULL NULL 1 +a 1 2 +a 1 2 +a NULL 1 +b 1 1 +b 2 2 + + +-- !query +SELECT * FROM (SELECT cate, val, row_number() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r = 1 +-- !query schema +struct +-- !query output +NULL NULL 1 +a NULL 1 +b 1 1 + + +-- !query +SELECT * FROM (SELECT cate, val, row_number() OVER(PARTITION BY cate ORDER BY val) as r FROM testData) where r <= 2 +-- !query schema +struct +-- !query output +NULL 3 2 +NULL NULL 1 +a 1 2 +a NULL 1 +b 1 1 +b 2 2 + + +-- !query +SELECT *, mean(val_double) over (partition BY val ORDER BY val_date RANGE INTERVAL '5' DAY PRECEDING) AS mean FROM testData +-- !query schema +struct +-- !query output +1 1 1.0 2017-08-01 2017-07-31 17:00:00 a 1.0 +1 2 2.5 2017-08-02 2017-08-05 23:13:20 a 1.5 +1 NULL 1.0 2017-08-01 2017-07-31 17:00:00 b 1.0 +2 2147483650 100.001 2020-12-31 2020-12-30 16:00:00 a 100.001 +2 3 3.3 2017-08-03 2017-08-17 13:00:00 b 3.3 +3 1 1.0 2017-08-01 2017-07-31 17:00:00 NULL 1.0 +3 2147483650 100.001 2020-12-31 2020-12-30 16:00:00 b 100.001 +NULL 1 1.0 2017-08-01 2017-07-31 17:00:00 a 1.0 +NULL NULL NULL NULL NULL NULL NULL + + +-- !query +SELECT *, mean(val_double) over (partition BY val ORDER BY val_date RANGE INTERVAL '1 2:3:4.001' DAY TO SECOND PRECEDING) AS mean FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.RANGE_FRAME_INVALID_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "orderSpecType" : "\"DATE\"", + "sqlExpr" : "\"(PARTITION BY val ORDER BY val_date ASC NULLS FIRST RANGE BETWEEN INTERVAL '1 02:03:04.001' DAY TO SECOND PRECEDING AND CURRENT ROW)\"", + "valueBoundaryType" : "\"INTERVAL DAY TO SECOND\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 33, + "stopIndex" : 121, + "fragment" : "(partition BY val ORDER BY val_date RANGE INTERVAL '1 2:3:4.001' DAY TO SECOND PRECEDING)" + } ] +} + + +-- !query +SELECT *, mean(val_double) over (partition BY val ORDER BY val_date RANGE DATE '2024-01-01' FOLLOWING) AS mean FROM testData +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "expectedType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")", + "exprType" : "\"DATE\"", + "location" : "lower", + "sqlExpr" : "\"RANGE BETWEEN DATE '2024-01-01' FOLLOWING AND CURRENT ROW\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 33, + "stopIndex" : 102, + "fragment" : "(partition BY val ORDER BY val_date RANGE DATE '2024-01-01' FOLLOWING)" + } ] +} diff --git a/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/xml-functions.sql.out b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/xml-functions.sql.out new file mode 100644 index 000000000000..60531b2bfd50 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/backends-velox/sql-tests/results/xml-functions.sql.out @@ -0,0 +1,437 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +select to_xml(named_struct('a', 1, 'b', 2), map('indent', '')) +-- !query schema +struct +-- !query output +12 + + +-- !query +select to_xml(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy', 'indent', '')) +-- !query schema +struct +-- !query output + + + +-- !query +select to_xml(array(named_struct('a', 1, 'b', 2))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"array(named_struct(a, 1, b, 2))\"", + "inputType" : "\"ARRAY>\"", + "paramIndex" : "first", + "requiredType" : "\"STRUCT\"", + "sqlExpr" : "\"to_xml(array(named_struct(a, 1, b, 2)))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 50, + "fragment" : "to_xml(array(named_struct('a', 1, 'b', 2)))" + } ] +} + + +-- !query +select to_xml(map('a', 1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + "sqlState" : "42K09", + "messageParameters" : { + "inputSql" : "\"map(a, 1)\"", + "inputType" : "\"MAP\"", + "paramIndex" : "first", + "requiredType" : "\"STRUCT\"", + "sqlExpr" : "\"to_xml(map(a, 1))\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "to_xml(map('a', 1))" + } ] +} + + +-- !query +select to_xml(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_MAP_FUNCTION", + "sqlState" : "42K06", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "to_xml(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE'))" + } ] +} + + +-- !query +select to_xml(named_struct('a', 1, 'b', 2), map('mode', 1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_STRING_TYPE", + "sqlState" : "42K06", + "messageParameters" : { + "mapType" : "\"MAP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "to_xml(named_struct('a', 1, 'b', 2), map('mode', 1))" + } ] +} + + +-- !query +select to_xml() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[1, 2]", + "functionName" : "`to_xml`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 15, + "fragment" : "to_xml()" + } ] +} + + +-- !query +select from_xml('

1

', 'a INT') +-- !query schema +struct1

):struct> +-- !query output +{"a":1} + + +-- !query +select from_xml('

', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')) +-- !query schema +struct

):struct> +-- !query output +{"time":2015-08-26 00:00:00} + + +-- !query +select from_xml('

1

', 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_SCHEMA.NON_STRING_LITERAL", + "sqlState" : "42K07", + "messageParameters" : { + "inputSchema" : "\"1\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 37, + "fragment" : "from_xml('

1

', 1)" + } ] +} + + +-- !query +select from_xml('

1

', 'a InvalidType') +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'InvalidType'", + "hint" : ": extra input 'InvalidType'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 51, + "fragment" : "from_xml('

1

', 'a InvalidType')" + } ] +} + + +-- !query +select from_xml('

1

', 'a INT', named_struct('mode', 'PERMISSIVE')) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_MAP_FUNCTION", + "sqlState" : "42K06", + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 79, + "fragment" : "from_xml('

1

', 'a INT', named_struct('mode', 'PERMISSIVE'))" + } ] +} + + +-- !query +select from_xml('

1

', 'a INT', map('mode', 1)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_OPTIONS.NON_STRING_TYPE", + "sqlState" : "42K06", + "messageParameters" : { + "mapType" : "\"MAP\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 59, + "fragment" : "from_xml('

1

', 'a INT', map('mode', 1))" + } ] +} + + +-- !query +select from_xml() +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "0", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "[2, 3]", + "functionName" : "`from_xml`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 17, + "fragment" : "from_xml()" + } ] +} + + +-- !query +DROP VIEW IF EXISTS xmlTable +-- !query schema +struct<> +-- !query output + + + +-- !query +select from_xml('

1

', 'struct>') +-- !query schema +struct1

):struct>> +-- !query output +{"a":[1]} + + +-- !query +select from_xml('

1"2"

', 'struct') +-- !query schema +struct1"2"

):struct> +-- !query output +{"a":1,"b":""2""} + + +-- !query +select schema_of_xml('

1"2"

') +-- !query schema +struct1"2"

):string> +-- !query output +STRUCT + + +-- !query +select from_xml('

123

', schema_of_xml('

12

')) +-- !query schema +struct123

):struct>> +-- !query output +{"a":[1,2,3]} + + +-- !query +select from_xml('

12

', 'struct>') +-- !query schema +struct12

):struct>> +-- !query output +{"a":[1,2]} + + +-- !query +select from_xml('

1"2"

', 'struct>') +-- !query schema +struct1"2"

):struct>> +-- !query output +{"a":[1]} + + +-- !query +select from_xml('

1

', 'struct>') +-- !query schema +struct1

):struct>> +-- !query output +{"a":[1,null]} + + +-- !query +select from_xml('

2

', 'struct>') +-- !query schema +struct2

):struct>> +-- !query output +{"a":{"_attr":1,"b":2}} + + +-- !query +select from_xml('

2012-12-152012-12-15 15:15:15

', 'd date, t timestamp') +-- !query schema +struct2012-12-152012-12-15 15:15:15

):struct> +-- !query output +{"d":2012-12-15,"t":2012-12-15 15:15:15} + + +-- !query +select from_xml( + '

12/15 201212/15 2012 15:15:15}

', + 'd date, t timestamp', + map('dateFormat', 'MM/dd yyyy', 'timestampFormat', 'MM/dd yyyy HH:mm:ss')) +-- !query schema +struct12/15 201212/15 2012 15:15:15}

):struct> +-- !query output +{"d":2012-12-15,"t":2012-12-15 15:15:15} + + +-- !query +select from_xml( + '

02-29

', + 'd date', + map('dateFormat', 'MM-dd')) +-- !query schema +struct02-29

):struct> +-- !query output +{"d":null} + + +-- !query +select from_xml( + '

02-29

', + 't timestamp', + map('timestampFormat', 'MM-dd')) +-- !query schema +struct02-29

):struct> +-- !query output +{"t":null} + + +-- !query +select schema_of_xml(null) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_NULL", + "sqlState" : "42K09", + "messageParameters" : { + "exprName" : "xml", + "sqlExpr" : "\"schema_of_xml(NULL)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 26, + "fragment" : "schema_of_xml(null)" + } ] +} + + +-- !query +CREATE TEMPORARY VIEW xmlTable(xmlField, a) AS SELECT * FROM VALUES ('

1"2"

', 'a') +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT schema_of_xml(xmlField) FROM xmlTable +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", + "sqlState" : "42K09", + "messageParameters" : { + "inputExpr" : "\"xmlField\"", + "inputName" : "`xml`", + "inputType" : "\"STRING\"", + "sqlExpr" : "\"schema_of_xml(xmlField)\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 30, + "fragment" : "schema_of_xml(xmlField)" + } ] +} + + +-- !query +DROP VIEW IF EXISTS xmlTable +-- !query schema +struct<> +-- !query output + diff --git a/gluten-ut/spark41/src/test/resources/log4j2.properties b/gluten-ut/spark41/src/test/resources/log4j2.properties new file mode 100644 index 000000000000..fb1cadec5f5d --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/log4j2.properties @@ -0,0 +1,39 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +rootLogger.level = info +rootLogger.appenderRef.stdout.ref = STDOUT +rootLogger.appenderRef.file.ref = File + +#Console Appender +appender.console.type = Console +appender.console.name = STDOUT +appender.console.target = SYSTEM_OUT +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{HH:mm:ss.SSS} %p %c: %maxLen{%m}{512}%n%ex{8}%n +appender.console.filter.threshold.type = ThresholdFilter +appender.console.filter.threshold.level = warn + +#File Appender +appender.file.type = File +appender.file.name = File +appender.file.fileName = target/unit-tests.log +appender.file.layout.type = PatternLayout +appender.file.layout.pattern = %d{HH:mm:ss.SSS} %t %p %c{1}: %m%n%ex + +appender.file.filter.threshold.type = ThresholdFilter +appender.file.filter.threshold.level = info diff --git a/gluten-ut/spark41/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/gluten-ut/spark41/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker new file mode 100644 index 000000000000..5e48c7d28041 --- /dev/null +++ b/gluten-ut/spark41/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker @@ -0,0 +1,18 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +mock-maker-inline \ No newline at end of file diff --git a/gluten-ut/spark41/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseSQLQueryTestSettings.scala b/gluten-ut/spark41/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseSQLQueryTestSettings.scala new file mode 100644 index 000000000000..937e3494d07a --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseSQLQueryTestSettings.scala @@ -0,0 +1,276 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.utils.clickhouse + +import org.apache.gluten.utils.SQLQueryTestSettings + +object ClickHouseSQLQueryTestSettings extends SQLQueryTestSettings { + override def getResourceFilePath: String = + getClass.getResource("/").getPath + "../../../src/test/resources/backends-clickhouse/sql-tests" + + override def getSupportedSQLQueryTests: Set[String] = SUPPORTED_SQL_QUERY_LIST + + override def getOverwriteSQLQueryTests: Set[String] = OVERWRITE_SQL_QUERY_LIST + + override def getIgnoredSQLQueryTests: List[String] = IGNORE_SQL_QUERY_LIST + + // Put relative path to "/path/to/spark/sql/core/src/test/resources/sql-tests/inputs" in this list + val SUPPORTED_SQL_QUERY_LIST: Set[String] = Set( + "bitwise.sql", + "cast.sql", + "change-column.sql", + // CH- "charvarchar.sql", + "columnresolution-negative.sql", + "columnresolution-views.sql", + "columnresolution.sql", + "comments.sql", + "comparator.sql", + "count.sql", + "cross-join.sql", + "csv-functions.sql", + // CH- "cte-legacy.sql", + "cte-nested.sql", + // CH- "cte-nonlegacy.sql", + // CH- "cte.sql", + "current_database_catalog.sql", + "date.sql", + "datetime-formatting-invalid.sql", + // Velox had different handling for some illegal cases. + // "datetime-formatting-legacy.sql", + // "datetime-formatting.sql", + "datetime-legacy.sql", + "datetime-parsing-invalid.sql", + "datetime-parsing-legacy.sql", + "datetime-parsing.sql", + "datetime-special.sql", + // CH - "decimalArithmeticOperations.sql", + // "describe-part-after-analyze.sql", + "describe-query.sql", + "describe-table-after-alter-table.sql", + "describe-table-column.sql", + "describe.sql", + "except-all.sql", + "except.sql", + "extract.sql", + // CH - "group-by-filter.sql", + // CH - "group-by-ordinal.sql", + "grouping_set.sql", + "having.sql", + "ignored.sql", + "ilike-all.sql", + "ilike-any.sql", + "inline-table.sql", + "inner-join.sql", + "intersect-all.sql", + // CH - "interval.sql", + "join-empty-relation.sql", + // CH - "join-lateral.sql", + // CH - "json-functions.sql", + "like-all.sql", + "like-any.sql", + "limit.sql", + "literals.sql", + "map.sql", + // CH- "misc-functions.sql", + "natural-join.sql", + "null-handling.sql", + // CH- "null-propagation.sql", + "operators.sql", + "order-by-nulls-ordering.sql", + "order-by-ordinal.sql", + "outer-join.sql", + "parse-schema-string.sql", + "pivot.sql", + "pred-pushdown.sql", + "predicate-functions.sql", + "query_regex_column.sql", + // CH- "random.sql", + // CH - "regexp-functions.sql", + "show-create-table.sql", + "show-tables.sql", + "show-tblproperties.sql", + "show-views.sql", + "show_columns.sql", + "sql-compatibility-functions.sql", + "string-functions.sql", + "struct.sql", + // CH - "subexp-elimination.sql", + "table-aliases.sql", + // CH -"table-valued-functions.sql", + "tablesample-negative.sql", + "subquery/exists-subquery/exists-aggregate.sql", + "subquery/exists-subquery/exists-basic.sql", + "subquery/exists-subquery/exists-cte.sql", + "subquery/exists-subquery/exists-having.sql", + "subquery/exists-subquery/exists-joins-and-set-ops.sql", + "subquery/exists-subquery/exists-orderby-limit.sql", + "subquery/exists-subquery/exists-within-and-or.sql", + "subquery/in-subquery/in-basic.sql", + "subquery/in-subquery/in-group-by.sql", + "subquery/in-subquery/in-having.sql", + "subquery/in-subquery/in-joins.sql", + "subquery/in-subquery/in-limit.sql", + "subquery/in-subquery/in-multiple-columns.sql", + "subquery/in-subquery/in-order-by.sql", + // CH- "subquery/in-subquery/in-set-operations.sql", + "subquery/in-subquery/in-with-cte.sql", + "subquery/in-subquery/nested-not-in.sql", + "subquery/in-subquery/not-in-group-by.sql", + "subquery/in-subquery/not-in-joins.sql", + "subquery/in-subquery/not-in-unit-tests-multi-column.sql", + "subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql", + "subquery/in-subquery/not-in-unit-tests-single-column.sql", + "subquery/in-subquery/not-in-unit-tests-single-column-literal.sql", + "subquery/in-subquery/simple-in.sql", + // CH -"subquery/negative-cases/invalid-correlation.sql", + "subquery/negative-cases/subq-input-typecheck.sql", + "subquery/scalar-subquery/scalar-subquery-predicate.sql", + "subquery/scalar-subquery/scalar-subquery-select.sql", + "subquery/subquery-in-from.sql", + "postgreSQL/aggregates_part1.sql", + "postgreSQL/aggregates_part2.sql", + "postgreSQL/aggregates_part3.sql", + "postgreSQL/aggregates_part4.sql", + "postgreSQL/boolean.sql", + "postgreSQL/case.sql", + "postgreSQL/comments.sql", + "postgreSQL/create_view.sql", + "postgreSQL/date.sql", + "postgreSQL/float4.sql", + "postgreSQL/insert.sql", + "postgreSQL/int2.sql", + "postgreSQL/int4.sql", + "postgreSQL/int8.sql", + "postgreSQL/interval.sql", + "postgreSQL/join.sql", + "postgreSQL/limit.sql", + "postgreSQL/numeric.sql", + "postgreSQL/select.sql", + "postgreSQL/select_distinct.sql", + "postgreSQL/select_having.sql", + "postgreSQL/select_implicit.sql", + "postgreSQL/strings.sql", + "postgreSQL/text.sql", + "postgreSQL/timestamp.sql", + "postgreSQL/union.sql", + "postgreSQL/window_part1.sql", + "postgreSQL/window_part2.sql", + "postgreSQL/window_part3.sql", + "postgreSQL/window_part4.sql", + "postgreSQL/with.sql", + "datetime-special.sql", + "timestamp-ansi.sql", + "timestamp.sql", + "arrayJoin.sql", + "binaryComparison.sql", + "booleanEquality.sql", + "caseWhenCoercion.sql", + "concat.sql", + "dateTimeOperations.sql", + "decimalPrecision.sql", + "division.sql", + "elt.sql", + "ifCoercion.sql", + "implicitTypeCasts.sql", + "inConversion.sql", + "mapZipWith.sql", + "promoteStrings.sql", + "stringCastAndExpressions.sql", + "widenSetOperationTypes.sql", + "windowFrameCoercion.sql", + "timestamp-ltz.sql", + "timestamp-ntz.sql", + "timezone.sql", + // CH- "transform.sql", + "try_arithmetic.sql", + "try_cast.sql", + "udaf.sql", + "union.sql", + "using-join.sql", + "window.sql", + "udf-union.sql", + "udf-window.sql", + "ansi/cast.sql", + "ansi/decimalArithmeticOperations.sql", + "ansi/map.sql", + "ansi/datetime-parsing-invalid.sql", + "ansi/string-functions.sql", + // CH - "ansi/interval.sql", + "ansi/date.sql", + "ansi/timestamp.sql", + "ansi/try_arithmetic.sql", + "ansi/literals.sql", + "timestampNTZ/timestamp-ansi.sql", + "timestampNTZ/timestamp.sql", + "udf/udf-intersect-all.sql - Scala UDF", + "udf/udf-except-all.sql - Scala UDF", + "udf/udf-udaf.sql - Scala UDF", + "udf/udf-except.sql - Scala UDF", + "udf/udf-pivot.sql - Scala UDF", + "udf/udf-inline-table.sql - Scala UDF", + "udf/postgreSQL/udf-select_having.sql - Scala UDF", + "typeCoercion/native/decimalPrecision.sql", + "typeCoercion/native/ifCoercion.sql", + "typeCoercion/native/dateTimeOperations.sql", + "typeCoercion/native/booleanEquality.sql", + "typeCoercion/native/mapZipWith.sql", + "typeCoercion/native/caseWhenCoercion.sql", + "typeCoercion/native/widenSetOperationTypes.sql", + "typeCoercion/native/stringCastAndExpressions.sql", + "typeCoercion/native/inConversion.sql", + "typeCoercion/native/division.sql", + "typeCoercion/native/mapconcat.sql" + ) + + val OVERWRITE_SQL_QUERY_LIST: Set[String] = Set( + // The calculation formulas for corr, skewness, kurtosis, variance, and stddev in Velox differ + // slightly from those in Spark, resulting in some differences in the final results. + // Overwrite below test cases. + // -- SPARK-24369 multiple distinct aggregations having the same argument set + // -- Aggregate with nulls. + "group-by.sql", + "udf/udf-group-by.sql" + // Overwrite some results of regr_intercept, regr_r2, corr. + // CH - "linear-regression.sql" + ) + + val IGNORE_SQL_QUERY_LIST: List[String] = List( + "udf/udf-count.sql - Regular Python UDF", + "udf/udf-except.sql - Regular Python UDF", + "udf/udf-except-all.sql - Regular Python UDF", + "udf/udf-natural-join.sql - Regular Python UDF", + "udf/udf-outer-join.sql - Regular Python UDF", + "udf/udf-pivot.sql - Regular Python UDF", + "udf/udf-intersect-all.sql - Regular Python UDF", + "udf/udf-union.sql - Regular Python UDF", + "udf/udf-having.sql - Regular Python UDF", + "udf/udf-group-analytics.sql - Regular Python UDF", + "udf/udf-group-by.sql - Regular Python UDF", + // CH excludes following + "typeCoercion/native/windowFrameCoercion.sql", + "typeCoercion/native/promoteStrings.sql", + "typeCoercion/native/concat.sql", + // Moved from GlutenSQLQueryTestSuite.ignoreList + "window.sql", + "udf/udf-window.sql", + "group-by.sql", + "udf/udf-group-by.sql - Scala UDF", + "udaf/udaf-group-analytics.sql", + "udaf/udaf-group-by-ordinal.sql", + "udaf/udaf-group-by.sql" + ) +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark41/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala new file mode 100644 index 000000000000..abccf9fe912d --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -0,0 +1,2208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.utils.clickhouse + +import org.apache.gluten.utils.{BackendTestSettings, SQLQueryTestSettings} + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.connector._ +import org.apache.spark.sql.errors.{GlutenQueryCompilationErrorsDSv2Suite, GlutenQueryCompilationErrorsSuite, GlutenQueryExecutionErrorsSuite, GlutenQueryParsingErrorsSuite} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive.clickhouse.ClickHouseAdaptiveQueryExecSuite +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.binaryfile.GlutenBinaryFileFormatSuite +import org.apache.spark.sql.execution.datasources.csv.{GlutenCSVLegacyTimeParserSuite, GlutenCSVv1Suite, GlutenCSVv2Suite} +import org.apache.spark.sql.execution.datasources.json.{GlutenJsonLegacyTimeParserSuite, GlutenJsonV1Suite, GlutenJsonV2Suite} +import org.apache.spark.sql.execution.datasources.orc._ +import org.apache.spark.sql.execution.datasources.parquet._ +import org.apache.spark.sql.execution.datasources.text.{GlutenTextV1Suite, GlutenTextV2Suite} +import org.apache.spark.sql.execution.datasources.v2.{GlutenDataSourceV2StrategySuite, GlutenFileTableSuite, GlutenV2PredicateSuite} +import org.apache.spark.sql.execution.exchange.{GlutenEnsureRequirementsSuite, GlutenValidateRequirementsSuite} +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.extension.{GlutenCollapseProjectExecTransformerSuite, GlutenCustomerExtensionSuite, GlutenSessionExtensionSuite} +import org.apache.spark.sql.gluten.{GlutenFallbackStrategiesSuite, GlutenFallbackSuite} +import org.apache.spark.sql.hive.execution.GlutenHiveSQLQueryCHSuite +import org.apache.spark.sql.sources._ + +// Some settings' line length exceeds 100 +// scalastyle:off line.size.limit + +class ClickHouseTestSettings extends BackendTestSettings { + + enableSuite[ClickHouseAdaptiveQueryExecSuite] + .includeAllGlutenTests() + .includeByPrefix( + // exclude SPARK-29906 because gluten columnar operator will have different number of shuffle + "SPARK-30291", + "SPARK-30403", + "SPARK-30719", + "SPARK-31384", + "SPARK-31658", + "SPARK-32649", + "SPARK-34533", + "SPARK-34781", + "SPARK-35585", + "SPARK-32932", + "SPARK-33494", + "SPARK-33933", + "SPARK-31220", + "SPARK-35874", + "SPARK-39551" + ) + .include( + "Union/Except/Intersect queries", + "Subquery de-correlation in Union queries", + "force apply AQE", + "tree string output", + "control a plan explain mode in listener vis SQLConf", + "AQE should set active session during execution", + "No deadlock in UI update", + "SPARK-35455: Unify empty relation optimization between normal and AQE optimizer - multi join" + ) + enableSuite[GlutenFallbackStrategiesSuite] + enableSuite[GlutenApproxCountDistinctForIntervalsQuerySuite] + .excludeCH("test ApproxCountDistinctForIntervals with large number of endpoints") + enableSuite[GlutenApproximatePercentileQuerySuite] + // requires resource files from Vanilla spark jar + .exclude("SPARK-32908: maximum target error in percentile_approx") + enableSuite[GlutenArithmeticExpressionSuite] + .exclude("SPARK-45786: Decimal multiply, divide, remainder, quot") + .excludeCH("% (Remainder)") + .excludeCH("SPARK-17617: % (Remainder) double % double on super big double") + .excludeCH("pmod") + enableSuite[GlutenBinaryFileFormatSuite] + // Exception. + .exclude("column pruning - non-readable file") + enableSuite[GlutenBitmapExpressionsQuerySuite] + enableSuite[GlutenBitwiseExpressionsSuite] + enableSuite[GlutenBloomFilterAggregateQuerySuite] + .excludeCH("Test bloom_filter_agg and might_contain") + enableSuite[GlutenBloomFilterAggregateQuerySuiteCGOff] + .excludeCH("Test bloom_filter_agg and might_contain") + enableSuite[GlutenBroadcastExchangeSuite] + // TODO: fix the hanging problem in GLUTEN-8890 followup +// enableSuite[GlutenBroadcastJoinSuite] +// .includeCH("Shouldn't change broadcast join buildSide if user clearly specified") +// .includeCH("Shouldn't bias towards build right if user didn't specify") +// .includeCH("SPARK-23192: broadcast hint should be retained after using the cached data") +// .includeCH("broadcast join where streamed side's output partitioning is HashPartitioning") + enableSuite[GlutenBucketedReadWithoutHiveSupportSuite] + // Exclude the following suite for plan changed from SMJ to SHJ. + .exclude("avoid shuffle when join 2 bucketed tables") + .exclude("avoid shuffle and sort when sort columns are a super set of join keys") + .exclude("only shuffle one side when join bucketed table and non-bucketed table") + .exclude("only shuffle one side when 2 bucketed tables have different bucket number") + .exclude("only shuffle one side when 2 bucketed tables have different bucket keys") + .exclude("shuffle when join keys are not equal to bucket keys") + .exclude("shuffle when join 2 bucketed tables with bucketing disabled") + .exclude("check sort and shuffle when bucket and sort columns are join keys") + .exclude("only sort one side when sort columns are different") + .exclude("only sort one side when sort columns are same but their ordering is different") + .exclude("SPARK-17698 Join predicates should not contain filter clauses") + .exclude("SPARK-19122 Re-order join predicates if they match with the child's" + + " output partitioning") + .exclude("SPARK-19122 No re-ordering should happen if set of join columns != set of child's " + + "partitioning columns") + .exclude("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") + .exclude("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") + .exclude("bucket coalescing eliminates shuffle") + .exclude("bucket coalescing is not satisfied") + // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .exclude("disable bucketing when the output doesn't contain all bucketing columns") + .excludeByPrefix("bucket coalescing is applied when join expressions match") + enableSuite[GlutenBucketedWriteWithoutHiveSupportSuite] + .includeCH("write bucketed data") + .includeCH("write bucketed data with sortBy") + .includeCH("write bucketed data without partitionBy") + .includeCH("write bucketed data without partitionBy with sortBy") + .includeCH("write bucketed data with bucketing disabled") + enableSuite[GlutenBucketingUtilsSuite] + enableSuite[GlutenCSVLegacyTimeParserSuite] + // file cars.csv include null string, Arrow not support to read + .exclude("DDL test with schema") + .exclude("save csv") + .exclude("save csv with compression codec option") + .exclude("save csv with empty fields with user defined empty values") + .exclude("save csv with quote") + .exclude("SPARK-13543 Write the output as uncompressed via option()") + // Arrow not support corrupt record + .exclude("SPARK-27873: disabling enforceSchema should not fail columnNameOfCorruptRecord") + .exclude("DDL test with tab separated file") + .exclude("DDL test parsing decimal type") + .exclude("test with tab delimiter and double quote") + // varchar + .exclude("SPARK-48241: CSV parsing failure with char/varchar type columns") + .excludeCH("simple csv test") + .excludeCH("simple csv test with calling another function to load") + .excludeCH("simple csv test with type inference") + .excludeCH("test with alternative delimiter and quote") + .excludeCH("SPARK-24540: test with multiple character delimiter (comma space)") + .excludeCH("SPARK-24540: test with multiple (crazy) character delimiter") + .excludeCH("test different encoding") + .excludeCH("crlf line separators in multiline mode") + .excludeCH("test aliases sep and encoding for delimiter and charset") + .excludeCH("test for DROPMALFORMED parsing mode") + .excludeCH("test for blank column names on read and select columns") + .excludeCH("test for FAILFAST parsing mode") + .excludeCH("test for tokens more than the fields in the schema") + .excludeCH("test with null quote character") + .excludeCH("save csv with quote escaping, using charToEscapeQuoteEscaping option") + .excludeCH("commented lines in CSV data") + .excludeCH("inferring schema with commented lines in CSV data") + .excludeCH("inferring timestamp types via custom date format") + .excludeCH("load date types via custom date format") + .excludeCH("nullable fields with user defined null value of \"null\"") + .excludeCH("empty fields with user defined empty values") + .excludeCH("old csv data source name works") + .excludeCH("nulls, NaNs and Infinity values can be parsed") + .excludeCH("SPARK-15585 turn off quotations") + .excludeCH("Write timestamps correctly in ISO8601 format by default") + .excludeCH("Write dates correctly in ISO8601 format by default") + .excludeCH("Roundtrip in reading and writing timestamps") + .excludeCH("SPARK-37326: Write and infer TIMESTAMP_LTZ values with a non-default pattern") + .excludeCH("SPARK-37326: Timestamp type inference for a mix of TIMESTAMP_NTZ and TIMESTAMP_LTZ") + .excludeCH("Write dates correctly with dateFormat option") + .excludeCH("Write timestamps correctly with timestampFormat option") + .excludeCH("Write timestamps correctly with timestampFormat option and timeZone option") + .excludeCH("SPARK-18699 put malformed records in a `columnNameOfCorruptRecord` field") + .excludeCH("Enabling/disabling ignoreCorruptFiles") + .excludeCH("SPARK-19610: Parse normal multi-line CSV files") + .excludeCH("SPARK-38523: referring to the corrupt record column") + .excludeCH( + "SPARK-17916: An empty string should not be coerced to null when nullValue is passed.") + .excludeCH( + "SPARK-25241: An empty string should not be coerced to null when emptyValue is passed.") + .excludeCH("SPARK-24329: skip lines with comments, and one or multiple whitespaces") + .excludeCH("SPARK-23786: Checking column names against schema in the multiline mode") + .excludeCH("SPARK-23786: Checking column names against schema in the per-line mode") + .excludeCH("SPARK-23786: Ignore column name case if spark.sql.caseSensitive is false") + .excludeCH("SPARK-23786: warning should be printed if CSV header doesn't conform to schema") + .excludeCH("SPARK-25134: check header on parsing of dataset with projection and column pruning") + .excludeCH("SPARK-24676 project required data from parsed data when columnPruning disabled") + .excludeCH("encoding in multiLine mode") + .excludeCH("Support line separator - default value \\r, \\r\\n and \\n") + .excludeCH("Support line separator in UTF-8 #0") + .excludeCH("Support line separator in UTF-16BE #1") + .excludeCH("Support line separator in ISO-8859-1 #2") + .excludeCH("Support line separator in UTF-32LE #3") + .excludeCH("Support line separator in UTF-8 #4") + .excludeCH("Support line separator in UTF-32BE #5") + .excludeCH("Support line separator in CP1251 #6") + .excludeCH("Support line separator in UTF-16LE #8") + .excludeCH("Support line separator in UTF-32BE #9") + .excludeCH("Support line separator in US-ASCII #10") + .excludeCH("Support line separator in utf-32le #11") + .excludeCH("lineSep with 2 chars when multiLine set to true") + .excludeCH("lineSep with 2 chars when multiLine set to false") + .excludeCH("SPARK-26208: write and read empty data to csv file with headers") + .excludeCH("Do not reuse last good value for bad input field") + .excludeCH("SPARK-29101 test count with DROPMALFORMED mode") + .excludeCH("return correct results when data columns overlap with partition columns") + .excludeCH("filters push down - malformed input in PERMISSIVE mode") + .excludeCH("case sensitivity of filters references") + .excludeCH("SPARK-33566: configure UnescapedQuoteHandling to parse unescaped quotes and unescaped delimiter data correctly") + .excludeCH("SPARK-36831: Support reading and writing ANSI intervals") + .excludeCH("SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern") + .excludeCH("SPARK-39731: Handle date and timestamp parsing fallback") + .excludeCH("SPARK-40215: enable parsing fallback for CSV in CORRECTED mode with a SQL config") + .excludeCH("SPARK-40496: disable parsing fallback when the date/timestamp format is provided") + .excludeCH("SPARK-42335: Pass the comment option through to univocity if users set it explicitly in CSV dataSource") + .excludeCH("SPARK-46862: column pruning in the multi-line mode") + enableSuite[GlutenCSVReadSchemaSuite] + enableSuite[GlutenCSVv1Suite] + // file cars.csv include null string, Arrow not support to read + .exclude("DDL test with schema") + .exclude("save csv") + .exclude("save csv with compression codec option") + .exclude("save csv with empty fields with user defined empty values") + .exclude("save csv with quote") + .exclude("SPARK-13543 Write the output as uncompressed via option()") + .exclude("DDL test with tab separated file") + .exclude("DDL test parsing decimal type") + .exclude("test with tab delimiter and double quote") + // Arrow not support corrupt record + .exclude("SPARK-27873: disabling enforceSchema should not fail columnNameOfCorruptRecord") + // varchar + .exclude("SPARK-48241: CSV parsing failure with char/varchar type columns") + .excludeCH("simple csv test") + .excludeCH("simple csv test with calling another function to load") + .excludeCH("simple csv test with type inference") + .excludeCH("test with alternative delimiter and quote") + .excludeCH("SPARK-24540: test with multiple character delimiter (comma space)") + .excludeCH("SPARK-24540: test with multiple (crazy) character delimiter") + .excludeCH("test different encoding") + .excludeCH("crlf line separators in multiline mode") + .excludeCH("test aliases sep and encoding for delimiter and charset") + .excludeCH("test for DROPMALFORMED parsing mode") + .excludeCH("test for blank column names on read and select columns") + .excludeCH("test for FAILFAST parsing mode") + .excludeCH("test for tokens more than the fields in the schema") + .excludeCH("test with null quote character") + .excludeCH("save csv with quote escaping, using charToEscapeQuoteEscaping option") + .excludeCH("commented lines in CSV data") + .excludeCH("inferring schema with commented lines in CSV data") + .excludeCH("inferring timestamp types via custom date format") + .excludeCH("load date types via custom date format") + .excludeCH("nullable fields with user defined null value of \"null\"") + .excludeCH("empty fields with user defined empty values") + .excludeCH("old csv data source name works") + .excludeCH("nulls, NaNs and Infinity values can be parsed") + .excludeCH("SPARK-15585 turn off quotations") + .excludeCH("Write timestamps correctly in ISO8601 format by default") + .excludeCH("Write dates correctly in ISO8601 format by default") + .excludeCH("Roundtrip in reading and writing timestamps") + .excludeCH("SPARK-37326: Write and infer TIMESTAMP_LTZ values with a non-default pattern") + .excludeCH("SPARK-37326: Timestamp type inference for a column with TIMESTAMP_NTZ values") + .excludeCH("SPARK-37326: Timestamp type inference for a mix of TIMESTAMP_NTZ and TIMESTAMP_LTZ") + .excludeCH("Write dates correctly with dateFormat option") + .excludeCH("Write timestamps correctly with timestampFormat option") + .excludeCH("Write timestamps correctly with timestampFormat option and timeZone option") + .excludeCH("SPARK-18699 put malformed records in a `columnNameOfCorruptRecord` field") + .excludeCH("Enabling/disabling ignoreCorruptFiles") + .excludeCH("SPARK-19610: Parse normal multi-line CSV files") + .excludeCH("SPARK-38523: referring to the corrupt record column") + .excludeCH( + "SPARK-17916: An empty string should not be coerced to null when nullValue is passed.") + .excludeCH( + "SPARK-25241: An empty string should not be coerced to null when emptyValue is passed.") + .excludeCH("SPARK-24329: skip lines with comments, and one or multiple whitespaces") + .excludeCH("SPARK-23786: Checking column names against schema in the multiline mode") + .excludeCH("SPARK-23786: Checking column names against schema in the per-line mode") + .excludeCH("SPARK-23786: Ignore column name case if spark.sql.caseSensitive is false") + .excludeCH("SPARK-23786: warning should be printed if CSV header doesn't conform to schema") + .excludeCH("SPARK-25134: check header on parsing of dataset with projection and column pruning") + .excludeCH("SPARK-24676 project required data from parsed data when columnPruning disabled") + .excludeCH("encoding in multiLine mode") + .excludeCH("Support line separator - default value \\r, \\r\\n and \\n") + .excludeCH("Support line separator in UTF-8 #0") + .excludeCH("Support line separator in UTF-16BE #1") + .excludeCH("Support line separator in ISO-8859-1 #2") + .excludeCH("Support line separator in UTF-32LE #3") + .excludeCH("Support line separator in UTF-8 #4") + .excludeCH("Support line separator in UTF-32BE #5") + .excludeCH("Support line separator in CP1251 #6") + .excludeCH("Support line separator in UTF-16LE #8") + .excludeCH("Support line separator in UTF-32BE #9") + .excludeCH("Support line separator in US-ASCII #10") + .excludeCH("Support line separator in utf-32le #11") + .excludeCH("lineSep with 2 chars when multiLine set to true") + .excludeCH("lineSep with 2 chars when multiLine set to false") + .excludeCH("SPARK-26208: write and read empty data to csv file with headers") + .excludeCH("Do not reuse last good value for bad input field") + .excludeCH("SPARK-29101 test count with DROPMALFORMED mode") + .excludeCH("return correct results when data columns overlap with partition columns") + .excludeCH("filters push down - malformed input in PERMISSIVE mode") + .excludeCH("case sensitivity of filters references") + .excludeCH("SPARK-33566: configure UnescapedQuoteHandling to parse unescaped quotes and unescaped delimiter data correctly") + .excludeCH("SPARK-36831: Support reading and writing ANSI intervals") + .excludeCH("SPARK-39469: Infer schema for columns with all dates") + .excludeCH("SPARK-40474: Infer schema for columns with a mix of dates and timestamp") + .excludeCH("SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern") + .excludeCH("SPARK-39731: Handle date and timestamp parsing fallback") + .excludeCH("SPARK-40215: enable parsing fallback for CSV in CORRECTED mode with a SQL config") + .excludeCH("SPARK-40496: disable parsing fallback when the date/timestamp format is provided") + .excludeCH("SPARK-42335: Pass the comment option through to univocity if users set it explicitly in CSV dataSource") + .excludeCH("SPARK-46862: column pruning in the multi-line mode") + // Flaky and already excluded in other cases + .exclude("Gluten - test for FAILFAST parsing mode") + + enableSuite[GlutenCSVv2Suite] + .exclude("Gluten - test for FAILFAST parsing mode") + // Rule org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown in batch + // Early Filter and Projection Push-Down generated an invalid plan + .exclude("SPARK-26208: write and read empty data to csv file with headers") + // file cars.csv include null string, Arrow not support to read + .exclude("old csv data source name works") + .exclude("DDL test with schema") + .exclude("save csv") + .exclude("save csv with compression codec option") + .exclude("save csv with empty fields with user defined empty values") + .exclude("save csv with quote") + .exclude("SPARK-13543 Write the output as uncompressed via option()") + .exclude("DDL test with tab separated file") + .exclude("DDL test parsing decimal type") + .exclude("test with tab delimiter and double quote") + // Arrow not support corrupt record + .exclude("SPARK-27873: disabling enforceSchema should not fail columnNameOfCorruptRecord") + // varchar + .exclude("SPARK-48241: CSV parsing failure with char/varchar type columns") + .excludeCH("SPARK-36831: Support reading and writing ANSI intervals") + enableSuite[GlutenCTEHintSuite] + enableSuite[GlutenCTEInlineSuiteAEOff] + enableSuite[GlutenCTEInlineSuiteAEOn] + enableSuite[GlutenCachedTableSuite] + .exclude("A cached table preserves the partitioning and ordering of its cached SparkPlan") + .includeCH("InMemoryRelation statistics") + // Extra ColumnarToRow is needed to transform vanilla columnar data to gluten columnar data. + .includeCH("SPARK-37369: Avoid redundant ColumnarToRow transition on InMemoryTableScan") + .excludeCH("Gluten - InMemoryRelation statistics") + enableSuite[GlutenCastWithAnsiOffSuite] + .exclude( + "Process Infinity, -Infinity, NaN in case insensitive manner" // +inf not supported in folly. + ) + // Set timezone through config. + .exclude("data type casting") + .excludeCH("null cast") + .excludeCH("cast string to date") + .excludeCH("cast string to timestamp") + .excludeGlutenTest("cast string to timestamp") + .excludeCH("SPARK-22825 Cast array to string") + .excludeCH("SPARK-33291: Cast array with null elements to string") + .excludeCH("SPARK-22973 Cast map to string") + .excludeCH("SPARK-22981 Cast struct to string") + .excludeCH("SPARK-33291: Cast struct with null elements to string") + .excludeCH("SPARK-35111: Cast string to year-month interval") + .excludeCH("Gluten - data type casting") + .exclude("cast string to date #2") + .exclude("casting to fixed-precision decimals") + .exclude("SPARK-28470: Cast should honor nullOnOverflow property") + .exclude("cast from array II") + .exclude("cast from map II") + .exclude("cast from struct II") + .exclude("cast from date") + .exclude("cast from timestamp II") + .exclude("cast a timestamp before the epoch 1970-01-01 00:00:00Z") + .exclude("SPARK-34727: cast from float II") + .exclude("SPARK-39749: cast Decimal to string") + .exclude("SPARK-42176: cast boolean to timestamp") + .exclude("null cast #2") + .exclude("cast from boolean to timestamp") + enableSuite[GlutenCoalesceShufflePartitionsSuite] + .excludeByPrefix("determining the number of reducers") + .excludeCH("SPARK-46590 adaptive query execution works correctly with broadcast join and union") + .excludeCH("SPARK-46590 adaptive query execution works correctly with cartesian join and union") + .excludeCH("SPARK-24705 adaptive query execution works correctly when exchange reuse enabled") + .excludeCH("Do not reduce the number of shuffle partition for repartition") + .excludeCH("Union two datasets with different pre-shuffle partition number") + .excludeCH("SPARK-34790: enable IO encryption in AQE partition coalescing") + .excludeCH("Gluten - determining the number of reducers: aggregate operator(minNumPostShufflePartitions: 5)") + .excludeCH( + "Gluten - determining the number of reducers: join operator(minNumPostShufflePartitions: 5)") + .excludeCH( + "Gluten - determining the number of reducers: complex query 1(minNumPostShufflePartitions: 5)") + .excludeCH( + "Gluten - determining the number of reducers: complex query 2(minNumPostShufflePartitions: 5)") + .excludeCH("Gluten - determining the number of reducers: plan already partitioned(minNumPostShufflePartitions: 5)") + .excludeCH("Gluten - determining the number of reducers: aggregate operator") + .excludeCH("Gluten - determining the number of reducers: join operator") + .excludeCH("Gluten - determining the number of reducers: complex query 1") + .excludeCH("Gluten - determining the number of reducers: complex query 2") + .excludeCH("Gluten - determining the number of reducers: plan already partitioned") + enableSuite[GlutenCollapseProjectExecTransformerSuite] + .excludeCH("Gluten - Support ProjectExecTransformer collapse") + enableSuite[GlutenCollectionExpressionsSuite] + // Rewrite in Gluten to replace Seq with Array + .exclude("Shuffle") + .excludeGlutenTest("Shuffle") + .excludeCH("Sequence of numbers") + .excludeCH("Array Insert") + .excludeCH("SPARK-36753: ArrayExcept should handle duplicated Double.NaN and Float.Nan") + .excludeCH( + "SPARK-36740: ArrayMin/ArrayMax/SortArray should handle NaN greater than non-NaN value") + .excludeCH("SPARK-42401: Array insert of null value (explicit)") + .excludeCH("SPARK-42401: Array insert of null value (implicit)") + enableSuite[GlutenColumnExpressionSuite] + // Velox raise_error('errMsg') throws a velox_user_error exception with the message 'errMsg'. + // The final caught Spark exception's getCause().getMessage() contains 'errMsg' but does not + // equal 'errMsg' exactly. The following two tests will be skipped and overridden in Gluten. + .includeCH("raise_error") + .includeCH("assert_true") + .excludeCH("withField should add field with no name") + .excludeCH("withField should replace all fields with given name in struct") + .excludeCH("withField user-facing examples") + .excludeCH("dropFields should drop field with no name in struct") + .excludeCH("dropFields should drop all fields with given name in struct") + enableSuite[GlutenComplexTypeSuite] + enableSuite[GlutenComplexTypesSuite] + enableSuite[GlutenConditionalExpressionSuite] + .excludeCH("case when") + enableSuite[GlutenConfigBehaviorSuite] + // Will be fixed by cleaning up ColumnarShuffleExchangeExec. + .exclude("SPARK-22160 spark.sql.execution.rangeExchange.sampleSizePerPartition") + // Gluten columnar operator will have different number of jobs + .exclude("SPARK-40211: customize initialNumPartitions for take") + enableSuite[GlutenCountMinSketchAggQuerySuite] + enableSuite[GlutenCreateTableAsSelectSuite] + .exclude("CREATE TABLE USING AS SELECT based on the file without write permission") + .exclude("create a table, drop it and create another one with the same name") + enableSuite[GlutenCsvFunctionsSuite] + enableSuite[GlutenCustomerExtensionSuite] + enableSuite[GlutenDDLSourceLoadSuite] + enableSuite[GlutenDSV2CharVarcharTestSuite] + enableSuite[GlutenDSV2SQLInsertTestSuite] + enableSuite[GlutenDataFrameAggregateSuite] + // Test for vanilla spark codegen, not apply for Gluten + .exclude("SPARK-43876: Enable fast hashmap for distinct queries") + .exclude( + "SPARK-26021: NaN and -0.0 in grouping expressions", // NaN case + // incorrect result, distinct NaN case + "SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate", + // Replaced with another test. + "SPARK-19471: AggregationIterator does not initialize the generated result projection" + + " before using it" + ) + .includeCH( + "zero moments", // [velox does not return NaN] + // Velox's collect_list / collect_set are by design declarative aggregate so plan check + // for ObjectHashAggregateExec will fail. + "SPARK-22223: ObjectHashAggregate should not introduce unnecessary shuffle", + "SPARK-31620: agg with subquery (whole-stage-codegen = true)", + "SPARK-31620: agg with subquery (whole-stage-codegen = false)" + ) + .excludeCH("linear regression") + .excludeCH("collect functions") + .excludeCH("collect functions structs") + .excludeCH("SPARK-17641: collect functions should not collect null values") + .excludeCH("collect functions should be able to cast to array type with no null values") + .excludeCH("SPARK-45599: Neither 0.0 nor -0.0 should be dropped when computing percentile") + .excludeCH("SPARK-34716: Support ANSI SQL intervals by the aggregate function `sum`") + .excludeCH("SPARK-34837: Support ANSI SQL intervals by the aggregate function `avg`") + .excludeCH("SPARK-35412: groupBy of year-month/day-time intervals should work") + .excludeCH("SPARK-36054: Support group by TimestampNTZ column") + enableSuite[GlutenDataFrameAsOfJoinSuite] + enableSuite[GlutenDataFrameComplexTypeSuite] + enableSuite[GlutenDataFrameFunctionsSuite] + // blocked by Velox-5768 + .exclude("aggregate function - array for primitive type containing null") + .exclude("aggregate function - array for non-primitive type") + // Expected exception org.apache.spark.SparkException to be thrown, but no exception was thrown + .exclude("map_concat function") + // Rewrite this test because Velox sorts rows by key for primitive data types, which disrupts the original row sequence. + .includeCH("map_zip_with function - map of primitive types") + .excludeCH("map with arrays") + .excludeCH("flatten function") + .excludeCH("SPARK-41233: array prepend") + .excludeCH("array_insert functions") + .excludeCH("aggregate function - array for primitive type not containing null") + .excludeCH("transform keys function - primitive data types") + .excludeCH("transform values function - test primitive data types") + .excludeCH("transform values function - test empty") + .excludeCH("SPARK-14393: values generated by non-deterministic functions shouldn't change after coalesce or union") + .excludeCH("mask function") + enableSuite[GlutenDataFrameHintSuite] + enableSuite[GlutenDataFrameImplicitsSuite] + enableSuite[GlutenDataFrameJoinSuite] + .excludeCH("SPARK-32693: Compare two dataframes with same schema except nullable property") + enableSuite[GlutenDataFrameNaFunctionsSuite] + .includeCH( + // NaN case + "replace nan with float", + "replace nan with double" + ) + enableSuite[GlutenDataFramePivotSuite] + // substring issue + .includeCH("pivot with column definition in groupby") + // array comparison not supported for values that contain nulls + .includeCH( + "pivot with null and aggregate type not supported by PivotFirst returns correct result") + .excludeCH("SPARK-38133: Grouping by TIMESTAMP_NTZ should not corrupt results") + enableSuite[GlutenDataFrameRangeSuite] + .excludeCH("SPARK-20430 Initialize Range parameters in a driver side") + .excludeByPrefix("Cancelling stage in a query with Range") + enableSuite[GlutenDataFrameSelfJoinSuite] + enableSuite[GlutenDataFrameSessionWindowingSuite] + .excludeCH("simple session window with record at window start") + .excludeCH("session window groupBy statement") + .excludeCH("session window groupBy with multiple keys statement") + .excludeCH("session window groupBy with multiple keys statement - two distinct") + .excludeCH( + "session window groupBy with multiple keys statement - keys overlapped with sessions") + .excludeCH("SPARK-36465: filter out events with negative/zero gap duration") + .excludeCH("SPARK-36724: Support timestamp_ntz as a type of time column for SessionWindow") + // TODO: fix the hanging problem in GLUTEN-8890 followup + .excludeCH( + "SPARK-49836 using window fn with window as parameter should preserve parent operator") + enableSuite[GlutenDataFrameSetOperationsSuite] + .exclude("SPARK-37371: UnionExec should support columnar if all children support columnar") + // Result depends on the implementation for nondeterministic expression rand. + // Not really an issue. + .exclude("SPARK-10740: handle nondeterministic expressions correctly for set operations") + .excludeCH("union should union DataFrames with UDTs (SPARK-13410)") + .excludeCH( + "SPARK-35756: unionByName support struct having same col names but different sequence") + .excludeCH("SPARK-36673: Only merge nullability for Unions of struct") + .excludeCH("SPARK-36797: Union should resolve nested columns as top-level columns") + enableSuite[GlutenDataFrameStatSuite] + .excludeCH("SPARK-30532 stat functions to understand fully-qualified column name") + .excludeCH("special crosstab elements (., '', null, ``)") + enableSuite[GlutenDataFrameSuite] + // Rewrite these tests because it checks Spark's physical operators. + .excludeByPrefix("SPARK-22520", "reuse exchange") + .exclude( + /** + * Rewrite these tests because the rdd partition is equal to the configuration + * "spark.sql.shuffle.partitions". + */ + "repartitionByRange", + // Rewrite this test because the describe functions creates unmatched plan. + "describe", + // decimal failed ut. + "SPARK-22271: mean overflows and returns null for some decimal variables", + // Result depends on the implementation for nondeterministic expression rand. + // Not really an issue. + "SPARK-9083: sort with non-deterministic expressions" + ) + .includeCH( + // Mismatch when max NaN and infinite value + "NaN is greater than all other non-NaN numeric values", + "distributeBy and localSort" + ) + // test for sort node not present but gluten uses shuffle hash join + .exclude("SPARK-41048: Improve output partitioning and ordering with AQE cache") + .exclude("SPARK-28224: Aggregate sum big decimal overflow") + // Rewrite this test since it checks the physical operator which is changed in Gluten + .excludeCH("SPARK-27439: Explain result should match collected result after view change") + .excludeCH("SPARK-28067: Aggregate sum should not return wrong results for decimal overflow") + .excludeCH("SPARK-35955: Aggregate avg should not return wrong results for decimal overflow") + .excludeCH("summary") + .excludeGlutenTest( + "SPARK-27439: Explain result should match collected result after view change") + .excludeCH( + "SPARK-8608: call `show` on local DataFrame with random columns should return same value") + .excludeCH( + "SPARK-8609: local DataFrame with random columns should return same value after sort") + .excludeCH("SPARK-10316: respect non-deterministic expressions in PhysicalOperation") + .excludeCH("Uuid expressions should produce same results at retries in the same DataFrame") + .excludeCH("Gluten - repartitionByRange") + .excludeCH("Gluten - describe") + .excludeCH("Gluten - Allow leading/trailing whitespace in string before casting") + enableSuite[GlutenDataFrameTimeWindowingSuite] + .excludeCH("simple tumbling window with record at window start") + .excludeCH("SPARK-21590: tumbling window using negative start time") + .excludeCH("tumbling window groupBy statement") + .excludeCH("tumbling window groupBy statement with startTime") + .excludeCH("SPARK-21590: tumbling window groupBy statement with negative startTime") + .excludeCH("sliding window grouping") + .excludeCH("time window joins") + .excludeCH("millisecond precision sliding windows") + enableSuite[GlutenDataFrameToSchemaSuite] + .excludeCH("struct value: compatible field nullability") + .excludeCH("map value: reorder inner fields by name") + enableSuite[GlutenDataFrameTungstenSuite] + enableSuite[GlutenDataFrameWindowFramesSuite] + // Local window fixes are not added. + .exclude("range between should accept int/long values as boundary") + .includeCH("unbounded preceding/following range between with aggregation") + .includeCH("sliding range between with aggregation") + .exclude("store and retrieve column stats in different time zones") + .excludeCH("rows between should accept int/long values as boundary") + .excludeCH("reverse preceding/following range between with aggregation") + .excludeCH( + "SPARK-41793: Incorrect result for window frames defined by a range clause on large decimals") + enableSuite[GlutenDataFrameWindowFunctionsSuite] + // does not support `spark.sql.legacy.statisticalAggregate=true` (null -> NAN) + .exclude("corr, covar_pop, stddev_pop functions in specific window") + .exclude("covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window") + // does not support spill + .exclude("Window spill with more than the inMemoryThreshold and spillThreshold") + .exclude("SPARK-21258: complex object in combination with spilling") + // rewrite `WindowExec -> WindowExecTransformer` + .exclude( + "SPARK-38237: require all cluster keys for child required distribution for window query") + .excludeCH("SPARK-13860: corr, covar_pop, stddev_pop functions in specific window LEGACY_STATISTICAL_AGGREGATE off") + .excludeCH("SPARK-13860: covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window LEGACY_STATISTICAL_AGGREGATE off") + .excludeCH("lead/lag with ignoreNulls") + .excludeCH("SPARK-37099: Insert window group limit node for top-k computation") + .excludeCH("Gluten - corr, covar_pop, stddev_pop functions in specific window") + enableSuite[GlutenDataFrameWriterV2Suite] + enableSuite[GlutenDataSourceStrategySuite] + enableSuite[GlutenDataSourceSuite] + enableSuite[GlutenDataSourceV2DataFrameSessionCatalogSuite] + enableSuite[GlutenDataSourceV2DataFrameSuite] + enableSuite[GlutenDataSourceV2FunctionSuite] + .excludeCH("view should use captured catalog and namespace for function lookup") + .excludeCH("aggregate function: lookup int average") + .excludeCH("aggregate function: lookup long average") + .excludeCH("aggregate function: lookup double average in Java") + .excludeCH("aggregate function: lookup int average w/ expression") + .excludeCH("SPARK-35390: aggregate function w/ type coercion") + enableSuite[GlutenDataSourceV2SQLSessionCatalogSuite] + enableSuite[GlutenDataSourceV2SQLSuiteV1Filter] + .excludeCH("DeleteFrom with v2 filtering: fail if has subquery") + .excludeCH("DeleteFrom with v2 filtering: delete with unsupported predicates") + .excludeCH("SPARK-33652: DeleteFrom should refresh caches referencing the table") + .excludeCH("DeleteFrom: - delete with invalid predicate") + enableSuite[GlutenDataSourceV2SQLSuiteV2Filter] + .excludeCH("DeleteFrom with v2 filtering: fail if has subquery") + .excludeCH("DeleteFrom with v2 filtering: delete with unsupported predicates") + .excludeCH("SPARK-33652: DeleteFrom should refresh caches referencing the table") + enableSuite[GlutenDataSourceV2StrategySuite] + enableSuite[GlutenDataSourceV2Suite] + // Rewrite the following tests in GlutenDataSourceV2Suite. + .exclude("partitioning reporting") + .exclude("ordering and partitioning reporting") + enableSuite[GlutenDatasetAggregatorSuite] + enableSuite[GlutenDatasetCacheSuite] + // Disable this since coalesece union clauses rule will rewrite the query. + .exclude("SPARK-44653: non-trivial DataFrame unions should not break caching") + enableSuite[GlutenDatasetOptimizationSuite] + enableSuite[GlutenDatasetPrimitiveSuite] + enableSuite[GlutenDatasetSerializerRegistratorSuite] + enableSuite[GlutenDatasetSuite] + // Rewrite the following two tests in GlutenDatasetSuite. + .exclude("dropDuplicates: columns with same column name") + .exclude("groupBy.as") + .exclude("dropDuplicates") + .exclude("select 2, primitive and tuple") + .exclude("SPARK-16853: select, case class and tuple") + // TODO: SPARK-16995 may dead loop!! + .exclude("SPARK-16995: flat mapping on Dataset containing a column created with lit/expr") + .exclude("SPARK-24762: typed agg on Option[Product] type") + .exclude("SPARK-40407: repartition should not result in severe data skew") + .exclude("SPARK-40660: Switch to XORShiftRandom to distribute elements") + enableSuite[GlutenDatasetUnpivotSuite] + enableSuite[GlutenDateExpressionsSuite] + // Has exception in fallback execution when we use resultDF.collect in evaluation. + .exclude("TIMESTAMP_MICROS") + // Replaced by a gluten test to pass timezone through config. + .exclude("unix_timestamp") + // Replaced by a gluten test to pass timezone through config. + .exclude("to_unix_timestamp") + // Replaced by a gluten test to pass timezone through config. + .exclude("Hour") + // Unsupported format: yyyy-MM-dd HH:mm:ss.SSS + .exclude("SPARK-33498: GetTimestamp,UnixTimestamp,ToUnixTimestamp with parseError") + // Replaced by a gluten test to pass timezone through config. + .exclude("DateFormat") + // Legacy mode is not supported, assuming this mode is not commonly used. + .exclude("to_timestamp exception mode") + // Replaced by a gluten test to pass timezone through config. + .exclude("from_unixtime") + .exclude("SPARK-42635: timestampadd near daylight saving transition") + // https://github.com/facebookincubator/velox/pull/10563/files#diff-140dc50e6dac735f72d29014da44b045509df0dd1737f458de1fe8cfd33d8145 + .excludeGlutenTest("from_unixtime") + .excludeCH("DayOfYear") + .excludeCH("Quarter") + .excludeCH("Month") + .excludeCH("Day / DayOfMonth") + .excludeCH("DayOfWeek") + .excludeCH("WeekDay") + .excludeCH("WeekOfYear") + .excludeCH("add_months") + .excludeCH("months_between") + .excludeCH("TruncDate") + .excludeCH("unsupported fmt fields for trunc/date_trunc results null") + .excludeCH("to_utc_timestamp") + .excludeCH("from_utc_timestamp") + .excludeCH("SPARK-31896: Handle am-pm timestamp parsing when hour is missing") + .excludeCH("UNIX_SECONDS") + .excludeCH("TIMESTAMP_SECONDS") + enableSuite[GlutenDateFunctionsSuite] + // The below two are replaced by two modified versions. + .exclude("unix_timestamp") + .exclude("to_unix_timestamp") + // Unsupported datetime format: specifier X is not supported by velox. + .exclude("to_timestamp with microseconds precision") + // Legacy mode is not supported, assuming this mode is not commonly used. + .exclude("SPARK-30668: use legacy timestamp parser in to_timestamp") + // Legacy mode is not supported and velox getTimestamp function does not throw + // exception when format is "yyyy-dd-aa". + .exclude("function to_date") + .excludeCH("SPARK-30766: date_trunc of old timestamps to hours and days") + .excludeCH("SPARK-30793: truncate timestamps before the epoch to seconds and minutes") + .excludeCH("try_to_timestamp") + .excludeCH("Gluten - to_unix_timestamp") + enableSuite[GlutenDecimalExpressionSuite] + enableSuite[GlutenDecimalPrecisionSuite] + enableSuite[GlutenDeleteFromTableSuite] + enableSuite[GlutenDeltaBasedDeleteFromTableSuite] + enableSuite[GlutenDeltaBasedMergeIntoTableSuite] + enableSuite[GlutenDeltaBasedMergeIntoTableUpdateAsDeleteAndInsertSuite] + enableSuite[GlutenDeltaBasedUpdateAsDeleteAndInsertTableSuite] + // FIXME: complex type result mismatch + .includeCH("update nested struct fields") + .includeCH("update char/varchar columns") + enableSuite[GlutenDeltaBasedUpdateTableSuite] + enableSuite[GlutenDeprecatedAPISuite] + enableSuite[GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuite] + .disable( + "DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type") + enableSuite[GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuiteAE] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOff] + .excludeGlutenTest("Subquery reuse across the whole plan") + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOffDisableScan] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOn] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOnDisableScan] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOff] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOffDisableScan] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOn] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOnDisableScan] + enableSuite[GlutenEmptyInSuite] + .excludeCH("IN with empty list") + enableSuite[GlutenEnsureRequirementsSuite] + enableSuite[GlutenExchangeSuite] + // ColumnarShuffleExchangeExec does not support doExecute() method + .exclude("shuffling UnsafeRows in exchange") + // ColumnarShuffleExchangeExec does not support SORT_BEFORE_REPARTITION + .exclude("SPARK-23207: Make repartition() generate consistent output") + // This test will re-run in GlutenExchangeSuite with shuffle partitions > 1 + .exclude("Exchange reuse across the whole plan") + enableSuite[GlutenExistenceJoinSuite] + .excludeCH("test single condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test single condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test single condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test single condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left semi join using BroadcastHashJoin (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left semi join using BroadcastHashJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left semi join using BroadcastNestedLoopJoin build left") + .excludeCH("test single unique condition (equal) for left semi join using BroadcastNestedLoopJoin build right (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left semi join using BroadcastNestedLoopJoin build right (whole-stage-codegen on)") + .excludeCH("test composed condition (equal & non-equal) for left semi join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test composed condition (equal & non-equal) for left semi join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test composed condition (equal & non-equal) for left semi join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test composed condition (equal & non-equal) for left semi join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("test single condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test single condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test single condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test single condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left anti join using BroadcastHashJoin (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left anti join using BroadcastHashJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("test single unique condition (equal) for left anti join using BroadcastNestedLoopJoin build left") + .excludeCH("test single unique condition (equal) for left anti join using BroadcastNestedLoopJoin build right (whole-stage-codegen off)") + .excludeCH("test single unique condition (equal) for left anti join using BroadcastNestedLoopJoin build right (whole-stage-codegen on)") + .excludeCH("test composed condition (equal & non-equal) test for left anti join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test composed condition (equal & non-equal) test for left anti join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test composed condition (equal & non-equal) test for left anti join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test composed condition (equal & non-equal) test for left anti join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("test composed unique condition (both non-equal) for left anti join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("test composed unique condition (both non-equal) for left anti join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("test composed unique condition (both non-equal) for left anti join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("test composed unique condition (both non-equal) for left anti join using SortMergeJoin (whole-stage-codegen on)") + enableSuite[GlutenExpressionsSchemaSuite] + enableSuite[GlutenExternalCommandRunnerSuite] + enableSuite[GlutenExtraStrategiesSuite] + enableSuite[GlutenFallbackSuite] + .excludeCH("Gluten - test fallback event") + enableSuite[GlutenFileBasedDataSourceSuite] + // test data path is jar path, rewrite + .exclude("Option recursiveFileLookup: disable partition inferring") + // gluten executor exception cannot get in driver, rewrite + .exclude("Spark native readers should respect spark.sql.caseSensitive - parquet") + // shuffle_partitions config is different, rewrite + .excludeByPrefix("SPARK-22790") + // plan is different cause metric is different, rewrite + .excludeByPrefix("SPARK-25237") + // error msg from velox is different & reader options is not supported, rewrite + .exclude("Enabling/disabling ignoreMissingFiles using parquet") + .exclude("Enabling/disabling ignoreMissingFiles using orc") + .exclude("Spark native readers should respect spark.sql.caseSensitive - orc") + .includeCH("Return correct results when data columns overlap with partition columns") + .includeCH("Return correct results when data columns overlap with partition " + + "columns (nested data)") + .exclude("SPARK-31116: Select nested schema with case insensitive mode") + // exclude as original metric not correct when task offloaded to velox + .exclude("SPARK-37585: test input metrics for DSV2 with output limits") + // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .exclude("File source v2: support passing data filters to FileScan without partitionFilters") + // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .exclude("File source v2: support partition pruning") + // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .exclude("SPARK-41017: filter pushdown with nondeterministic predicates") + .excludeCH("SPARK-23072 Write and read back unicode column names - csv") + .excludeCH("Enabling/disabling ignoreMissingFiles using csv") + .excludeCH("SPARK-30362: test input metrics for DSV2") + .excludeCH("SPARK-35669: special char in CSV header with filter pushdown") + .excludeCH("Gluten - Spark native readers should respect spark.sql.caseSensitive - parquet") + .excludeCH("Gluten - SPARK-25237 compute correct input metrics in FileScanRDD") + .excludeCH("Gluten - Enabling/disabling ignoreMissingFiles using orc") + .excludeCH("Gluten - Enabling/disabling ignoreMissingFiles using parquet") + enableSuite[GlutenFileDataSourceV2FallBackSuite] + // Rewritten + .exclude("Fallback Parquet V2 to V1") + enableSuite[GlutenFileFormatWriterSuite] + // TODO: fix "empty file should be skipped while write to file" + .excludeCH("empty file should be skipped while write to file") + enableSuite[GlutenFileIndexSuite] + enableSuite[GlutenFileMetadataStructSuite] + enableSuite[GlutenFileScanSuite] + enableSuite[GlutenFileSourceCharVarcharTestSuite] + .includeCH("length check for input string values: nested in array") + .includeCH("length check for input string values: nested in array") + .includeCH("length check for input string values: nested in map key") + .includeCH("length check for input string values: nested in map value") + .includeCH("length check for input string values: nested in both map key and value") + .includeCH("length check for input string values: nested in array of struct") + .includeCH("length check for input string values: nested in array of array") + enableSuite[GlutenFileSourceCustomMetadataStructSuite] + enableSuite[GlutenFileSourceSQLInsertTestSuite] + .excludeCH("SPARK-33474: Support typed literals as partition spec values") + .excludeCH( + "SPARK-34556: checking duplicate static partition columns should respect case sensitive conf") + enableSuite[GlutenFileSourceStrategySuite] + // Plan comparison. + .exclude("partitioned table - after scan filters") + .excludeCH("unpartitioned table, single partition") + .excludeCH("SPARK-32019: Add spark.sql.files.minPartitionNum config") + .excludeCH( + "SPARK-32352: Partially push down support data filter if it mixed in partition filters") + .excludeCH("SPARK-44021: Test spark.sql.files.maxPartitionNum works as expected") + enableSuite[GlutenFileTableSuite] + enableSuite[GlutenFilteredScanSuite] + enableSuite[GlutenFiltersSuite] + enableSuite[GlutenGeneratorFunctionSuite] + .exclude("SPARK-45171: Handle evaluated nondeterministic expression") + .excludeCH("single explode_outer") + .excludeCH("single posexplode_outer") + .excludeCH("explode_outer and other columns") + .excludeCH("aliased explode_outer") + .excludeCH("explode_outer on map") + .excludeCH("explode_outer on map with aliases") + .excludeCH("SPARK-40963: generator output has correct nullability") + .excludeCH("Gluten - SPARK-45171: Handle evaluated nondeterministic expression") + enableSuite[GlutenGroupBasedDeleteFromTableSuite] + enableSuite[GlutenGroupBasedMergeIntoTableSuite] + enableSuite[GlutenHadoopFileLinesReaderSuite] + enableSuite[GlutenHashExpressionsSuite] + .excludeCH("sha2") + .excludeCH("SPARK-30633: xxHash with different type seeds") + enableSuite[GlutenHeaderCSVReadSchemaSuite] + .excludeCH("append column at the end") + .excludeCH("hide column at the end") + .excludeCH("change column type from byte to short/int/long") + .excludeCH("change column type from short to int/long") + .excludeCH("change column type from int to long") + .excludeCH("read byte, int, short, long together") + .excludeCH("change column type from float to double") + .excludeCH("read float and double together") + .excludeCH("change column type from float to decimal") + .excludeCH("change column type from double to decimal") + .excludeCH("read float, double, decimal together") + .excludeCH("read as string") + enableSuite[GlutenHigherOrderFunctionsSuite] + .excludeCH("ArraySort") + .excludeCH("ArrayAggregate") + .excludeCH("TransformKeys") + .excludeCH("TransformValues") + .excludeCH("SPARK-39419: ArraySort should throw an exception when the comparator returns null") + enableSuite[GlutenHiveSQLQueryCHSuite] + enableSuite[GlutenInjectRuntimeFilterSuite] + // FIXME: yan + .includeCH("Merge runtime bloom filters") + .excludeGlutenTest("GLUTEN-9849: bloom filter applied to partition filter") + enableSuite[GlutenInnerJoinSuiteForceShjOff] + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("inner join, one match per row using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("inner join, one match per row using SortMergeJoin (whole-stage-codegen on)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("inner join, multiple matches using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("inner join, multiple matches using SortMergeJoin (whole-stage-codegen on)") + .excludeCH( + "inner join, null safe using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH("inner join, null safe using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH( + "inner join, null safe using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH( + "inner join, null safe using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("inner join, null safe using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("inner join, null safe using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using CartesianProduct") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build left (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build left (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build right (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build right (whole-stage-codegen on)") + enableSuite[GlutenInnerJoinSuiteForceShjOn] + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH( + "inner join, one match per row using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("inner join, one match per row using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("inner join, one match per row using SortMergeJoin (whole-stage-codegen on)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH( + "inner join, multiple matches using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("inner join, multiple matches using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("inner join, multiple matches using SortMergeJoin (whole-stage-codegen on)") + .excludeCH( + "inner join, null safe using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH("inner join, null safe using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH( + "inner join, null safe using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH( + "inner join, null safe using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("inner join, null safe using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("inner join, null safe using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using CartesianProduct") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build left (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build left (whole-stage-codegen on)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build right (whole-stage-codegen off)") + .excludeCH("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build right (whole-stage-codegen on)") + enableSuite[GlutenInsertSuite] + // the native write staing dir is differnt with vanilla Spark for coustom partition paths + .exclude("SPARK-35106: Throw exception when rename custom partition paths returns false") + .exclude("Stop task set if FileAlreadyExistsException was thrown") + // Rewrite: Additional support for file scan with default values has been added in Spark-3.4. + // It appends the default value in record if it is not present while scanning. + // Velox supports default values for new records but it does not backfill the + // existing records and provides null for the existing ones. + .exclude("INSERT rows, ALTER TABLE ADD COLUMNS with DEFAULTs, then SELECT them") + .exclude("SPARK-39557 INSERT INTO statements with tables with array defaults") + .exclude("SPARK-39557 INSERT INTO statements with tables with struct defaults") + .exclude("SPARK-39557 INSERT INTO statements with tables with map defaults") + .excludeCH("Gluten - insert partition table") + .excludeCH("Gluten - remove v1writes sort and project") + .excludeCH("Gluten - remove v1writes sort") + .excludeCH("Gluten - do not remove non-v1writes sort and project") + .excludeCH( + "Gluten - SPARK-35106: Throw exception when rename custom partition paths returns false") + .excludeCH( + "Gluten - Do not fallback write files if output columns contain Spark internal metadata") + .excludeCH("Gluten - Add metadata white list to allow native write files") + .excludeCH("Gluten - INSERT rows, ALTER TABLE ADD COLUMNS with DEFAULTs, then SELECT them") + enableSuite[GlutenIntervalExpressionsSuite] + enableSuite[GlutenIntervalFunctionsSuite] + enableSuite[GlutenJoinSuite] + // exclude as it check spark plan + .exclude("SPARK-36794: Ignore duplicated key when building relation for semi/anti hash join") + .excludeCH( + "SPARK-45882: BroadcastHashJoinExec propagate partitioning should respect CoalescedHashPartitioning") + enableSuite[GlutenJsonExpressionsSuite] + // https://github.com/apache/incubator-gluten/issues/8102 + .includeCH("$.store.book") + .includeCH("$") + .includeCH("$.store.book[0]") + .includeCH("$.store.book[*]") + .includeCH("$.store.book[*].category") + .includeCH("$.store.book[*].isbn") + .includeCH("$.store.book[*].reader") + .includeCH("$.store.basket[*]") + .includeCH("$.store.basket[*][0]") + .includeCH("$.store.basket[0][*]") + .includeCH("$.store.basket[*][*]") + .exclude("$.store.basket[0][*].b") + // Exception class different. + .exclude("from_json - invalid data") + .excludeCH("from_json - input=object, schema=array, output=array of single row") + .excludeCH("from_json - input=empty object, schema=array, output=array of single row with null") + .excludeCH("from_json - input=array of single object, schema=struct, output=single row") + .excludeCH("from_json - input=array, schema=struct, output=single row") + .excludeCH("from_json - input=empty array, schema=struct, output=single row with null") + .excludeCH("from_json - input=empty object, schema=struct, output=single row with null") + .excludeCH("SPARK-20549: from_json bad UTF-8") + .excludeCH("from_json with timestamp") + .excludeCH("to_json - struct") + .excludeCH("to_json - array") + .excludeCH("to_json - array with single empty row") + .excludeCH("to_json with timestamp") + .excludeCH("SPARK-21513: to_json support map[string, struct] to json") + .excludeCH("SPARK-21513: to_json support map[struct, struct] to json") + .excludeCH("parse date with locale") + .excludeCH("parse decimals using locale") + enableSuite[GlutenJsonFunctionsSuite] + // * in get_json_object expression not supported in velox + .exclude("SPARK-42782: Hive compatibility check for get_json_object") + // Velox does not support single quotes in get_json_object function. + .includeCH("function get_json_object - support single quotes") + .excludeCH("from_json with option (allowComments)") + .excludeCH("from_json with option (allowUnquotedFieldNames)") + .excludeCH("from_json with option (allowSingleQuotes)") + .excludeCH("from_json with option (allowNumericLeadingZeros)") + .excludeCH("from_json with option (allowBackslashEscapingAnyCharacter)") + .excludeCH("from_json with option (dateFormat)") + .excludeCH("from_json with option (allowUnquotedControlChars)") + .excludeCH("from_json with option (allowNonNumericNumbers)") + .excludeCH("from_json missing columns") + .excludeCH("from_json invalid json") + .excludeCH("from_json array support") + .excludeCH("to_json with option (timestampFormat)") + .excludeCH("to_json with option (dateFormat)") + .excludeCH("SPARK-19637 Support to_json in SQL") + .excludeCH("pretty print - roundtrip from_json -> to_json") + .excludeCH("from_json invalid json - check modes") + .excludeCH("SPARK-36069: from_json invalid json schema - check field name and field value") + .excludeCH("corrupt record column in the middle") + .excludeCH("parse timestamps with locale") + .excludeCH("SPARK-33134: return partial results only for root JSON objects") + .excludeCH("SPARK-40646: return partial results for JSON arrays with objects") + .excludeCH("SPARK-40646: return partial results for JSON maps") + .excludeCH("SPARK-40646: return partial results for objects with values as JSON arrays") + .excludeCH("SPARK-48863: parse object as an array with partial results enabled") + .excludeCH("SPARK-33907: bad json input with json pruning optimization: GetStructField") + .excludeCH("SPARK-33907: bad json input with json pruning optimization: GetArrayStructFields") + .excludeCH("SPARK-33907: json pruning optimization with corrupt record field") + enableSuite[GlutenJsonLegacyTimeParserSuite] + .includeCH("Complex field and type inferring") + .includeCH("SPARK-4228 DataFrame to JSON") + .excludeCH("SPARK-37360: Write and infer TIMESTAMP_NTZ values with a non-default pattern") + .excludeCH("SPARK-37360: Timestamp type inference for a column with TIMESTAMP_NTZ values") + .excludeCH("SPARK-36830: Support reading and writing ANSI intervals") + enableSuite[GlutenJsonReadSchemaSuite] + enableSuite[GlutenJsonV1Suite] + // FIXME: Array direct selection fails + .includeCH("Complex field and type inferring") + .includeCH("SPARK-4228 DataFrame to JSON") + .excludeCH("SPARK-37360: Write and infer TIMESTAMP_NTZ values with a non-default pattern") + .excludeCH("SPARK-37360: Timestamp type inference for a column with TIMESTAMP_NTZ values") + .excludeCH("SPARK-36830: Support reading and writing ANSI intervals") + enableSuite[GlutenJsonV2Suite] + // exception test + .exclude("SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern") + .includeCH("Complex field and type inferring") + .includeCH("SPARK-4228 DataFrame to JSON") + .excludeCH("SPARK-37360: Write and infer TIMESTAMP_NTZ values with a non-default pattern") + .excludeCH("SPARK-37360: Timestamp type inference for a column with TIMESTAMP_NTZ values") + .excludeCH("SPARK-36830: Support reading and writing ANSI intervals") + enableSuite[GlutenKeyGroupedPartitioningSuite] + // NEW SUITE: disable as they check vanilla spark plan + .exclude("partitioned join: number of buckets mismatch should trigger shuffle") + .exclude("partitioned join: only one side reports partitioning") + .exclude("partitioned join: join with two partition keys and different # of partition keys") + // disable due to check for SMJ node + .excludeByPrefix("SPARK-41413: partitioned join:") + .excludeByPrefix("SPARK-42038: partially clustered:") + .exclude("SPARK-44641: duplicated records when SPJ is not triggered") + .excludeCH("Gluten - partitioned join: only one side reports partitioning") + .excludeCH("Gluten - SPARK-41413: partitioned join: partition values from one side are subset of those from the other side") + .excludeCH("Gluten - SPARK-41413: partitioned join: partition values from both sides overlaps") + .excludeCH( + "Gluten - SPARK-41413: partitioned join: non-overlapping partition values from both sides") + .excludeCH("Gluten - SPARK-42038: partially clustered: with different partition keys and both sides partially clustered") + .excludeCH("Gluten - SPARK-42038: partially clustered: with different partition keys and missing keys on left-hand side") + .excludeCH("Gluten - SPARK-42038: partially clustered: with different partition keys and missing keys on right-hand side") + .excludeCH("Gluten - SPARK-42038: partially clustered: left outer join") + .excludeCH("Gluten - SPARK-42038: partially clustered: right outer join") + .excludeCH("Gluten - SPARK-42038: partially clustered: full outer join is not applicable") + .excludeCH("Gluten - SPARK-44641: duplicated records when SPJ is not triggered") + .excludeCH( + "Gluten - partitioned join: join with two partition keys and different # of partition keys") + enableSuite[GlutenLateralColumnAliasSuite] + .excludeCH("Lateral alias conflicts with table column - Project") + .excludeCH("Lateral alias conflicts with table column - Aggregate") + .excludeCH("Lateral alias of a complex type") + .excludeCH("Lateral alias reference works with having and order by") + .excludeCH("Lateral alias basics - Window on Project") + .excludeCH("Lateral alias basics - Window on Aggregate") + enableSuite[GlutenLiteralExpressionSuite] + .exclude("default") + // FIXME(yma11): ObjectType is not covered in RowEncoder/Serializer in vanilla spark + .exclude("SPARK-37967: Literal.create support ObjectType") + enableSuite[GlutenLocalBroadcastExchangeSuite] + .excludeCH("SPARK-39983 - Broadcasted relation is not cached on the driver") + enableSuite[GlutenLocalScanSuite] + enableSuite[GlutenMathExpressionsSuite] + // Spark round UT for round(3.1415,3) is not correct. + .exclude("round/bround/floor/ceil") + .excludeCH("tanh") + .excludeCH("unhex") + .excludeCH("atan2") + .excludeCH("SPARK-42045: integer overflow in round/bround") + .excludeCH("Gluten - round/bround/floor/ceil") + enableSuite[GlutenMathFunctionsSuite] + enableSuite[GlutenMergedOrcReadSchemaSuite] + .includeCH("append column into middle") + .includeCH("add a nested column at the end of the leaf struct column") + .includeCH("add a nested column in the middle of the leaf struct column") + .includeCH("add a nested column at the end of the middle struct column") + .includeCH("add a nested column in the middle of the middle struct column") + .includeCH("hide a nested column at the end of the leaf struct column") + .includeCH("hide a nested column in the middle of the leaf struct column") + .includeCH("hide a nested column at the end of the middle struct column") + .includeCH("hide a nested column in the middle of the middle struct column") + .includeCH("change column type from boolean to byte/short/int/long") + .includeCH("change column type from byte to short/int/long") + .includeCH("change column type from short to int/long") + .includeCH("change column type from int to long") + .includeCH("read byte, int, short, long together") + .includeCH("change column type from float to double") + .includeCH("read float and double together") + enableSuite[GlutenMergedParquetReadSchemaSuite] + enableSuite[GlutenMetadataCacheSuite] + .exclude("SPARK-16336,SPARK-27961 Suggest fixing FileNotFoundException") + enableSuite[GlutenMetadataColumnSuite] + .excludeCH("SPARK-34923: propagate metadata columns through Sort") + .excludeCH("SPARK-34923: propagate metadata columns through RepartitionBy") + .excludeCH("SPARK-40149: select outer join metadata columns with DataFrame API") + .excludeCH("SPARK-42683: Project a metadata column by its logical name - column not found") + enableSuite[GlutenMiscExpressionsSuite] + enableSuite[GlutenMiscFunctionsSuite] + enableSuite[GlutenNestedDataSourceV1Suite] + enableSuite[GlutenNestedDataSourceV2Suite] + enableSuite[GlutenNondeterministicSuite] + .exclude("MonotonicallyIncreasingID") + .exclude("SparkPartitionID") + enableSuite[GlutenNullExpressionsSuite] + enableSuite[GlutenOrcCodecSuite] + enableSuite[GlutenOrcColumnarBatchReaderSuite] + enableSuite[GlutenOrcFilterSuite] + .exclude("SPARK-32622: case sensitivity in predicate pushdown") + enableSuite[GlutenOrcPartitionDiscoverySuite] + .includeCH("read partitioned table - normal case") + .includeCH("read partitioned table - with nulls") + enableSuite[GlutenOrcReadSchemaSuite] + .includeCH("append column into middle") + .includeCH("hide column in the middle") + .includeCH("change column position") + .exclude("change column type from boolean to byte/short/int/long") + .exclude("read as string") + .exclude("change column type from byte to short/int/long") + .exclude("change column type from short to int/long") + .exclude("change column type from int to long") + .exclude("read byte, int, short, long together") + .exclude("change column type from float to double") + .exclude("read float and double together") + .exclude("change column type from float to decimal") + .exclude("change column type from double to decimal") + .exclude("read float, double, decimal together") + .includeCH("add a nested column at the end of the leaf struct column") + .includeCH("add a nested column in the middle of the leaf struct column") + .includeCH("add a nested column at the end of the middle struct column") + .includeCH("add a nested column in the middle of the middle struct column") + .includeCH("hide a nested column at the end of the leaf struct column") + .includeCH("hide a nested column in the middle of the leaf struct column") + .includeCH("hide a nested column at the end of the middle struct column") + .includeCH("hide a nested column in the middle of the middle struct column") + enableSuite[GlutenOrcSourceSuite] + // Rewrite to disable Spark's columnar reader. + .exclude("SPARK-31238: compatibility with Spark 2.4 in reading dates") + .exclude("SPARK-31238, SPARK-31423: rebasing dates in write") + .exclude("SPARK-31284: compatibility with Spark 2.4 in reading timestamps") + .exclude("SPARK-31284, SPARK-31423: rebasing timestamps in write") + .includeCH("SPARK-34862: Support ORC vectorized reader for nested column") + // Ignored to disable vectorized reading check. + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + .includeCH("create temporary orc table") + .includeCH("create temporary orc table as") + .includeCH("appending insert") + .includeCH("overwrite insert") + .includeCH("SPARK-34897: Support reconcile schemas based on index after nested column pruning") + .excludeGlutenTest("SPARK-31238: compatibility with Spark 2.4 in reading dates") + .excludeGlutenTest("SPARK-31238, SPARK-31423: rebasing dates in write") + .excludeGlutenTest("SPARK-34862: Support ORC vectorized reader for nested column") + // exclude as struct not supported + .includeCH("SPARK-36663: OrcUtils.toCatalystSchema should correctly handle a column name which consists of only numbers") + .includeCH("SPARK-37812: Reuse result row when deserializing a struct") + // rewrite + .exclude("SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=true, spark.sql.orc.enableNestedColumnVectorizedReader=true)") + .exclude("SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=true, spark.sql.orc.enableNestedColumnVectorizedReader=false)") + .excludeCH("SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=false, spark.sql.orc.enableNestedColumnVectorizedReader=true)") + .excludeCH("SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=false, spark.sql.orc.enableNestedColumnVectorizedReader=false)") + .excludeCH("Gluten - SPARK-31284: compatibility with Spark 2.4 in reading timestamps") + .excludeCH("Gluten - SPARK-31284, SPARK-31423: rebasing timestamps in write") + .excludeCH("Gluten - SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=false, spark.sql.orc.enableNestedColumnVectorizedReader=false)") + enableSuite[GlutenOrcV1FilterSuite] + .exclude("SPARK-32622: case sensitivity in predicate pushdown") + enableSuite[GlutenOrcV1AggregatePushDownSuite] + .includeCH("nested column: Count(nested sub-field) not push down") + enableSuite[GlutenOrcV1PartitionDiscoverySuite] + .includeCH("read partitioned table - normal case") + .includeCH("read partitioned table - with nulls") + .includeCH("read partitioned table - partition key included in orc file") + .includeCH("read partitioned table - with nulls and partition keys are included in Orc file") + enableSuite[GlutenOrcV1QuerySuite] + // Rewrite to disable Spark's columnar reader. + .includeCH("Simple selection form ORC table") + .includeCH("simple select queries") + .includeCH("overwriting") + .includeCH("self-join") + .includeCH("columns only referenced by pushed down filters should remain") + .includeCH("SPARK-5309 strings stored using dictionary compression in orc") + // For exception test. + .exclude("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") + .includeCH("Read/write binary data") + .includeCH("Read/write all types with non-primitive type") + .includeCH("Creating case class RDD table") + .includeCH("save and load case class RDD with `None`s as orc") + .includeCH("SPARK-16610: Respect orc.compress (i.e., OrcConf.COMPRESS) when" + + " compression is unset") + .includeCH("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") + .includeCH("appending") + .includeCH("nested data - struct with array field") + .includeCH("nested data - array of struct") + .includeCH("SPARK-9170: Don't implicitly lowercase of user-provided columns") + .includeCH("SPARK-10623 Enable ORC PPD") + .includeCH("SPARK-14962 Produce correct results on array type with isnotnull") + .includeCH("SPARK-15198 Support for pushing down filters for boolean types") + .includeCH("Support for pushing down filters for decimal types") + .includeCH("Support for pushing down filters for timestamp types") + .includeCH("column nullability and comment - write and then read") + .includeCH("Empty schema does not read data from ORC file") + .includeCH("read from multiple orc input paths") + .exclude("Enabling/disabling ignoreCorruptFiles") + .includeCH("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") + .includeCH("LZO compression options for writing to an ORC file") + .includeCH("Schema discovery on empty ORC files") + .includeCH("SPARK-21791 ORC should support column names with dot") + .includeCH("SPARK-25579 ORC PPD should support column names with dot") + .exclude("SPARK-34862: Support ORC vectorized reader for nested column") + .includeCH("SPARK-37728: Reading nested columns with ORC vectorized reader should not") + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + .includeCH("Read/write all timestamp types") + .includeCH("SPARK-37463: read/write Timestamp ntz to Orc with different time zone") + .includeCH("SPARK-39381: Make vectorized orc columar writer batch size configurable") + .includeCH("SPARK-39830: Reading ORC table that requires type promotion may throw AIOOBE") + .excludeCH("SPARK-37728: Reading nested columns with ORC vectorized reader should not cause ArrayIndexOutOfBoundsException") + enableSuite[GlutenOrcV1SchemaPruningSuite] + .includeCH( + "Spark vectorized reader - without partition data column - select only top-level fields") + .includeCH( + "Spark vectorized reader - with partition data column - select only top-level fields") + .includeCH("Spark vectorized reader - " + + "without partition data column - select one deep nested complex field after join") + .includeCH("Spark vectorized reader - " + + "with partition data column - select one deep nested complex field after join") + .includeCH("Spark vectorized reader - " + + "without partition data column - select one deep nested complex field after outer join") + .includeCH("Spark vectorized reader - " + + "with partition data column - select one deep nested complex field after outer join") + // Vectorized reading. + .includeCH("Spark vectorized reader - without partition data column - " + + "select only expressions without references") + .includeCH("Spark vectorized reader - with partition data column - " + + "select only expressions without references") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .includeCH( + "Spark vectorized reader - without partition data column - select a single complex field") + .includeCH( + "Spark vectorized reader - with partition data column - select a single complex field") + .includeCH( + "Non-vectorized reader - without partition data column - select a single complex field") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field and its parent struct") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field and its parent struct") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field and its parent struct") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field and its parent struct") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field and the partition column") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field and the partition column") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field and the partition column") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field and the partition column") + .includeCH("Spark vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .includeCH("Spark vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .includeCH("Non-vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .includeCH("Non-vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .includeCH( + "Spark vectorized reader - without partition data column - no unnecessary schema pruning") + .includeCH( + "Spark vectorized reader - with partition data column - no unnecessary schema pruning") + .includeCH( + "Non-vectorized reader - without partition data column - no unnecessary schema pruning") + .includeCH("Non-vectorized reader - with partition data column - no unnecessary schema pruning") + .includeCH( + "Spark vectorized reader - without partition data column - empty schema intersection") + .includeCH("Spark vectorized reader - with partition data column - empty schema intersection") + .includeCH("Non-vectorized reader - without partition data column - empty schema intersection") + .includeCH("Non-vectorized reader - with partition data column - empty schema intersection") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field and is null expression in project") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field and is null expression in project") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field and is null expression in project") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field and is null expression in project") + .includeCH("Spark vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .includeCH("Spark vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .includeCH("Non-vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .includeCH("Non-vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .includeCH("Spark vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .includeCH("Spark vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .includeCH("Non-vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .includeCH("Non-vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .includeCH("Spark vectorized reader - without partition data column - select explode of nested field of array of struct") + .includeCH("Spark vectorized reader - with partition data column - select explode of nested field of array of struct") + .includeCH("Non-vectorized reader - without partition data column - select explode of nested field of array of struct") + .includeCH("Non-vectorized reader - with partition data column - select explode of nested field of array of struct") + .includeCH("Non-vectorized reader - without partition data column - select one deep nested complex field after join") + .includeCH("Non-vectorized reader - with partition data column - select one deep nested complex field after join") + .includeCH("Non-vectorized reader - without partition data column - select one deep nested complex field after outer join") + .includeCH("Non-vectorized reader - with partition data column - select one deep nested complex field after outer join") + .includeCH("Spark vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Spark vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Non-vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Non-vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Spark vectorized reader - without partition data column - select nested field in window function") + .includeCH("Spark vectorized reader - with partition data column - select nested field in window function") + .includeCH("Non-vectorized reader - without partition data column - select nested field in window function") + .includeCH( + "Non-vectorized reader - with partition data column - select nested field in window function") + .includeCH("Spark vectorized reader - without partition data column - select nested field in window function and then order by") + .includeCH("Spark vectorized reader - with partition data column - select nested field in window function and then order by") + .includeCH("Non-vectorized reader - without partition data column - select nested field in window function and then order by") + .includeCH("Non-vectorized reader - with partition data column - select nested field in window function and then order by") + .includeCH( + "Spark vectorized reader - without partition data column - select nested field in Sort") + .includeCH("Spark vectorized reader - with partition data column - select nested field in Sort") + .includeCH( + "Non-vectorized reader - without partition data column - select nested field in Sort") + .includeCH("Non-vectorized reader - with partition data column - select nested field in Sort") + .includeCH( + "Spark vectorized reader - without partition data column - select nested field in Expand") + .includeCH( + "Spark vectorized reader - with partition data column - select nested field in Expand") + .includeCH( + "Non-vectorized reader - without partition data column - select nested field in Expand") + .includeCH("Non-vectorized reader - with partition data column - select nested field in Expand") + .includeCH("Spark vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Spark vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Non-vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Non-vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Case-sensitive parser - mixed-case schema - select with exact column names") + .exclude("Case-insensitive parser - mixed-case schema - select with exact column names") + .exclude("Case-insensitive parser - mixed-case schema - select with lowercase column names") + .exclude( + "Case-insensitive parser - mixed-case schema - select with different-case column names") + .exclude("Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") + .includeCH("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .includeCH("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .includeCH("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .includeCH("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("SPARK-36352: Spark should check result plan's output schema name") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .excludeCH( + "Case-insensitive parser - mixed-case schema - filter with different-case column names") + .excludeCH("Spark vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Spark vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + enableSuite[GlutenOrcV2AggregatePushDownSuite] + .includeCH("nested column: Max(top level column) not push down") + .includeCH("nested column: Count(nested sub-field) not push down") + enableSuite[GlutenOrcV2QuerySuite] + .includeCH("Read/write binary data") + .includeCH("Read/write all types with non-primitive type") + // Rewrite to disable Spark's columnar reader. + .includeCH("Simple selection form ORC table") + .includeCH("Creating case class RDD table") + .includeCH("save and load case class RDD with `None`s as orc") + .includeCH( + "SPARK-16610: Respect orc.compress (i.e., OrcConf.COMPRESS) when compression is unset") + .includeCH("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") + .includeCH("appending") + .includeCH("nested data - struct with array field") + .includeCH("nested data - array of struct") + .includeCH("SPARK-9170: Don't implicitly lowercase of user-provided columns") + .includeCH("SPARK-10623 Enable ORC PPD") + .includeCH("SPARK-14962 Produce correct results on array type with isnotnull") + .includeCH("SPARK-15198 Support for pushing down filters for boolean types") + .includeCH("Support for pushing down filters for decimal types") + .includeCH("Support for pushing down filters for timestamp types") + .includeCH("column nullability and comment - write and then read") + .includeCH("Empty schema does not read data from ORC file") + .includeCH("read from multiple orc input paths") + .exclude("Enabling/disabling ignoreCorruptFiles") + .includeCH("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") + .includeCH("LZO compression options for writing to an ORC file") + .includeCH("Schema discovery on empty ORC files") + .includeCH("SPARK-21791 ORC should support column names with dot") + .includeCH("SPARK-25579 ORC PPD should support column names with dot") + .exclude("SPARK-34862: Support ORC vectorized reader for nested column") + .includeCH("SPARK-37728: Reading nested columns with ORC vectorized reader should not") + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + .includeCH("Read/write all timestamp types") + .includeCH("SPARK-37463: read/write Timestamp ntz to Orc with different time zone") + .includeCH("SPARK-39381: Make vectorized orc columar writer batch size configurable") + .includeCH("SPARK-39830: Reading ORC table that requires type promotion may throw AIOOBE") + .includeCH("simple select queries") + .includeCH("overwriting") + .includeCH("self-join") + .includeCH("columns only referenced by pushed down filters should remain") + .includeCH("SPARK-5309 strings stored using dictionary compression in orc") + // For exception test. + .exclude("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") + .excludeCH("SPARK-37728: Reading nested columns with ORC vectorized reader should not cause ArrayIndexOutOfBoundsException") + enableSuite[GlutenOrcV2SchemaPruningSuite] + .includeCH( + "Spark vectorized reader - without partition data column - select only top-level fields") + .includeCH( + "Spark vectorized reader - with partition data column - select only top-level fields") + .includeCH("Spark vectorized reader - " + + "without partition data column - select one deep nested complex field after join") + .includeCH("Spark vectorized reader - " + + "with partition data column - select one deep nested complex field after join") + .includeCH("Spark vectorized reader - " + + "without partition data column - select one deep nested complex field after outer join") + .includeCH("Spark vectorized reader - " + + "with partition data column - select one deep nested complex field after outer join") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .includeCH( + "Spark vectorized reader - without partition data column - select a single complex field") + .includeCH( + "Spark vectorized reader - with partition data column - select a single complex field") + .includeCH( + "Non-vectorized reader - without partition data column - select a single complex field") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field and its parent struct") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field and its parent struct") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field and its parent struct") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field and its parent struct") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field and the partition column") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field and the partition column") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field and the partition column") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field and the partition column") + .includeCH("Spark vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .includeCH("Spark vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .includeCH("Non-vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .includeCH("Non-vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .includeCH( + "Spark vectorized reader - without partition data column - no unnecessary schema pruning") + .includeCH( + "Spark vectorized reader - with partition data column - no unnecessary schema pruning") + .includeCH( + "Non-vectorized reader - without partition data column - no unnecessary schema pruning") + .includeCH("Non-vectorized reader - with partition data column - no unnecessary schema pruning") + .includeCH( + "Spark vectorized reader - without partition data column - empty schema intersection") + .includeCH("Spark vectorized reader - with partition data column - empty schema intersection") + .includeCH("Non-vectorized reader - without partition data column - empty schema intersection") + .includeCH("Non-vectorized reader - with partition data column - empty schema intersection") + .includeCH("Spark vectorized reader - without partition data column - select a single complex field and is null expression in project") + .includeCH("Spark vectorized reader - with partition data column - select a single complex field and is null expression in project") + .includeCH("Non-vectorized reader - without partition data column - select a single complex field and is null expression in project") + .includeCH("Non-vectorized reader - with partition data column - select a single complex field and is null expression in project") + .includeCH("Spark vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .includeCH("Spark vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .includeCH("Non-vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .includeCH("Non-vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .includeCH("Spark vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .includeCH("Spark vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .includeCH("Non-vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .includeCH("Non-vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .includeCH("Spark vectorized reader - without partition data column - select explode of nested field of array of struct") + .includeCH("Spark vectorized reader - with partition data column - select explode of nested field of array of struct") + .includeCH("Non-vectorized reader - without partition data column - select explode of nested field of array of struct") + .includeCH("Non-vectorized reader - with partition data column - select explode of nested field of array of struct") + .includeCH("Non-vectorized reader - without partition data column - select one deep nested complex field after join") + .includeCH("Non-vectorized reader - with partition data column - select one deep nested complex field after join") + .includeCH("Non-vectorized reader - without partition data column - select one deep nested complex field after outer join") + .includeCH("Non-vectorized reader - with partition data column - select one deep nested complex field after outer join") + .includeCH("Spark vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Spark vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Non-vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Non-vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .includeCH("Spark vectorized reader - without partition data column - select nested field in window function") + .includeCH("Spark vectorized reader - with partition data column - select nested field in window function") + .includeCH("Non-vectorized reader - without partition data column - select nested field in window function") + .includeCH( + "Non-vectorized reader - with partition data column - select nested field in window function") + .includeCH("Spark vectorized reader - without partition data column - select nested field in window function and then order by") + .includeCH("Spark vectorized reader - with partition data column - select nested field in window function and then order by") + .includeCH("Non-vectorized reader - without partition data column - select nested field in window function and then order by") + .includeCH("Non-vectorized reader - with partition data column - select nested field in window function and then order by") + .includeCH( + "Spark vectorized reader - without partition data column - select nested field in Sort") + .includeCH("Spark vectorized reader - with partition data column - select nested field in Sort") + .includeCH( + "Non-vectorized reader - without partition data column - select nested field in Sort") + .includeCH("Non-vectorized reader - with partition data column - select nested field in Sort") + .includeCH( + "Spark vectorized reader - without partition data column - select nested field in Expand") + .includeCH( + "Spark vectorized reader - with partition data column - select nested field in Expand") + .includeCH( + "Non-vectorized reader - without partition data column - select nested field in Expand") + .includeCH("Non-vectorized reader - with partition data column - select nested field in Expand") + .includeCH("Spark vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Spark vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Non-vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Non-vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .includeCH("Case-sensitive parser - mixed-case schema - select with exact column names") + .exclude("Case-insensitive parser - mixed-case schema - select with exact column names") + .exclude("Case-insensitive parser - mixed-case schema - select with lowercase column names") + .exclude( + "Case-insensitive parser - mixed-case schema - select with different-case column names") + .exclude("Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") + .includeCH("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .includeCH("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .includeCH("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .includeCH("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .includeCH("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("SPARK-36352: Spark should check result plan's output schema name") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .includeCH("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .includeCH("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .includeCH("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .includeCH("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .excludeCH("Spark vectorized reader - without partition data column - select a single complex field and in where clause") + .excludeCH("Spark vectorized reader - with partition data column - select a single complex field and in where clause") + .excludeCH("Non-vectorized reader - without partition data column - select a single complex field and in where clause") + .excludeCH("Non-vectorized reader - with partition data column - select a single complex field and in where clause") + .excludeCH("Spark vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Spark vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Non-vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Non-vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Spark vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Spark vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Non-vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Non-vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Spark vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Spark vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Non-vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Non-vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .excludeCH( + "Case-insensitive parser - mixed-case schema - filter with different-case column names") + .excludeCH("SPARK-37450: Prunes unnecessary fields from Explode for count aggregation") + .excludeCH("Spark vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Spark vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + enableSuite[GlutenOuterJoinSuiteForceShjOff] + .excludeCH("basic left outer join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("basic left outer join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("basic left outer join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("basic left outer join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("basic right outer join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("basic right outer join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("basic right outer join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("basic right outer join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("basic full outer join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("basic full outer join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("basic full outer join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("basic full outer join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("left outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("left outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("left outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("left outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("right outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("right outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("right outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("right outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("full outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("full outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("full outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("full outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("SPARK-32717: AQEOptimizer should respect excludedRules configuration") + // TODO: fix the hanging problem in GLUTEN-8890 followup + .excludeCH("SPARK-46037: ShuffledHashJoin build left with left outer join, codegen off (whole-stage-codegen off)") + .excludeCH("SPARK-46037: ShuffledHashJoin build left with left outer join, codegen off (whole-stage-codegen on)") + enableSuite[GlutenOuterJoinSuiteForceShjOn] + .excludeCH("basic left outer join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("basic left outer join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("basic left outer join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("basic left outer join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("basic right outer join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("basic right outer join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("basic right outer join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("basic right outer join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("basic full outer join using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("basic full outer join using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("basic full outer join using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("basic full outer join using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("left outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("left outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("left outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("left outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("right outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("right outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("right outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("right outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + .excludeCH("full outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .excludeCH("full outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .excludeCH("full outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .excludeCH("full outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + // TODO: fix the hanging problem in GLUTEN-8890 followup + .excludeCH("SPARK-46037: ShuffledHashJoin build left with left outer join, codegen off (whole-stage-codegen off)") + .excludeCH("SPARK-46037: ShuffledHashJoin build left with left outer join, codegen off (whole-stage-codegen on)") + enableSuite[GlutenParametersSuite] + enableSuite[GlutenParquetCodecSuite] + // codec not supported in native + .includeCH("write and read - file source parquet - codec: lz4_raw") + .includeCH("write and read - file source parquet - codec: lz4raw") + enableSuite[GlutenParquetColumnIndexSuite] + // Rewrite by just removing test timestamp. + .exclude("test reading unaligned pages - test all types") + .excludeCH("test reading unaligned pages - test all types (dict encode)") + enableSuite[GlutenParquetCompressionCodecPrecedenceSuite] + enableSuite[GlutenParquetDeltaByteArrayEncodingSuite] + enableSuite[GlutenParquetDeltaEncodingInteger] + enableSuite[GlutenParquetDeltaEncodingLong] + enableSuite[GlutenParquetDeltaLengthByteArrayEncodingSuite] + enableSuite[GlutenParquetEncodingSuite] + // Velox does not support rle encoding, but it can pass when native writer enabled. + .includeCH("parquet v2 pages - rle encoding for boolean value columns") + .excludeCH("All Types Dictionary") + .excludeCH("All Types Null") + enableSuite[GlutenParquetFieldIdIOSuite] + enableSuite[GlutenParquetFileFormatV1Suite] + .excludeCH( + "SPARK-36825, SPARK-36854: year-month/day-time intervals written and read as INT32/INT64") + enableSuite[GlutenParquetFileFormatV2Suite] + .excludeCH( + "SPARK-36825, SPARK-36854: year-month/day-time intervals written and read as INT32/INT64") + enableSuite[GlutenParquetFileMetadataStructRowIndexSuite] + enableSuite[GlutenParquetIOSuite] + // Velox doesn't write file metadata into parquet file. + .includeCH("Write Spark version into Parquet metadata") + // Exception. + .exclude("SPARK-35640: read binary as timestamp should throw schema incompatible error") + // Exception msg. + .exclude("SPARK-35640: int as long should throw schema incompatible error") + // Velox parquet reader not allow offset zero. + .includeCH("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") + enableSuite[GlutenParquetInteroperabilitySuite] + .exclude("parquet timestamp conversion") + enableSuite[GlutenParquetProtobufCompatibilitySuite] + .excludeCH("struct with unannotated array") + enableSuite[GlutenParquetReadSchemaSuite] + enableSuite[GlutenParquetRebaseDatetimeV1Suite] + // Velox doesn't write file metadata into parquet file. + .excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys") + .excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key") + // jar path and ignore PARQUET_REBASE_MODE_IN_READ, rewrite some + .excludeByPrefix("SPARK-31159") + .excludeByPrefix("SPARK-35427") + .excludeCH("Gluten - SPARK-31159: rebasing dates in write") + enableSuite[GlutenParquetRebaseDatetimeV2Suite] + // Velox doesn't write file metadata into parquet file. + .excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys") + .excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key") + // jar path and ignore PARQUET_REBASE_MODE_IN_READ + .excludeByPrefix("SPARK-31159") + .excludeByPrefix("SPARK-35427") + enableSuite[GlutenParquetRowIndexSuite] + .excludeByPrefix("row index generation") + .excludeByPrefix("invalid row index column type") + enableSuite[GlutenParquetSchemaInferenceSuite] + enableSuite[GlutenParquetSchemaSuite] + // error message mismatch is accepted + .exclude("schema mismatch failure error message for parquet reader") + .exclude("schema mismatch failure error message for parquet vectorized reader") + // [PATH_NOT_FOUND] Path does not exist: + // file:/opt/spark331/sql/core/src/test/resources/test-data/timestamp-nanos.parquet + // May require for newer spark.test.home + .excludeByPrefix("SPARK-40819") + enableSuite[GlutenParquetThriftCompatibilitySuite] + // Rewrite for file locating. + .exclude("Read Parquet file generated by parquet-thrift") + .excludeCH("SPARK-10136 list of primitive list") + enableSuite[GlutenParquetV1AggregatePushDownSuite] + enableSuite[GlutenParquetV1FilterSuite] + // Rewrite. + .includeCH("SPARK-23852: Broken Parquet push-down for partially-written stats") + // Rewrite for supported INT96 - timestamp. + .exclude("filter pushdown - timestamp") + .exclude("filter pushdown - date") + // Exception bebaviour. + .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") + // Ignore Spark's filter pushdown check. + .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") + .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") + .exclude("Filters should be pushed down for Parquet readers at row group level") + .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") + .exclude("Support Parquet column index") + .exclude("SPARK-34562: Bloom filter push down") + .includeCH("SPARK-16371 Do not push down filters when inner name and outer name are the same") + .exclude("filter pushdown - StringPredicate") + .excludeCH("filter pushdown - StringContains") + .excludeCH("SPARK-36866: filter pushdown - year-month interval") + // avoid Velox compile error + enableSuite( + "org.apache.gluten.execution.parquet.GlutenParquetV1FilterSuite2" + ) + // Rewrite. + .includeCH("SPARK-23852: Broken Parquet push-down for partially-written stats") + // Rewrite for supported INT96 - timestamp. + .exclude("filter pushdown - timestamp") + .exclude("filter pushdown - date") + // Exception bebaviour. + .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") + // Ignore Spark's filter pushdown check. + .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") + .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") + .exclude("Filters should be pushed down for Parquet readers at row group level") + .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") + .exclude("Support Parquet column index") + .exclude("SPARK-34562: Bloom filter push down") + .includeCH("SPARK-16371 Do not push down filters when inner name and outer name are the same") + .exclude("filter pushdown - StringPredicate") + .excludeCH("filter pushdown - StringContains") + .excludeCH("SPARK-36866: filter pushdown - year-month interval") + enableSuite[GlutenParquetV1PartitionDiscoverySuite] + .excludeCH("Various partition value types") + .excludeCH("Various inferred partition value types") + .excludeCH("Resolve type conflicts - decimals, dates and timestamps in partition column") + enableSuite[GlutenParquetV1QuerySuite] + .exclude("row group skipping doesn't overflow when reading into larger type") + // Unsupport spark.sql.files.ignoreCorruptFiles. + .exclude("Enabling/disabling ignoreCorruptFiles") + // decimal failed ut + .exclude("SPARK-34212 Parquet should read decimals correctly") + // new added in spark-3.3 and need fix later, random failure may caused by memory free + .includeCH("SPARK-39833: pushed filters with project without filter columns") + .includeCH("SPARK-39833: pushed filters with count()") + // Rewrite because the filter after datasource is not needed. + .exclude( + "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") + enableSuite[GlutenParquetV1SchemaPruningSuite] + .excludeCH("Case-insensitive parser - mixed-case schema - select with exact column names") + .excludeCH("Case-insensitive parser - mixed-case schema - select with lowercase column names") + .excludeCH( + "Case-insensitive parser - mixed-case schema - select with different-case column names") + .excludeCH( + "Case-insensitive parser - mixed-case schema - filter with different-case column names") + .excludeCH( + "Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") + .excludeCH("SPARK-36352: Spark should check result plan's output schema name") + .excludeCH("Spark vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Spark vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + enableSuite[GlutenParquetV2AggregatePushDownSuite] + // TODO: Timestamp columns stats will lost if using int64 in parquet writer. + .includeCH("aggregate push down - different data types") + enableSuite[GlutenParquetV2FilterSuite] + // Rewrite. + .includeCH("SPARK-23852: Broken Parquet push-down for partially-written stats") + // Rewrite for supported INT96 - timestamp. + .exclude("filter pushdown - timestamp") + .exclude("filter pushdown - date") + // Exception bebaviour. + .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") + // Ignore Spark's filter pushdown check. + .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") + .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") + .exclude("Filters should be pushed down for Parquet readers at row group level") + .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") + .exclude("Support Parquet column index") + .exclude("SPARK-34562: Bloom filter push down") + .includeCH("SPARK-16371 Do not push down filters when inner name and outer name are the same") + .exclude("filter pushdown - StringPredicate") + .excludeCH("filter pushdown - StringContains") + .excludeCH("SPARK-36866: filter pushdown - year-month interval") + .excludeCH("Gluten - filter pushdown - date") + enableSuite[GlutenParquetV2PartitionDiscoverySuite] + .excludeCH("Various partition value types") + .excludeCH("Various inferred partition value types") + .excludeCH("Resolve type conflicts - decimals, dates and timestamps in partition column") + enableSuite[GlutenParquetV2QuerySuite] + .exclude("row group skipping doesn't overflow when reading into larger type") + // Unsupport spark.sql.files.ignoreCorruptFiles. + .exclude("Enabling/disabling ignoreCorruptFiles") + // decimal failed ut + .exclude("SPARK-34212 Parquet should read decimals correctly") + // Rewrite because the filter after datasource is not needed. + .exclude( + "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") + enableSuite[GlutenParquetV2SchemaPruningSuite] + .excludeCH("Spark vectorized reader - without partition data column - select a single complex field and in where clause") + .excludeCH("Spark vectorized reader - with partition data column - select a single complex field and in where clause") + .excludeCH("Non-vectorized reader - without partition data column - select a single complex field and in where clause") + .excludeCH("Non-vectorized reader - with partition data column - select a single complex field and in where clause") + .excludeCH("Spark vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Spark vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Non-vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Non-vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .excludeCH("Spark vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Spark vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Non-vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Non-vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output - case-sensitivity") + .excludeCH("Spark vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Spark vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Non-vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Non-vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .excludeCH("Case-insensitive parser - mixed-case schema - select with exact column names") + .excludeCH("Case-insensitive parser - mixed-case schema - select with lowercase column names") + .excludeCH( + "Case-insensitive parser - mixed-case schema - select with different-case column names") + .excludeCH( + "Case-insensitive parser - mixed-case schema - filter with different-case column names") + .excludeCH("Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") + .excludeCH("SPARK-36352: Spark should check result plan's output schema name") + .excludeCH("SPARK-37450: Prunes unnecessary fields from Explode for count aggregation") + .excludeCH("Spark vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Spark vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - without partition data column - SPARK-40033: Schema pruning support through element_at") + .excludeCH("Non-vectorized reader - with partition data column - SPARK-40033: Schema pruning support through element_at") + enableSuite[GlutenParquetVectorizedSuite] + enableSuite[GlutenPartitionedWriteSuite] + .excludeCH("SPARK-37231, SPARK-37240: Dynamic writes/reads of ANSI interval partitions") + enableSuite[GlutenPathFilterStrategySuite] + enableSuite[GlutenPathFilterSuite] + enableSuite[GlutenPathOptionSuite] + enableSuite[GlutenPredicateSuite] + .excludeCH("basic IN/INSET predicate test") + .excludeCH("IN with different types") + .excludeCH("IN/INSET: binary") + .excludeCH("IN/INSET: struct") + .excludeCH("IN/INSET: array") + .excludeCH("BinaryComparison: lessThan") + .excludeCH("BinaryComparison: LessThanOrEqual") + .excludeCH("BinaryComparison: GreaterThan") + .excludeCH("BinaryComparison: GreaterThanOrEqual") + .excludeCH("EqualTo on complex type") + .excludeCH("SPARK-32764: compare special double/float values") + .excludeCH("SPARK-32110: compare special double/float values in struct") + enableSuite[GlutenProcessingTimeSuite] + enableSuite[GlutenProductAggSuite] + enableSuite[GlutenPruneFileSourcePartitionsSuite] + enableSuite[GlutenPrunedScanSuite] + enableSuite[GlutenQueryCompilationErrorsDSv2Suite] + enableSuite[GlutenQueryCompilationErrorsSuite] + .excludeCH("CREATE NAMESPACE with LOCATION for JDBC catalog should throw an error") + .excludeCH( + "ALTER NAMESPACE with property other than COMMENT for JDBC catalog should throw an exception") + enableSuite[GlutenQueryExecutionErrorsSuite] + // NEW SUITE: disable as it expects exception which doesn't happen when offloaded to gluten + .exclude( + "INCONSISTENT_BEHAVIOR_CROSS_VERSION: compatibility with Spark 2.4/3.2 in reading/writing dates") + // Doesn't support unhex with failOnError=true. + .exclude("CONVERSION_INVALID_INPUT: to_binary conversion function hex") + .excludeCH("CONVERSION_INVALID_INPUT: to_binary conversion function base64") + .excludeCH("UNSUPPORTED_FEATURE - SPARK-38504: can't read TimestampNTZ as TimestampLTZ") + .excludeCH("CANNOT_PARSE_DECIMAL: unparseable decimal") + .excludeCH("UNRECOGNIZED_SQL_TYPE: unrecognized SQL type DATALINK") + .excludeCH("UNSUPPORTED_FEATURE.MULTI_ACTION_ALTER: The target JDBC server hosting table does not support ALTER TABLE with multiple actions.") + .excludeCH("INVALID_BITMAP_POSITION: position out of bounds") + .excludeCH("INVALID_BITMAP_POSITION: negative position") + enableSuite[GlutenQueryParsingErrorsSuite] + enableSuite[GlutenRandomSuite] + .exclude("random") + .exclude("SPARK-9127 codegen with long seed") + enableSuite[GlutenRegexpExpressionsSuite] + .excludeCH("LIKE Pattern") + .excludeCH("LIKE Pattern ESCAPE '/'") + .excludeCH("LIKE Pattern ESCAPE '#'") + .excludeCH("LIKE Pattern ESCAPE '\"'") + .excludeCH("RLIKE Regular Expression") + .excludeCH("RegexReplace") + .excludeCH("RegexExtract") + .excludeCH("RegexExtractAll") + .excludeCH("SPLIT") + enableSuite[GlutenRemoveRedundantWindowGroupLimitsSuite] + .excludeCH("remove redundant WindowGroupLimits") + enableSuite[GlutenReplaceHashWithSortAggSuite] + .exclude("replace partial hash aggregate with sort aggregate") + .exclude("replace partial and final hash aggregate together with sort aggregate") + .exclude("do not replace hash aggregate if child does not have sort order") + .exclude("do not replace hash aggregate if there is no group-by column") + enableSuite[GlutenReplaceNullWithFalseInPredicateEndToEndSuite] + enableSuite[GlutenResolveDefaultColumnsSuite] + enableSuite[GlutenResolvedDataSourceSuite] + enableSuite[GlutenReuseExchangeAndSubquerySuite] + enableSuite[GlutenRuntimeNullChecksV2Writes] + enableSuite[GlutenSQLAggregateFunctionSuite] + .excludeGlutenTest("Return NaN or null when dividing by zero") + enableSuite[GlutenSQLQuerySuite] + // Decimal precision exceeds. + .includeCH("should be able to resolve a persistent view") + // Unstable. Needs to be fixed. + .includeCH("SPARK-36093: RemoveRedundantAliases should not change expression's name") + // Rewrite from ORC scan to Parquet scan because ORC is not well supported. + .includeCH("SPARK-28156: self-join should not miss cached view") + .includeCH("SPARK-33338: GROUP BY using literal map should not fail") + // Rewrite to disable plan check for SMJ because SHJ is preferred in Gluten. + .includeCH("SPARK-11111 null-safe join should not use cartesian product") + // Rewrite to change the information of a caught exception. + .exclude("SPARK-33677: LikeSimplification should be skipped if pattern contains any escapeChar") + // Different exception. + .exclude("run sql directly on files") + // Not useful and time consuming. + .includeCH("SPARK-33084: Add jar support Ivy URI in SQL") + .exclude("SPARK-33084: Add jar support Ivy URI in SQL -- jar contains udf class") + // exception test, rewritten in gluten + .exclude("the escape character is not allowed to end with") + // ORC related + .includeCH("SPARK-37965: Spark support read/write orc file with invalid char in field name") + .includeCH("SPARK-38173: Quoted column cannot be recognized correctly when quotedRegexColumnNames is true") + // Need to support MAP + .exclude( + "SPARK-27619: When spark.sql.legacy.allowHashOnMapType is true, hash can be used on Maptype") + .excludeCH("SPARK-6743: no columns from cache") + .excludeCH("external sorting updates peak execution memory") + .excludeCH("Struct Star Expansion") + .excludeCH("Common subexpression elimination") + .excludeCH("SPARK-24940: coalesce and repartition hint") + .excludeCH("normalize special floating numbers in subquery") + .excludeCH("SPARK-38548: try_sum should return null if overflow happens before merging") + .excludeCH("SPARK-38589: try_avg should return null if overflow happens before merging") + .excludeCH("Gluten - SPARK-33677: LikeSimplification should be skipped if pattern contains any escapeChar") + .excludeCH("Gluten - the escape character is not allowed to end with") + // enableSuite[GlutenSQLQueryTestSuite] + enableSuite[GlutenSQLWindowFunctionSuite] + // spill not supported yet. + .exclude("test with low buffer spill threshold") + .excludeCH( + "window function: multiple window expressions specified by range in a single expression") + .excludeCH("Gluten - Filter on row number") + enableSuite[GlutenSameResultSuite] + enableSuite[GlutenSaveLoadSuite] + enableSuite[GlutenScalaReflectionRelationSuite] + enableSuite[GlutenSerializationSuite] + enableSuite[GlutenSessionExtensionSuite] + enableSuite[GlutenSortOrderExpressionsSuite] + // TODO: fix the hanging problem in GLUTEN-8890 followup +// enableSuite[GlutenSortShuffleSuite] + enableSuite[GlutenSortSuite] + .excludeCH("basic sorting using ExternalSort") + .excludeCH("SPARK-33260: sort order is a Stream") + .excludeCH("SPARK-40089: decimal values sort correctly") + .excludeCH( + "sorting on YearMonthIntervalType(0,1) with nullable=true, sortOrder=List('a ASC NULLS FIRST)") + .excludeCH( + "sorting on YearMonthIntervalType(0,1) with nullable=true, sortOrder=List('a ASC NULLS LAST)") + .excludeCH( + "sorting on YearMonthIntervalType(0,1) with nullable=true, sortOrder=List('a DESC NULLS LAST)") + .excludeCH("sorting on YearMonthIntervalType(0,1) with nullable=true, sortOrder=List('a DESC NULLS FIRST)") + .excludeCH("sorting on YearMonthIntervalType(0,1) with nullable=false, sortOrder=List('a ASC NULLS FIRST)") + .excludeCH( + "sorting on YearMonthIntervalType(0,1) with nullable=false, sortOrder=List('a ASC NULLS LAST)") + .excludeCH("sorting on YearMonthIntervalType(0,1) with nullable=false, sortOrder=List('a DESC NULLS LAST)") + .excludeCH("sorting on YearMonthIntervalType(0,1) with nullable=false, sortOrder=List('a DESC NULLS FIRST)") + enableSuite[GlutenSparkSessionExtensionSuite] + enableSuite[GlutenStatisticsCollectionSuite] + // The output byte size of Velox is different + .includeCH("SPARK-33687: analyze all tables in a specific database") + .excludeCH("analyze empty table") + .excludeCH("analyze column command - result verification") + .excludeCH("column stats collection for null columns") + .excludeCH("store and retrieve column stats in different time zones") + .excludeCH("SPARK-42777: describe column stats (min, max) for timestamp_ntz column") + .excludeCH("Gluten - store and retrieve column stats in different time zones") + .excludeCH("statistics collection of a table with zero column") + enableSuite[GlutenStringExpressionsSuite] + .excludeCH("StringComparison") + .excludeCH("Substring") + .excludeCH("string substring_index function") + .excludeCH("SPARK-40213: ascii for Latin-1 Supplement characters") + .excludeCH("ascii for string") + .excludeCH("Mask") + .excludeCH("SPARK-42384: Mask with null input") + .excludeCH("base64/unbase64 for string") + .excludeCH("encode/decode for string") + .excludeCH("SPARK-47307: base64 encoding without chunking") + .excludeCH("Levenshtein distance threshold") + .excludeCH("soundex unit test") + .excludeCH("overlay for string") + .excludeCH("overlay for byte array") + .excludeCH("translate") + .excludeCH("FORMAT") + .excludeCH("LOCATE") + .excludeCH("REPEAT") + .excludeCH("ParseUrl") + .excludeCH("SPARK-33468: ParseUrl in ANSI mode should fail if input string is not a valid url") + enableSuite[GlutenStringFunctionsSuite] + .excludeCH("string Levenshtein distance") + .excludeCH("string regexp_count") + .excludeCH("string regex_replace / regex_extract") + .excludeCH("string regexp_extract_all") + .excludeCH("string regexp_substr") + .excludeCH("string overlay function") + .excludeCH("binary overlay function") + .excludeCH("string / binary length function") + .excludeCH("SPARK-36751: add octet length api for scala") + .excludeCH("SPARK-36751: add bit length api for scala") + .excludeCH("str_to_map function") + .excludeCH("SPARK-42384: mask with null input") + .excludeCH("like & ilike function") + .excludeCH("parse_url") + .excludeCH("url_decode") + .excludeCH("url_encode") + enableSuite[GlutenSubqueryHintPropagationSuite] + enableSuite[GlutenSubquerySuite] + .excludeByPrefix( + "SPARK-26893" // Rewrite this test because it checks Spark's physical operators. + ) + // exclude as it checks spark plan + .exclude("SPARK-36280: Remove redundant aliases after RewritePredicateSubquery") + .excludeCH("SPARK-39355: Single column uses quoted to construct UnresolvedAttribute") + .excludeCH("SPARK-40800: always inline expressions in OptimizeOneRowRelationSubquery") + .excludeCH("SPARK-40862: correlated one-row subquery with non-deterministic expressions") + enableSuite[GlutenSupportsCatalogOptionsSuite] + enableSuite[GlutenTableCapabilityCheckSuite] + enableSuite[GlutenTableLocationSuite] + enableSuite[GlutenTableOptionsConstantFoldingSuite] + enableSuite[GlutenTableScanSuite] + enableSuite[GlutenTakeOrderedAndProjectSuite] + .excludeCH("TakeOrderedAndProject.doExecute without project") + .excludeCH("TakeOrderedAndProject.doExecute with project") + .excludeCH("TakeOrderedAndProject.doExecute with local sort") + enableSuite[GlutenTextV1Suite] + enableSuite[GlutenTextV2Suite] + enableSuite[GlutenTryCastSuite] + .exclude( + "Process Infinity, -Infinity, NaN in case insensitive manner" // +inf not supported in folly. + ) + .exclude("ANSI mode: Throw exception on casting out-of-range value to byte type") + .exclude("ANSI mode: Throw exception on casting out-of-range value to short type") + .exclude("ANSI mode: Throw exception on casting out-of-range value to int type") + .exclude("ANSI mode: Throw exception on casting out-of-range value to long type") + .exclude("cast from invalid string to numeric should throw NumberFormatException") + .exclude("SPARK-26218: Fix the corner case of codegen when casting float to Integer") + // Set timezone through config. + .exclude("data type casting") + .excludeCH("null cast") + .excludeCH("cast string to date") + .excludeCH("cast string to timestamp") + .excludeGlutenTest("cast string to timestamp") + .excludeCH("SPARK-22825 Cast array to string") + .excludeCH("SPARK-33291: Cast array with null elements to string") + .excludeCH("SPARK-22973 Cast map to string") + .excludeCH("SPARK-22981 Cast struct to string") + .excludeCH("SPARK-33291: Cast struct with null elements to string") + .excludeCH("SPARK-35111: Cast string to year-month interval") + .excludeCH("cast from timestamp II") + .excludeCH("cast a timestamp before the epoch 1970-01-01 00:00:00Z II") + .excludeCH("cast a timestamp before the epoch 1970-01-01 00:00:00Z") + .excludeCH("cast from array II") + .excludeCH("cast from array III") + .excludeCH("cast from struct III") + .excludeCH("ANSI mode: cast string to timestamp with parse error") + .excludeCH("ANSI mode: cast string to date with parse error") + .excludeCH("Gluten - data type casting") + enableSuite[GlutenTryEvalSuite] + .excludeCH("try_subtract") + enableSuite[GlutenTypedImperativeAggregateSuite] + enableSuite[GlutenUnwrapCastInComparisonEndToEndSuite] + // Rewrite with NaN test cases excluded. + .exclude("cases when literal is max") + enableSuite[GlutenUrlFunctionsSuite] + .excludeCH("url encode/decode function") + enableSuite[GlutenV1WriteCommandSuite] + // Rewrite to match SortExecTransformer. + .excludeByPrefix("SPARK-41914:") + .excludeCH( + "Gluten - SPARK-41914: v1 write with AQE and in-partition sorted - non-string partition column") + .excludeCH( + "Gluten - SPARK-41914: v1 write with AQE and in-partition sorted - string partition column") + enableSuite[GlutenV2PredicateSuite] + enableSuite[GlutenValidateRequirementsSuite] + enableSuite[GlutenVectorizedOrcReadSchemaSuite] + // Rewrite to disable Spark's vectorized reading. + .includeCH("change column position") + .includeCH("read byte, int, short, long together") + .includeCH("read float and double together") + .includeCH("append column into middle") + .includeCH("add a nested column at the end of the leaf struct column") + .includeCH("add a nested column in the middle of the leaf struct column") + .includeCH("add a nested column at the end of the middle struct column") + .includeCH("add a nested column in the middle of the middle struct column") + .includeCH("hide a nested column at the end of the leaf struct column") + .includeCH("hide a nested column in the middle of the leaf struct column") + .includeCH("hide a nested column at the end of the middle struct column") + .includeCH("hide a nested column in the middle of the middle struct column") + .includeCH("change column type from boolean to byte/short/int/long") + .includeCH("change column type from byte to short/int/long") + .includeCH("change column type from short to int/long") + .includeCH("change column type from int to long") + .includeCH("change column type from float to double") + // .excludeGlutenTest("read byte, int, short, long together") + // .excludeGlutenTest("read float and double together") + enableSuite[GlutenVectorizedParquetReadSchemaSuite] + enableSuite[GlutenWriteDistributionAndOrderingSuite] + enableSuite[GlutenXPathFunctionsSuite] + + override def getSQLQueryTestSettings: SQLQueryTestSettings = ClickHouseSQLQueryTestSettings +} +// scalastyle:on line.size.limit diff --git a/gluten-ut/spark41/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala b/gluten-ut/spark41/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.scala new file mode 100644 index 000000000000..92c62a4ed314 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/gluten/utils/velox/VeloxSQLQueryTestSettings.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.gluten.utils.velox + +import org.apache.gluten.utils.SQLQueryTestSettings + +object VeloxSQLQueryTestSettings extends SQLQueryTestSettings { + override def getResourceFilePath: String = + getClass.getResource("/").getPath + "../../../src/test/resources/backends-velox/sql-tests" + + override def getSupportedSQLQueryTests: Set[String] = SUPPORTED_SQL_QUERY_LIST + + override def getOverwriteSQLQueryTests: Set[String] = OVERWRITE_SQL_QUERY_LIST + + // Put relative path to "/path/to/spark/sql/core/src/test/resources/sql-tests/inputs" in this list + // Gluten currently only supports `SET spark.sql.legacy.timeParserPolicy=LEGACY` + // Queries in `date.sql` and `timestamp.sql` are tested in `datetime-legacy.sql`. + // There is a limitation in Spark-4.0 on ANSI gap and exception message gap. + // we disabled these sql tests temporarily. + // limit.sql, postgreSQL/limit.sql, subquery/in-subquery/in-limit.sql, + // subquery/in-subquery/in-null-semantics.sql + // try_arithmetic.sql, try_element_at.sql, typeCoercion/native/stringCastAndExpressions.sql, + // window.sql + val SUPPORTED_SQL_QUERY_LIST: Set[String] = Set( + "ansi/conditional-functions.sql", + "ansi/decimalArithmeticOperations.sql", + "cast.sql", + "change-column.sql", + "ceil-floor-with-scale-param.sql", + "column-resolution-aggregate.sql", + "column-resolution-sort.sql", + "columnresolution-negative.sql", + "columnresolution-views.sql", + "columnresolution.sql", + "comments.sql", + "comparator.sql", + "cross-join.sql", + "csv-functions.sql", + "cte-legacy.sql", + "cte-nested.sql", + "cte-nonlegacy.sql", + "current_database_catalog.sql", + // "datetime-formatting-invalid.sql", + "datetime-special.sql", + // TODO: fix on Spark-4.1 "decimalArithmeticOperations.sql", + "describe.sql", + "describe-part-after-analyze.sql", + "describe-table-after-alter-table.sql", + "describe-query.sql", + "double-quoted-identifiers.sql", + "except.sql", + "except-all.sql", + "extract.sql", + "group-by.sql", + "group-by-all.sql", + "group-by-all-duckdb.sql", + "group-by-all-mosha.sql", + "group-analytics.sql", + "group-by-filter.sql", + "group-by-ordinal.sql", + "grouping_set.sql", + "having.sql", + "higher-order-functions.sql", + // TODO: fix on Spark-4.1 "identifier-clause.sql", + "ignored.sql", + "ilike.sql", + "ilike-all.sql", + "ilike-any.sql", + "inline-table.sql", + "inner-join.sql", + "intersect-all.sql", + "join-empty-relation.sql", + "join-lateral.sql", + "json-functions.sql", + // TODO: fix on Spark-4.1 "keywords.sql", + "like-all.sql", + "like-any.sql", + // "limit.sql", + // TODO: fix on Spark-4.1 "literals.sql", + "map.sql", + "mask-functions.sql", + "math.sql", + "named-function-arguments.sql", + "natural-join.sql", + "non-excludable-rule.sql", + "null-handling.sql", + "null-propagation.sql", + // TODO: fix on Spark-4.1 "operators.sql", + "order-by-all.sql", + // "order-by-nulls-ordering.sql", + "order-by-ordinal.sql", + "outer-join.sql", + "parse-schema-string.sql", + "pivot.sql", + "pred-pushdown.sql", + "predicate-functions.sql", + "query_regex_column.sql", + "random.sql", + "show-create-table.sql", + "show-tables.sql", + "show-tblproperties.sql", + "show-views.sql", + "show_columns.sql", + "sql-compatibility-functions.sql", + "struct.sql", + "subexp-elimination.sql", + "table-aliases.sql", + "table-valued-functions.sql", + "tablesample-negative.sql", + "try-string-functions.sql", + "subquery/exists-subquery/exists-aggregate.sql", + "subquery/exists-subquery/exists-basic.sql", + "subquery/exists-subquery/exists-cte.sql", + "subquery/exists-subquery/exists-having.sql", + "subquery/exists-subquery/exists-joins-and-set-ops.sql", + // TODO: fix on Spark-4.1 "subquery/exists-subquery/exists-orderby-limit.sql", + "subquery/exists-subquery/exists-outside-filter.sql", + "subquery/exists-subquery/exists-within-and-or.sql", + "subquery/in-subquery/in-basic.sql", + "subquery/in-subquery/in-group-by.sql", + "subquery/in-subquery/in-having.sql", + "subquery/in-subquery/in-joins.sql", + // "subquery/in-subquery/in-limit.sql", + "subquery/in-subquery/in-multiple-columns.sql", + "subquery/in-subquery/in-nullability.sql", + "subquery/in-subquery/in-order-by.sql", + "subquery/in-subquery/in-set-operations.sql", + "subquery/in-subquery/in-with-cte.sql", + "subquery/in-subquery/nested-not-in.sql", + "subquery/in-subquery/not-in-group-by.sql", + "subquery/in-subquery/not-in-joins.sql", + "subquery/in-subquery/not-in-unit-tests-multi-column.sql", + "subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql", + "subquery/in-subquery/not-in-unit-tests-single-column.sql", + "subquery/in-subquery/not-in-unit-tests-single-column-literal.sql", + "subquery/in-subquery/simple-in.sql", + "subquery/negative-cases/invalid-correlation.sql", + "subquery/negative-cases/subq-input-typecheck.sql", + "subquery/scalar-subquery/scalar-subquery-count-bug.sql", + "subquery/scalar-subquery/scalar-subquery-predicate.sql", + "subquery/scalar-subquery/scalar-subquery-select.sql", + "subquery/scalar-subquery/scalar-subquery-set-op.sql", + "subquery/subquery-in-from.sql", + "postgreSQL/aggregates_part1.sql", + "postgreSQL/aggregates_part2.sql", + "postgreSQL/aggregates_part3.sql", + "postgreSQL/aggregates_part4.sql", + "postgreSQL/boolean.sql", + "postgreSQL/case.sql", + "postgreSQL/comments.sql", + "postgreSQL/create_view.sql", + // TODO: fix on Spark-4.1 "postgreSQL/date.sql", + "postgreSQL/float4.sql", + "postgreSQL/float8.sql", + "postgreSQL/groupingsets.sql", + "postgreSQL/insert.sql", + "postgreSQL/int2.sql", + "postgreSQL/int4.sql", + "postgreSQL/int8.sql", + "postgreSQL/interval.sql", + "postgreSQL/join.sql", + // "postgreSQL/limit.sql", + "postgreSQL/numeric.sql", + "postgreSQL/select.sql", + "postgreSQL/select_distinct.sql", + "postgreSQL/select_having.sql", + "postgreSQL/select_implicit.sql", + "postgreSQL/strings.sql", + "postgreSQL/text.sql", + "postgreSQL/timestamp.sql", + "postgreSQL/union.sql", + "postgreSQL/window_part2.sql", + "postgreSQL/with.sql", + "datetime-special.sql", + "timestamp-ltz.sql", + "timestamp-ntz.sql", + "timezone.sql", + "transform.sql", + "try-string-functions.sql", + "try_aggregates.sql", + "typeCoercion/native/arrayJoin.sql", + "typeCoercion/native/binaryComparison.sql", + "typeCoercion/native/booleanEquality.sql", + "typeCoercion/native/caseWhenCoercion.sql", + "typeCoercion/native/concat.sql", + "typeCoercion/native/dateTimeOperations.sql", + "typeCoercion/native/decimalPrecision.sql", + "typeCoercion/native/division.sql", + "typeCoercion/native/elt.sql", + "typeCoercion/native/ifCoercion.sql", + // "typeCoercion/native/implicitTypeCasts.sql", + "typeCoercion/native/inConversion.sql", + "typeCoercion/native/mapconcat.sql", + "typeCoercion/native/mapZipWith.sql", + "typeCoercion/native/promoteStrings.sql", + "typeCoercion/native/widenSetOperationTypes.sql", + "typeCoercion/native/windowFrameCoercion.sql", + "udaf/udaf.sql - Grouped Aggregate Pandas UDF", + "udf/udf-union.sql - Scala UDF", + "udf/udf-intersect-all.sql - Scala UDF", + "udf/udf-except-all.sql - Scala UDF", + "udf/udf-udaf.sql - Scala UDF", + "udf/udf-except.sql - Scala UDF", + "udf/udf-pivot.sql - Scala UDF", + "udf/udf-inline-table.sql - Scala UDF", + "udf/postgreSQL/udf-select_having.sql - Scala UDF", + "union.sql", + "unpivot.sql", + "using-join.sql", + // newly added in Spark-4.0 + "variant/named-function-arguments.sql", + "timestampNTZ/datetime-special-ansi.sql", + "timestampNTZ/datetime-special.sql", + // "timestampNTZ/timestamp-ansi.sql", + // "timestampNTZ/timestamp.sql", + // "nonansi/array.sql", + "nonansi/cast.sql", + "nonansi/conditional-functions.sql", + // "nonansi/date.sql", + // "nonansi/datetime-parsing-invalid.sql", + "nonansi/datetime-special.sql", + "nonansi/decimalArithmeticOperations.sql", + "nonansi/double-quoted-identifiers.sql", + "nonansi/higher-order-functions.sql", + // "nonansi/interval.sql", + // TODO: fix on Spark-4.1 "nonansi/keywords.sql", + // TODO: fix on Spark-4.1 "nonansi/literals.sql", + "nonansi/map.sql", + "nonansi/math.sql", + "nonansi/parse-schema-string.sql", + // "nonansi/string-functions.sql", + // "nonansi/timestamp.sql", + "nonansi/try_aggregates.sql" + // "nonansi/try_arithmetic.sql", + // "nonansi/try_datetime_functions.sql", + // "nonansi/try_element_at.sql" + ) + + val OVERWRITE_SQL_QUERY_LIST: Set[String] = Set( + // The calculation formulas for corr, skewness, kurtosis, variance, and stddev in Velox differ + // slightly from those in Spark, resulting in some differences in the final results. + // Overwrite below test cases. + // -- SPARK-24369 multiple distinct aggregations having the same argument set + // -- Aggregate with nulls. + "group-by.sql", + "udf/udf-group-by.sql", + // Overwrite some results of regr_intercept, regr_r2, corr. + "linear-regression.sql", + // Overwrite exception message. + // TODO: Disable due to schema & ANSI gap + // "array.sql", + // Overwrite exception message. + "bitwise.sql", + // Enable NullPropagation rule for + // "legacy behavior: allow calling function count without parameters". + "count.sql", + // Enable ConstantFolding rule for "typeof(...)". + "charvarchar.sql", + // Enable ConstantFolding rule for "typeof(...)". + "cte.sql", + // Removed some result mismatch cases. + // TODO: fix on Spark-4.1 "datetime-legacy.sql", + // Removed some result mismatch cases. + "datetime-parsing.sql", + // Removed some result mismatch cases. + "datetime-parsing-legacy.sql", + // Removed some result mismatch cases. + // TODO: fix on Spark-4.1 "datetime-parsing-invalid.sql", + // Overwrite exception message. See Spark-46550. + "hll.sql", + // Overwrite exception message. + // TODO: Disable due to schema & ANSI gap + // "interval.sql", + // Enable ConstantFolding rule for "typeof(...)". + // TODO: fix on Spark-4.1 "misc-functions.sql", + // Removed some result mismatch cases. + "regexp-functions.sql", + // Removed some result mismatch cases. + "string-functions.sql", + // Removed some result mismatch cases. + // TODO: Disable due to schema & ANSI gap + // "try_arithmetic.sql", + // Removed some result mismatch cases. + "try_cast.sql", + // Removed SQLs that can only pass with `set spark.sql.legacy.timeParserPolicy=LEGACY;` + // TODO: Disable due to schema & ANSI gap + // "typeCoercion/native/stringCastAndExpressions.sql", + // Enable ConstantFolding rule for some queries. + "percentiles.sql", + // Enable ConstantFolding rule for some queries, otherwise Spark will throw an exception. + "postgreSQL/window_part1.sql", + // Enable ConstantFolding rule for some queries, otherwise Spark will throw an exception. + "postgreSQL/window_part3.sql", + // Enable ConstantFolding rule for some queries, otherwise Spark will throw an exception. + "postgreSQL/window_part4.sql", + // Enable NullPropagation rule for some queries that rely on the rule. + // TODO: Disable due to schema & ANSI gap + // "subquery/in-subquery/in-null-semantics.sql", + // Removed some result mismatch cases. + "try_datetime_functions.sql", + // Overwrite exception message. + // TODO: Disable due to schema & ANSI gap + // "try_element_at.sql", + // Overwrite exception message. + "url-functions.sql" + // Removed failed query. Adjust the output order for some queries. + // TODO: Disable due to schema & ANSI gap + // "window.sql" + ) +} 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 new file mode 100644 index 000000000000..540479a11fa0 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -0,0 +1,979 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.utils.velox + +import org.apache.gluten.utils.{BackendTestSettings, SQLQueryTestSettings} + +import org.apache.spark.GlutenSortShuffleSuite +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.connector._ +import org.apache.spark.sql.errors.{GlutenQueryCompilationErrorsDSv2Suite, GlutenQueryCompilationErrorsSuite, GlutenQueryExecutionErrorsSuite, GlutenQueryParsingErrorsSuite} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive.velox.VeloxAdaptiveQueryExecSuite +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.binaryfile.GlutenBinaryFileFormatSuite +import org.apache.spark.sql.execution.datasources.json.{GlutenJsonLegacyTimeParserSuite, GlutenJsonV1Suite, GlutenJsonV2Suite} +import org.apache.spark.sql.execution.datasources.orc._ +import org.apache.spark.sql.execution.datasources.parquet._ +import org.apache.spark.sql.execution.datasources.text.{GlutenTextV1Suite, GlutenTextV2Suite} +import org.apache.spark.sql.execution.datasources.v2.{GlutenDataSourceV2StrategySuite, GlutenFileTableSuite, GlutenV2PredicateSuite} +import org.apache.spark.sql.execution.exchange.{GlutenEnsureRequirementsSuite, GlutenValidateRequirementsSuite} +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.python._ +import org.apache.spark.sql.extension.{GlutenCollapseProjectExecTransformerSuite, GlutenSessionExtensionSuite, TestFileSourceScanExecTransformer} +import org.apache.spark.sql.gluten.{GlutenFallbackStrategiesSuite, GlutenFallbackSuite} +import org.apache.spark.sql.hive.execution.GlutenHiveSQLQuerySuite +import org.apache.spark.sql.sources._ + +// Some settings' line length exceeds 100 +// scalastyle:off line.size.limit + +class VeloxTestSettings extends BackendTestSettings { + enableSuite[GlutenStringFunctionsSuite] + enableSuite[GlutenBloomFilterAggregateQuerySuite] + enableSuite[GlutenBloomFilterAggregateQuerySuiteCGOff] + enableSuite[GlutenDataSourceV2DataFrameSessionCatalogSuite] + enableSuite[GlutenDataSourceV2DataFrameSuite] + enableSuite[GlutenDataSourceV2FunctionSuite] + enableSuite[GlutenDataSourceV2SQLSessionCatalogSuite] + enableSuite[GlutenDataSourceV2SQLSuiteV1Filter] + enableSuite[GlutenDataSourceV2SQLSuiteV2Filter] + enableSuite[GlutenDataSourceV2Suite] + // Rewrite the following tests in GlutenDataSourceV2Suite. + .exclude("partitioning reporting") + .exclude("ordering and partitioning reporting") + enableSuite[GlutenDeleteFromTableSuite] + enableSuite[GlutenFileDataSourceV2FallBackSuite] + // Rewritten + .exclude("Fallback Parquet V2 to V1") + enableSuite[GlutenKeyGroupedPartitioningSuite] + // NEW SUITE: disable as they check vanilla spark plan + .exclude("partitioned join: number of buckets mismatch should trigger shuffle") + .exclude("partitioned join: only one side reports partitioning") + .exclude("partitioned join: join with two partition keys and different # of partition keys") + .excludeByPrefix("SPARK-47094") + .excludeByPrefix("SPARK-48655") + .excludeByPrefix("SPARK-48012") + .excludeByPrefix("SPARK-44647") + .excludeByPrefix("SPARK-41471") + // disable due to check for SMJ node + .excludeByPrefix("SPARK-41413: partitioned join:") + .excludeByPrefix("SPARK-42038: partially clustered:") + .exclude("SPARK-44641: duplicated records when SPJ is not triggered") + // TODO: fix on Spark-4.1 + .excludeByPrefix("SPARK-53322") // see https://github.com/apache/spark/pull/53132 + .excludeByPrefix("SPARK-54439") // see https://github.com/apache/spark/pull/53142 + enableSuite[GlutenLocalScanSuite] + enableSuite[GlutenMetadataColumnSuite] + enableSuite[GlutenSupportsCatalogOptionsSuite] + enableSuite[GlutenTableCapabilityCheckSuite] + enableSuite[GlutenWriteDistributionAndOrderingSuite] + enableSuite[GlutenQueryCompilationErrorsDSv2Suite] + enableSuite[GlutenQueryCompilationErrorsSuite] + enableSuite[GlutenQueryExecutionErrorsSuite] + // NEW SUITE: disable as it expects exception which doesn't happen when offloaded to gluten + .exclude( + "INCONSISTENT_BEHAVIOR_CROSS_VERSION: compatibility with Spark 2.4/3.2 in reading/writing dates") + // Doesn't support unhex with failOnError=true. + .exclude("CONVERSION_INVALID_INPUT: to_binary conversion function hex") + enableSuite[GlutenQueryParsingErrorsSuite] + enableSuite[GlutenArithmeticExpressionSuite] + .exclude("SPARK-45786: Decimal multiply, divide, remainder, quot") + enableSuite[GlutenBitwiseExpressionsSuite] + enableSuite[GlutenCastWithAnsiOffSuite] + .exclude( + "Process Infinity, -Infinity, NaN in case insensitive manner" // +inf not supported in folly. + ) + // Set timezone through config. + .exclude("data type casting") + // Revised by setting timezone through config and commented unsupported cases. + .exclude("cast string to timestamp") + .exclude("cast from timestamp II") + .exclude("SPARK-36286: invalid string cast to timestamp") + .exclude("SPARK-39749: cast Decimal to string") + enableSuite[GlutenTryCastSuite] + .exclude( + "Process Infinity, -Infinity, NaN in case insensitive manner" // +inf not supported in folly. + ) + .exclude("cast from timestamp II") // Rewrite test for Gluten not supported with ANSI mode + .exclude("ANSI mode: Throw exception on casting out-of-range value to byte type") + .exclude("ANSI mode: Throw exception on casting out-of-range value to short type") + .exclude("ANSI mode: Throw exception on casting out-of-range value to int type") + .exclude("ANSI mode: Throw exception on casting out-of-range value to long type") + .exclude("cast from invalid string to numeric should throw NumberFormatException") + .exclude("SPARK-26218: Fix the corner case of codegen when casting float to Integer") + // Set timezone through config. + .exclude("data type casting") + // Revised by setting timezone through config and commented unsupported cases. + .exclude("cast string to timestamp") + enableSuite[GlutenCollectionExpressionsSuite] + // Rewrite in Gluten to replace Seq with Array + .exclude("Shuffle") + .excludeGlutenTest("Shuffle") + enableSuite[GlutenConditionalExpressionSuite] + enableSuite[GlutenDateExpressionsSuite] + // Has exception in fallback execution when we use resultDF.collect in evaluation. + .exclude("TIMESTAMP_MICROS") + // Replaced by a gluten test to pass timezone through config. + .exclude("unix_timestamp") + // Replaced by a gluten test to pass timezone through config. + .exclude("to_unix_timestamp") + // Replaced by a gluten test to pass timezone through config. + .exclude("Hour") + // Unsupported format: yyyy-MM-dd HH:mm:ss.SSS + .exclude("SPARK-33498: GetTimestamp,UnixTimestamp,ToUnixTimestamp with parseError") + // Replaced by a gluten test to pass timezone through config. + .exclude("DateFormat") + // Legacy mode is not supported, assuming this mode is not commonly used. + .exclude("to_timestamp exception mode") + // Replaced by a gluten test to pass timezone through config. + .exclude("from_unixtime") + // Vanilla Spark does not have a unified DST Timestamp fastTime. 1320570000000L and + // 1320566400000L both represent 2011-11-06 01:00:00. + .exclude("SPARK-42635: timestampadd near daylight saving transition") + // https://github.com/facebookincubator/velox/pull/10563/files#diff-140dc50e6dac735f72d29014da44b045509df0dd1737f458de1fe8cfd33d8145 + .excludeGlutenTest("from_unixtime") + // Replaced by a gluten test to pass timezone through config. + .exclude("months_between") + enableSuite[GlutenDecimalExpressionSuite] + enableSuite[GlutenDecimalPrecisionSuite] + enableSuite[GlutenGeneratorExpressionSuite] + enableSuite[GlutenHashExpressionsSuite] + enableSuite[GlutenHigherOrderFunctionsSuite] + enableSuite[GlutenIntervalExpressionsSuite] + enableSuite[GlutenJsonExpressionsSuite] + // https://github.com/apache/incubator-gluten/issues/10948 + .exclude("$['key with spaces']") + // https://github.com/apache/incubator-gluten/issues/8102 + .exclude("$.store.book") + .exclude("$") + .exclude("$.store.book[0]") + .exclude("$.store.book[*]") + .exclude("$.store.book[*].category") + .exclude("$.store.book[*].isbn") + .exclude("$.store.book[*].reader") + .exclude("$.store.basket[*]") + .exclude("$.store.basket[*][0]") + .exclude("$.store.basket[0][*]") + .exclude("$.store.basket[*][*]") + .exclude("$.store.basket[0][*].b") + // Exception class different. + .exclude("from_json - invalid data") + enableSuite[GlutenJsonFunctionsSuite] + // * in get_json_object expression not supported in velox + .exclude("SPARK-42782: Hive compatibility check for get_json_object") + // Velox does not support single quotes in get_json_object function. + .exclude("function get_json_object - support single quotes") + .exclude("function get_json_object - path is null") + .exclude("function get_json_object - json is null") + .exclude("function get_json_object - Codegen Support") + enableSuite[GlutenLiteralExpressionSuite] + .exclude("default") + // FIXME(yma11): ObjectType is not covered in RowEncoder/Serializer in vanilla spark + .exclude("SPARK-37967: Literal.create support ObjectType") + enableSuite[GlutenMathExpressionsSuite] + // Spark round UT for round(3.1415,3) is not correct. + .exclude("round/bround/floor/ceil") + enableSuite[GlutenMiscExpressionsSuite] + enableSuite[GlutenNondeterministicSuite] + .exclude("MonotonicallyIncreasingID") + .exclude("SparkPartitionID") + enableSuite[GlutenNullExpressionsSuite] + enableSuite[GlutenPredicateSuite] + enableSuite[GlutenRandomSuite] + .exclude("random") + .exclude("SPARK-9127 codegen with long seed") + enableSuite[GlutenRegexpExpressionsSuite] + // TODO: fix on Spark-4.1 introduced by https://github.com/apache/spark/pull/48470 + .exclude("SPLIT") + enableSuite[GlutenSortShuffleSuite] + enableSuite[GlutenSortOrderExpressionsSuite] + enableSuite[GlutenStringExpressionsSuite] + enableSuite[GlutenTryEvalSuite] + enableSuite[VeloxAdaptiveQueryExecSuite] + .includeAllGlutenTests() + .includeByPrefix( + "SPARK-30291", + "SPARK-30403", + "SPARK-30719", + "SPARK-31384", + "SPARK-31658", + "SPARK-32717", + "SPARK-32649", + "SPARK-34533", + "SPARK-34781", + "SPARK-35585", + "SPARK-32932", + "SPARK-33494", + "SPARK-33933", + "SPARK-31220", + "SPARK-35874", + "SPARK-39551" + ) + .include( + "Union/Except/Intersect queries", + "Subquery de-correlation in Union queries", + "force apply AQE", + "tree string output", + "control a plan explain mode in listener vis SQLConf", + "AQE should set active session during execution", + "No deadlock in UI update", + "SPARK-35455: Unify empty relation optimization between normal and AQE optimizer - multi join" + ) + enableSuite[GlutenBinaryFileFormatSuite] + // Exception. + .exclude("column pruning - non-readable file") + // TODO: fix in Spark-4.0 + // enableSuite[GlutenCSVv1Suite] + // // file cars.csv include null string, Arrow not support to read + // .exclude("DDL test with schema") + // .exclude("save csv") + // .exclude("save csv with compression codec option") + // .exclude("save csv with empty fields with user defined empty values") + // .exclude("save csv with quote") + // .exclude("SPARK-13543 Write the output as uncompressed via option()") + // .exclude("DDL test with tab separated file") + // .exclude("DDL test parsing decimal type") + // .exclude("test with tab delimiter and double quote") + // // Arrow not support corrupt record + // .exclude("SPARK-27873: disabling enforceSchema should not fail columnNameOfCorruptRecord") + // // varchar + // .exclude("SPARK-48241: CSV parsing failure with char/varchar type columns") + // // Flaky and already excluded in other cases + // .exclude("Gluten - test for FAILFAST parsing mode") + + // enableSuite[GlutenCSVv2Suite] + // .exclude("Gluten - test for FAILFAST parsing mode") + // // Rule org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown in batch + // // Early Filter and Projection Push-Down generated an invalid plan + // .exclude("SPARK-26208: write and read empty data to csv file with headers") + // // file cars.csv include null string, Arrow not support to read + // .exclude("old csv data source name works") + // .exclude("DDL test with schema") + // .exclude("save csv") + // .exclude("save csv with compression codec option") + // .exclude("save csv with empty fields with user defined empty values") + // .exclude("save csv with quote") + // .exclude("SPARK-13543 Write the output as uncompressed via option()") + // .exclude("DDL test with tab separated file") + // .exclude("DDL test parsing decimal type") + // .exclude("test with tab delimiter and double quote") + // // Arrow not support corrupt record + // .exclude("SPARK-27873: disabling enforceSchema should not fail columnNameOfCorruptRecord") + // // varchar + // .exclude("SPARK-48241: CSV parsing failure with char/varchar type columns") + + // enableSuite[GlutenCSVLegacyTimeParserSuite] + // // file cars.csv include null string, Arrow not support to read + // .exclude("DDL test with schema") + // .exclude("save csv") + // .exclude("save csv with compression codec option") + // .exclude("save csv with empty fields with user defined empty values") + // .exclude("save csv with quote") + // .exclude("SPARK-13543 Write the output as uncompressed via option()") + // // Arrow not support corrupt record + // .exclude("SPARK-27873: disabling enforceSchema should not fail columnNameOfCorruptRecord") + // .exclude("DDL test with tab separated file") + // .exclude("DDL test parsing decimal type") + // .exclude("test with tab delimiter and double quote") + // // varchar + // .exclude("SPARK-48241: CSV parsing failure with char/varchar type columns") + enableSuite[GlutenJsonV1Suite] + // FIXME: Array direct selection fails + .exclude("Complex field and type inferring") + .exclude("SPARK-4228 DataFrame to JSON") + enableSuite[GlutenJsonV2Suite] + // exception test + .exclude("SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern") + .exclude("Complex field and type inferring") + .exclude("SPARK-4228 DataFrame to JSON") + enableSuite[GlutenJsonLegacyTimeParserSuite] + .exclude("Complex field and type inferring") + .exclude("SPARK-4228 DataFrame to JSON") + enableSuite[GlutenValidateRequirementsSuite] + enableSuite[GlutenOrcColumnarBatchReaderSuite] + enableSuite[GlutenOrcFilterSuite] + .exclude("SPARK-32622: case sensitivity in predicate pushdown") + enableSuite[GlutenOrcPartitionDiscoverySuite] + .exclude("read partitioned table - normal case") + .exclude("read partitioned table - with nulls") + enableSuite[GlutenOrcV1PartitionDiscoverySuite] + .exclude("read partitioned table - normal case") + .exclude("read partitioned table - with nulls") + .exclude("read partitioned table - partition key included in orc file") + .exclude("read partitioned table - with nulls and partition keys are included in Orc file") + enableSuite[GlutenOrcV1QuerySuite] + // For exception test. + .exclude("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") + .exclude("Read/write binary data") + .exclude("Read/write all types with non-primitive type") + .exclude("Creating case class RDD table") + .exclude("save and load case class RDD with `None`s as orc") + .exclude("SPARK-16610: Respect orc.compress (i.e., OrcConf.COMPRESS) when" + + " compression is unset") + .exclude("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") + .exclude("appending") + .exclude("nested data - struct with array field") + .exclude("nested data - array of struct") + .exclude("SPARK-9170: Don't implicitly lowercase of user-provided columns") + .exclude("SPARK-10623 Enable ORC PPD") + .exclude("SPARK-14962 Produce correct results on array type with isnotnull") + .exclude("SPARK-15198 Support for pushing down filters for boolean types") + .exclude("Support for pushing down filters for decimal types") + .exclude("Support for pushing down filters for timestamp types") + .exclude("column nullability and comment - write and then read") + .exclude("Empty schema does not read data from ORC file") + .exclude("read from multiple orc input paths") + .exclude("Enabling/disabling ignoreCorruptFiles") + .exclude("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") + .exclude("LZO compression options for writing to an ORC file") + .exclude("Schema discovery on empty ORC files") + .exclude("SPARK-21791 ORC should support column names with dot") + .exclude("SPARK-25579 ORC PPD should support column names with dot") + .exclude("SPARK-34862: Support ORC vectorized reader for nested column") + .exclude("SPARK-37728: Reading nested columns with ORC vectorized reader should not") + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + .exclude("Read/write all timestamp types") + .exclude("SPARK-37463: read/write Timestamp ntz to Orc with different time zone") + .exclude("SPARK-39381: Make vectorized orc columar writer batch size configurable") + .exclude("SPARK-39830: Reading ORC table that requires type promotion may throw AIOOBE") + enableSuite[GlutenOrcV2QuerySuite] + .exclude("Read/write binary data") + .exclude("Read/write all types with non-primitive type") + // Rewrite to disable Spark's columnar reader. + .exclude("Simple selection form ORC table") + .exclude("Creating case class RDD table") + .exclude("save and load case class RDD with `None`s as orc") + .exclude("SPARK-16610: Respect orc.compress (i.e., OrcConf.COMPRESS) when compression is unset") + .exclude("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") + .exclude("appending") + .exclude("nested data - struct with array field") + .exclude("nested data - array of struct") + .exclude("SPARK-9170: Don't implicitly lowercase of user-provided columns") + .exclude("SPARK-10623 Enable ORC PPD") + .exclude("SPARK-14962 Produce correct results on array type with isnotnull") + .exclude("SPARK-15198 Support for pushing down filters for boolean types") + .exclude("Support for pushing down filters for decimal types") + .exclude("Support for pushing down filters for timestamp types") + .exclude("column nullability and comment - write and then read") + .exclude("Empty schema does not read data from ORC file") + .exclude("read from multiple orc input paths") + .exclude("Enabling/disabling ignoreCorruptFiles") + .exclude("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") + .exclude("LZO compression options for writing to an ORC file") + .exclude("Schema discovery on empty ORC files") + .exclude("SPARK-21791 ORC should support column names with dot") + .exclude("SPARK-25579 ORC PPD should support column names with dot") + .exclude("SPARK-34862: Support ORC vectorized reader for nested column") + .exclude("SPARK-37728: Reading nested columns with ORC vectorized reader should not") + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + .exclude("Read/write all timestamp types") + .exclude("SPARK-37463: read/write Timestamp ntz to Orc with different time zone") + .exclude("SPARK-39381: Make vectorized orc columar writer batch size configurable") + .exclude("SPARK-39830: Reading ORC table that requires type promotion may throw AIOOBE") + .exclude("simple select queries") + .exclude("overwriting") + .exclude("self-join") + .exclude("columns only referenced by pushed down filters should remain") + .exclude("SPARK-5309 strings stored using dictionary compression in orc") + // For exception test. + .exclude("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") + enableSuite[GlutenOrcSourceSuite] + // https://github.com/apache/incubator-gluten/issues/11218 + .exclude("SPARK-31238: compatibility with Spark 2.4 in reading dates") + .exclude("SPARK-31238, SPARK-31423: rebasing dates in write") + // Ignored to disable vectorized reading check. + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + .exclude("create temporary orc table") + .exclude("create temporary orc table as") + .exclude("appending insert") + .exclude("overwrite insert") + .exclude("SPARK-34897: Support reconcile schemas based on index after nested column pruning") + .excludeGlutenTest("SPARK-31238: compatibility with Spark 2.4 in reading dates") + .excludeGlutenTest("SPARK-31238, SPARK-31423: rebasing dates in write") + .excludeGlutenTest("SPARK-34862: Support ORC vectorized reader for nested column") + // exclude as struct not supported + .exclude("SPARK-36663: OrcUtils.toCatalystSchema should correctly handle a column name which consists of only numbers") + .exclude("SPARK-37812: Reuse result row when deserializing a struct") + // rewrite + .exclude("SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=true, spark.sql.orc.enableNestedColumnVectorizedReader=true)") + .exclude("SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=true, spark.sql.orc.enableNestedColumnVectorizedReader=false)") + enableSuite[GlutenOrcV1FilterSuite] + .exclude("SPARK-32622: case sensitivity in predicate pushdown") + enableSuite[GlutenOrcV1SchemaPruningSuite] + enableSuite[GlutenOrcV2SchemaPruningSuite] + enableSuite[GlutenParquetColumnIndexSuite] + enableSuite[GlutenParquetCompressionCodecPrecedenceSuite] + enableSuite[GlutenParquetDeltaByteArrayEncodingSuite] + enableSuite[GlutenParquetDeltaEncodingInteger] + enableSuite[GlutenParquetDeltaEncodingLong] + enableSuite[GlutenParquetDeltaLengthByteArrayEncodingSuite] + enableSuite[GlutenParquetEncodingSuite] + enableSuite[GlutenParquetFieldIdIOSuite] + enableSuite[GlutenParquetFileFormatV1Suite] + enableSuite[GlutenParquetFileFormatV2Suite] + enableSuite[GlutenParquetV1FilterSuite] + // Rewrite. + .exclude("SPARK-23852: Broken Parquet push-down for partially-written stats") + // Rewrite for supported INT96 - timestamp. + .exclude("filter pushdown - timestamp") + .exclude("filter pushdown - date") + // Exception bebaviour. + .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") + // Ignore Spark's filter pushdown check. + .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") + .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") + .exclude("Filters should be pushed down for Parquet readers at row group level") + .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") + .exclude("Support Parquet column index") + .exclude("SPARK-34562: Bloom filter push down") + .exclude("SPARK-16371 Do not push down filters when inner name and outer name are the same") + .exclude("filter pushdown - StringPredicate") + .exclude("SPARK-38825: in and notIn filters") + enableSuite[GlutenParquetV2FilterSuite] + // Rewrite. + .exclude("SPARK-23852: Broken Parquet push-down for partially-written stats") + // Rewrite for supported INT96 - timestamp. + .exclude("filter pushdown - timestamp") + .exclude("filter pushdown - date") + // Exception bebaviour. + .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") + // Ignore Spark's filter pushdown check. + .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") + .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") + .exclude("Filters should be pushed down for Parquet readers at row group level") + .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") + .exclude("Support Parquet column index") + .exclude("SPARK-34562: Bloom filter push down") + .exclude("SPARK-16371 Do not push down filters when inner name and outer name are the same") + .exclude("filter pushdown - StringPredicate") + .exclude("SPARK-38825: in and notIn filters") + enableSuite[GlutenParquetInteroperabilitySuite] + .exclude("parquet timestamp conversion") + enableSuite[GlutenParquetIOSuite] + // Velox doesn't write file metadata into parquet file. + .exclude("Write Spark version into Parquet metadata") + // Exception. + .exclude("SPARK-35640: read binary as timestamp should throw schema incompatible error") + // Exception msg. + .exclude("SPARK-35640: int as long should throw schema incompatible error") + // Velox parquet reader not allow offset zero. + .exclude("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") + // TODO: fix in Spark-4.0 + .exclude("explode nested lists crossing a rowgroup boundary") + // TODO: fix on Spark-4.1 + .excludeByPrefix("SPARK-53535") // see https://issues.apache.org/jira/browse/SPARK-53535 + .excludeByPrefix("vectorized reader: missing all struct fields") + .excludeByPrefix("SPARK-54220") // https://issues.apache.org/jira/browse/SPARK-54220 + enableSuite[GlutenParquetV1PartitionDiscoverySuite] + enableSuite[GlutenParquetV2PartitionDiscoverySuite] + enableSuite[GlutenParquetProtobufCompatibilitySuite] + enableSuite[GlutenParquetV1QuerySuite] + .exclude("row group skipping doesn't overflow when reading into larger type") + // Unsupport spark.sql.files.ignoreCorruptFiles. + .exclude("Enabling/disabling ignoreCorruptFiles") + // decimal failed ut + .exclude("SPARK-34212 Parquet should read decimals correctly") + // new added in spark-3.3 and need fix later, random failure may caused by memory free + .exclude("SPARK-39833: pushed filters with project without filter columns") + .exclude("SPARK-39833: pushed filters with count()") + // Rewrite because the filter after datasource is not needed. + .exclude( + "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") + enableSuite[GlutenParquetV2QuerySuite] + .exclude("row group skipping doesn't overflow when reading into larger type") + // Unsupport spark.sql.files.ignoreCorruptFiles. + .exclude("Enabling/disabling ignoreCorruptFiles") + // decimal failed ut + .exclude("SPARK-34212 Parquet should read decimals correctly") + // Rewrite because the filter after datasource is not needed. + .exclude( + "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") + enableSuite[GlutenParquetV1SchemaPruningSuite] + enableSuite[GlutenParquetV2SchemaPruningSuite] + enableSuite[GlutenParquetRebaseDatetimeV1Suite] + // Velox doesn't write file metadata into parquet file. + .excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys") + .excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key") + // jar path and ignore PARQUET_REBASE_MODE_IN_READ, rewrite some + .excludeByPrefix("SPARK-31159") + .excludeByPrefix("SPARK-35427") + enableSuite[GlutenParquetRebaseDatetimeV2Suite] + // Velox doesn't write file metadata into parquet file. + .excludeByPrefix("SPARK-33163, SPARK-37705: write the metadata keys") + .excludeByPrefix("SPARK-33160, SPARK-37705: write the metadata key") + // jar path and ignore PARQUET_REBASE_MODE_IN_READ + .excludeByPrefix("SPARK-31159") + .excludeByPrefix("SPARK-35427") + enableSuite[GlutenParquetSchemaInferenceSuite] + enableSuite[GlutenParquetSchemaSuite] + // error message mismatch is accepted + .exclude("schema mismatch failure error message for parquet reader") + .exclude("schema mismatch failure error message for parquet vectorized reader") + // https://github.com/apache/incubator-gluten/issues/11220 + .excludeByPrefix("SPARK-40819") + .excludeByPrefix("SPARK-46056") // TODO: fix in Spark-4.0 + .exclude("CANNOT_MERGE_SCHEMAS: Failed merging schemas") + enableSuite[GlutenParquetThriftCompatibilitySuite] + // Rewrite for file locating. + .exclude("Read Parquet file generated by parquet-thrift") + enableSuite[GlutenParquetVectorizedSuite] + enableSuite[GlutenTextV1Suite] + enableSuite[GlutenTextV2Suite] + enableSuite[GlutenDataSourceV2StrategySuite] + enableSuite[GlutenFileTableSuite] + enableSuite[GlutenV2PredicateSuite] + enableSuite[GlutenBucketingUtilsSuite] + enableSuite[GlutenDataSourceStrategySuite] + enableSuite[GlutenDataSourceSuite] + enableSuite[GlutenFileFormatWriterSuite] + enableSuite[GlutenFileIndexSuite] + enableSuite[GlutenFileMetadataStructSuite] + enableSuite[GlutenParquetV1AggregatePushDownSuite] + enableSuite[GlutenParquetV2AggregatePushDownSuite] + // TODO: Timestamp columns stats will lost if using int64 in parquet writer. + .exclude("aggregate push down - different data types") + enableSuite[GlutenOrcV1AggregatePushDownSuite] + .exclude("nested column: Count(nested sub-field) not push down") + enableSuite[GlutenOrcV2AggregatePushDownSuite] + .exclude("nested column: Max(top level column) not push down") + .exclude("nested column: Count(nested sub-field) not push down") + enableSuite[GlutenOrcCodecSuite] + enableSuite[GlutenFileSourceStrategySuite] + // Plan comparison. + .exclude("partitioned table - after scan filters") + enableSuite[GlutenHadoopFileLinesReaderSuite] + enableSuite[GlutenPathFilterStrategySuite] + enableSuite[GlutenPathFilterSuite] + enableSuite[GlutenPruneFileSourcePartitionsSuite] + // TODO: fix in Spark-4.0 + // enableSuite[GlutenCSVReadSchemaSuite] + // enableSuite[GlutenHeaderCSVReadSchemaSuite] + // .exclude("change column type from int to long") + enableSuite[GlutenJsonReadSchemaSuite] + enableSuite[GlutenOrcReadSchemaSuite] + enableSuite[GlutenVectorizedOrcReadSchemaSuite] + enableSuite[GlutenMergedOrcReadSchemaSuite] + enableSuite[GlutenParquetReadSchemaSuite] + enableSuite[GlutenVectorizedParquetReadSchemaSuite] + enableSuite[GlutenMergedParquetReadSchemaSuite] + enableSuite[GlutenParquetCodecSuite] + enableSuite[GlutenV1WriteCommandSuite] + // Rewrite to match SortExecTransformer. + .excludeByPrefix("SPARK-41914:") + enableSuite[GlutenEnsureRequirementsSuite] + + enableSuite[GlutenBroadcastJoinSuite] + .exclude("Shouldn't change broadcast join buildSide if user clearly specified") + .exclude("Shouldn't bias towards build right if user didn't specify") + .exclude("SPARK-23192: broadcast hint should be retained after using the cached data") + .exclude("broadcast join where streamed side's output partitioning is HashPartitioning") + + enableSuite[GlutenExistenceJoinSuite] + enableSuite[GlutenInnerJoinSuiteForceShjOn] + enableSuite[GlutenInnerJoinSuiteForceShjOff] + enableSuite[GlutenOuterJoinSuiteForceShjOn] + enableSuite[GlutenOuterJoinSuiteForceShjOff] + enableSuite[GlutenFallbackStrategiesSuite] + enableSuite[GlutenBroadcastExchangeSuite] + // TODO: fix on Spark-4.1 introduced by see https://github.com/apache/spark/pull/51623 + .exclude("SPARK-52962: broadcast exchange should not reset metrics") + enableSuite[GlutenLocalBroadcastExchangeSuite] + enableSuite[GlutenCoalesceShufflePartitionsSuite] + // Rewrite for Gluten. Change details are in the inline comments in individual tests. + .excludeByPrefix("determining the number of reducers") + enableSuite[GlutenExchangeSuite] + enableSuite[GlutenReplaceHashWithSortAggSuite] + // Rewrite to check plan and some adds order by for result sort order + .exclude("replace partial hash aggregate with sort aggregate") + .exclude("replace partial and final hash aggregate together with sort aggregate") + .exclude("do not replace hash aggregate if child does not have sort order") + .exclude("do not replace hash aggregate if there is no group-by column") + enableSuite[GlutenReuseExchangeAndSubquerySuite] + enableSuite[GlutenSameResultSuite] + enableSuite[GlutenSortSuite] + enableSuite[GlutenSQLAggregateFunctionSuite] + // spill not supported yet. + enableSuite[GlutenSQLWindowFunctionSuite] + .exclude("test with low buffer spill threshold") + enableSuite[GlutenTakeOrderedAndProjectSuite] + // The results of rand() differ between vanilla spark and velox. + .exclude("SPARK-47104: Non-deterministic expressions in projection") + enableSuite[GlutenSessionExtensionSuite] + enableSuite[TestFileSourceScanExecTransformer] + enableSuite[GlutenBucketedReadWithoutHiveSupportSuite] + // Exclude the following suite for plan changed from SMJ to SHJ. + .exclude("avoid shuffle when join 2 bucketed tables") + .exclude("avoid shuffle and sort when sort columns are a super set of join keys") + .exclude("only shuffle one side when join bucketed table and non-bucketed table") + .exclude("only shuffle one side when 2 bucketed tables have different bucket number") + .exclude("only shuffle one side when 2 bucketed tables have different bucket keys") + .exclude("shuffle when join keys are not equal to bucket keys") + .exclude("shuffle when join 2 bucketed tables with bucketing disabled") + .exclude("check sort and shuffle when bucket and sort columns are join keys") + .exclude("only sort one side when sort columns are different") + .exclude("only sort one side when sort columns are same but their ordering is different") + .exclude("SPARK-17698 Join predicates should not contain filter clauses") + .exclude("SPARK-19122 Re-order join predicates if they match with the child's" + + " output partitioning") + .exclude("SPARK-19122 No re-ordering should happen if set of join columns != set of child's " + + "partitioning columns") + .exclude("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") + .exclude("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") + .exclude("bucket coalescing eliminates shuffle") + .exclude("bucket coalescing is not satisfied") + // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .exclude("disable bucketing when the output doesn't contain all bucketing columns") + .excludeByPrefix("bucket coalescing is applied when join expressions match") + enableSuite[GlutenBucketedWriteWithoutHiveSupportSuite] + enableSuite[GlutenCreateTableAsSelectSuite] + // TODO Gluten can not catch the spark exception in Driver side. + .exclude("CREATE TABLE USING AS SELECT based on the file without write permission") + .exclude("create a table, drop it and create another one with the same name") + enableSuite[GlutenDDLSourceLoadSuite] + enableSuite[GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuite] + .disable( + "DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type") + enableSuite[GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuiteAE] + enableSuite[GlutenExternalCommandRunnerSuite] + enableSuite[GlutenFilteredScanSuite] + enableSuite[GlutenFiltersSuite] + enableSuite[GlutenInsertSuite] + // the native write staing dir is differnt with vanilla Spark for coustom partition paths + .exclude("SPARK-35106: Throw exception when rename custom partition paths returns false") + .exclude("Stop task set if FileAlreadyExistsException was thrown") + // Rewrite: Additional support for file scan with default values has been added in Spark-3.4. + // It appends the default value in record if it is not present while scanning. + // Velox supports default values for new records but it does not backfill the + // existing records and provides null for the existing ones. + .exclude("INSERT rows, ALTER TABLE ADD COLUMNS with DEFAULTs, then SELECT them") + .exclude("SPARK-39557 INSERT INTO statements with tables with array defaults") + .exclude("SPARK-39557 INSERT INTO statements with tables with struct defaults") + .exclude("SPARK-39557 INSERT INTO statements with tables with map defaults") + enableSuite[GlutenPartitionedWriteSuite] + enableSuite[GlutenPathOptionSuite] + enableSuite[GlutenPrunedScanSuite] + enableSuite[GlutenResolvedDataSourceSuite] + enableSuite[GlutenSaveLoadSuite] + enableSuite[GlutenTableScanSuite] + enableSuite[GlutenApproxCountDistinctForIntervalsQuerySuite] + enableSuite[GlutenApproximatePercentileQuerySuite] + enableSuite[GlutenCachedTableSuite] + .exclude("A cached table preserves the partitioning and ordering of its cached SparkPlan") + .exclude("InMemoryRelation statistics") + // Extra ColumnarToRow is needed to transform vanilla columnar data to gluten columnar data. + .exclude("SPARK-37369: Avoid redundant ColumnarToRow transition on InMemoryTableScan") + enableSuite[GlutenFileSourceCharVarcharTestSuite] + enableSuite[GlutenDSV2CharVarcharTestSuite] + enableSuite[GlutenColumnExpressionSuite] + // Velox raise_error('errMsg') throws a velox_user_error exception with the message 'errMsg'. + // The final caught Spark exception's getCause().getMessage() contains 'errMsg' but does not + // equal 'errMsg' exactly. The following two tests will be skipped and overridden in Gluten. + .exclude("raise_error") + .exclude("assert_true") + enableSuite[GlutenComplexTypeSuite] + enableSuite[GlutenConfigBehaviorSuite] + // Gluten columnar operator will have different number of jobs + .exclude("SPARK-40211: customize initialNumPartitions for take") + enableSuite[GlutenCountMinSketchAggQuerySuite] + enableSuite[GlutenCsvFunctionsSuite] + enableSuite[GlutenCTEHintSuite] + enableSuite[GlutenCTEInlineSuiteAEOff] + enableSuite[GlutenCTEInlineSuiteAEOn] + enableSuite[GlutenDataFrameAggregateSuite] + // Test for vanilla spark codegen, not apply for Gluten + .exclude("SPARK-43876: Enable fast hashmap for distinct queries") + .exclude( + "SPARK-26021: NaN and -0.0 in grouping expressions", // NaN case + // Replaced with another test. + "SPARK-19471: AggregationIterator does not initialize the generated result projection" + + " before using it", + // Velox's collect_list / collect_set are by design declarative aggregate so plan check + // for ObjectHashAggregateExec will fail. + "SPARK-22223: ObjectHashAggregate should not introduce unnecessary shuffle", + "SPARK-31620: agg with subquery (whole-stage-codegen = true)", + "SPARK-31620: agg with subquery (whole-stage-codegen = false)" + ) + enableSuite[GlutenDataFrameAsOfJoinSuite] + enableSuite[GlutenDataFrameComplexTypeSuite] + enableSuite[GlutenDataFrameFunctionsSuite] + // Rewrite this test because Velox sorts rows by key for primitive data types, which disrupts the original row sequence. + .exclude("map_zip_with function - map of primitive types") + // Vanilla spark throw SparkRuntimeException, gluten throw SparkException. + .exclude("map_concat function") + .exclude("transform keys function - primitive data types") + enableSuite[GlutenDataFrameHintSuite] + enableSuite[GlutenDataFrameImplicitsSuite] + enableSuite[GlutenDataFrameJoinSuite] + enableSuite[GlutenDataFrameNaFunctionsSuite] + enableSuite[GlutenDataFramePivotSuite] + enableSuite[GlutenDataFrameRangeSuite] + .exclude("SPARK-20430 Initialize Range parameters in a driver side") + .excludeByPrefix("Cancelling stage in a query with Range") + enableSuite[GlutenDataFrameSelfJoinSuite] + enableSuite[GlutenDataFrameSessionWindowingSuite] + enableSuite[GlutenDataFrameSetOperationsSuite] + // Ignore because it checks Spark's physical operators not ColumnarUnionExec + .exclude("SPARK-37371: UnionExec should support columnar if all children support columnar") + // Result depends on the implementation for nondeterministic expression rand. + // Not really an issue. + .exclude("SPARK-10740: handle nondeterministic expressions correctly for set operations") + // TODO: fix on Spark-4.1 + .excludeByPrefix("SPARK-52921") // see https://github.com/apache/spark/pull/51623 + enableSuite[GlutenDataFrameStatSuite] + enableSuite[GlutenDataFrameSuite] + // Rewrite these tests because it checks Spark's physical operators. + .excludeByPrefix("SPARK-22520", "reuse exchange") + .exclude( + /** + * Rewrite these tests because the rdd partition is equal to the configuration + * "spark.sql.shuffle.partitions". + */ + "repartitionByRange", + "distributeBy and localSort", + // Rewrite this test because the describe functions creates unmatched plan. + "describe", + // Result depends on the implementation for nondeterministic expression rand. + // Not really an issue. + "SPARK-9083: sort with non-deterministic expressions" + ) + // test for sort node not present but gluten uses shuffle hash join + .exclude("SPARK-41048: Improve output partitioning and ordering with AQE cache") + // Rewrite this test since it checks the physical operator which is changed in Gluten + .exclude("SPARK-27439: Explain result should match collected result after view change") + enableSuite[GlutenDataFrameTimeWindowingSuite] + enableSuite[GlutenDataFrameTungstenSuite] + enableSuite[GlutenDataFrameWindowFunctionsSuite] + // does not support `spark.sql.legacy.statisticalAggregate=true` (null -> NAN) + .exclude("corr, covar_pop, stddev_pop functions in specific window") + .exclude("covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window") + // does not support spill + .exclude("Window spill with more than the inMemoryThreshold and spillThreshold") + .exclude("SPARK-21258: complex object in combination with spilling") + // rewrite `WindowExec -> WindowExecTransformer` + .exclude( + "SPARK-38237: require all cluster keys for child required distribution for window query") + // TODO: fix on Spark-4.1 introduced by https://github.com/apache/spark/pull/47856 + .exclude( + "SPARK-49386: Window spill with more than the inMemoryThreshold and spillSizeThreshold") + enableSuite[GlutenDataFrameWindowFramesSuite] + enableSuite[GlutenDataFrameWriterV2Suite] + enableSuite[GlutenDatasetAggregatorSuite] + enableSuite[GlutenDatasetCacheSuite] + enableSuite[GlutenDatasetOptimizationSuite] + enableSuite[GlutenDatasetPrimitiveSuite] + enableSuite[GlutenDatasetSerializerRegistratorSuite] + enableSuite[GlutenDatasetSuite] + // Rewrite the following two tests in GlutenDatasetSuite. + .exclude("dropDuplicates: columns with same column name") + .exclude("groupBy.as") + enableSuite[GlutenDateFunctionsSuite] + // The below two are replaced by two modified versions. + .exclude("unix_timestamp") + .exclude("to_unix_timestamp") + // Unsupported datetime format: specifier X is not supported by velox. + .exclude("to_timestamp with microseconds precision") + // Legacy mode is not supported, assuming this mode is not commonly used. + .exclude("SPARK-30668: use legacy timestamp parser in to_timestamp") + // Legacy mode is not supported and velox getTimestamp function does not throw + // exception when format is "yyyy-dd-aa". + .exclude("function to_date") + enableSuite[GlutenDeprecatedAPISuite] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOff] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOn] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOnDisableScan] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOffDisableScan] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOff] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOn] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOnDisableScan] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOffDisableScan] + enableSuite[GlutenExpressionsSchemaSuite] + enableSuite[GlutenExtraStrategiesSuite] + enableSuite[GlutenFileBasedDataSourceSuite] + // test data path is jar path, rewrite + .exclude("Option recursiveFileLookup: disable partition inferring") + // gluten executor exception cannot get in driver, rewrite + .exclude("Spark native readers should respect spark.sql.caseSensitive - parquet") + // shuffle_partitions config is different, rewrite + .excludeByPrefix("SPARK-22790") + // plan is different cause metric is different, rewrite + .excludeByPrefix("SPARK-25237") + // error msg from velox is different & reader options is not supported, rewrite + .exclude("Enabling/disabling ignoreMissingFiles using parquet") + .exclude("Enabling/disabling ignoreMissingFiles using orc") + .exclude("Spark native readers should respect spark.sql.caseSensitive - orc") + .exclude("Return correct results when data columns overlap with partition columns") + .exclude("Return correct results when data columns overlap with partition " + + "columns (nested data)") + .exclude("SPARK-31116: Select nested schema with case insensitive mode") + // exclude as original metric not correct when task offloaded to velox + .exclude("SPARK-37585: test input metrics for DSV2 with output limits") + // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .exclude("File source v2: support passing data filters to FileScan without partitionFilters") + // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .exclude("File source v2: support partition pruning") + // DISABLED: GLUTEN-4893 Vanilla UT checks scan operator by exactly matching the class type + .exclude("SPARK-41017: filter pushdown with nondeterministic predicates") + enableSuite[GlutenFileScanSuite] + enableSuite[GlutenGeneratorFunctionSuite] + .exclude("SPARK-45171: Handle evaluated nondeterministic expression") + enableSuite[GlutenInjectRuntimeFilterSuite] + // FIXME: yan + .exclude("Merge runtime bloom filters") + enableSuite[GlutenIntervalFunctionsSuite] + enableSuite[GlutenJoinSuite] + // exclude as it check spark plan + .exclude("SPARK-36794: Ignore duplicated key when building relation for semi/anti hash join") + // TODO: fix on Spark-4.1 introduced by https://github.com/apache/spark/pull/47856 + .exclude("SPARK-49386: test SortMergeJoin (with spill by size threshold)") + enableSuite[GlutenMathFunctionsSuite] + enableSuite[GlutenMetadataCacheSuite] + .exclude("SPARK-16336,SPARK-27961 Suggest fixing FileNotFoundException") + enableSuite[GlutenMiscFunctionsSuite] + enableSuite[GlutenNestedDataSourceV1Suite] + enableSuite[GlutenNestedDataSourceV2Suite] + enableSuite[GlutenProcessingTimeSuite] + enableSuite[GlutenProductAggSuite] + enableSuite[GlutenReplaceNullWithFalseInPredicateEndToEndSuite] + enableSuite[GlutenScalaReflectionRelationSuite] + enableSuite[GlutenSerializationSuite] + enableSuite[GlutenFileSourceSQLInsertTestSuite] + enableSuite[GlutenDSV2SQLInsertTestSuite] + enableSuite[GlutenSQLQuerySuite] + // Decimal precision exceeds. + .exclude("should be able to resolve a persistent view") + // Unstable. Needs to be fixed. + .exclude("SPARK-36093: RemoveRedundantAliases should not change expression's name") + // Rewrite from ORC scan to Parquet scan because ORC is not well supported. + .exclude("SPARK-28156: self-join should not miss cached view") + .exclude("SPARK-33338: GROUP BY using literal map should not fail") + // Rewrite to disable plan check for SMJ because SHJ is preferred in Gluten. + .exclude("SPARK-11111 null-safe join should not use cartesian product") + // Rewrite to change the information of a caught exception. + .exclude("SPARK-33677: LikeSimplification should be skipped if pattern contains any escapeChar") + // Different exception. + .exclude("run sql directly on files") + // Not useful and time consuming. + .exclude("SPARK-33084: Add jar support Ivy URI in SQL") + .exclude("SPARK-33084: Add jar support Ivy URI in SQL -- jar contains udf class") + // exception test, rewritten in gluten + .exclude("the escape character is not allowed to end with") + // ORC related + .exclude("SPARK-37965: Spark support read/write orc file with invalid char in field name") + .exclude("SPARK-38173: Quoted column cannot be recognized correctly when quotedRegexColumnNames is true") + // Rewrite with Gluten's explained result. + .exclude("SPARK-47939: Explain should work with parameterized queries") + // TODO: fix on Spark-4.1 based on https://github.com/apache/incubator-gluten/pull/11252 + .excludeGlutenTest("SPARK-47939: Explain should work with parameterized queries") + enableSuite[GlutenSQLQueryTestSuite] + enableSuite[GlutenStatisticsCollectionSuite] + // The output byte size of Velox is different + .exclude("SPARK-33687: analyze all tables in a specific database") + .exclude("column stats collection for null columns") + .exclude("analyze column command - result verification") + enableSuite[GlutenSubquerySuite] + .excludeByPrefix( + "SPARK-26893" // Rewrite this test because it checks Spark's physical operators. + ) + // exclude as it checks spark plan + .exclude("SPARK-36280: Remove redundant aliases after RewritePredicateSubquery") + // TODO: fix in Spark-4.0 + .excludeByPrefix("SPARK-51738") + .excludeByPrefix("SPARK-43402") + .exclude("non-aggregated correlated scalar subquery") + .exclude("SPARK-18504 extra GROUP BY column in correlated scalar subquery is not permitted") + .exclude("SPARK-43402: FileSourceScanExec supports push down data filter with scalar subquery") + .exclude("SPARK-51738: IN subquery with struct type") + enableSuite[GlutenTypedImperativeAggregateSuite] + enableSuite[GlutenUnwrapCastInComparisonEndToEndSuite] + enableSuite[GlutenXPathFunctionsSuite] + enableSuite[GlutenFallbackSuite] + enableSuite[GlutenHiveSQLQuerySuite] + enableSuite[GlutenCollapseProjectExecTransformerSuite] + enableSuite[GlutenSparkSessionExtensionSuite] + enableSuite[GlutenGroupBasedDeleteFromTableSuite] + enableSuite[GlutenDeltaBasedDeleteFromTableSuite] + enableSuite[GlutenDataFrameToSchemaSuite] + enableSuite[GlutenDatasetUnpivotSuite] + enableSuite[GlutenLateralColumnAliasSuite] + enableSuite[GlutenParametersSuite] + enableSuite[GlutenResolveDefaultColumnsSuite] + enableSuite[GlutenSubqueryHintPropagationSuite] + enableSuite[GlutenUrlFunctionsSuite] + enableSuite[GlutenParquetRowIndexSuite] + .excludeByPrefix("row index generation") + .excludeByPrefix("invalid row index column type") + enableSuite[GlutenBitmapExpressionsQuerySuite] + enableSuite[GlutenEmptyInSuite] + enableSuite[GlutenRuntimeNullChecksV2Writes] + enableSuite[GlutenTableOptionsConstantFoldingSuite] + enableSuite[GlutenDeltaBasedMergeIntoTableSuite] + // Replaced by Gluten versions that handle wrapped exceptions + .excludeByPrefix("merge cardinality check with") + enableSuite[GlutenDeltaBasedMergeIntoTableUpdateAsDeleteAndInsertSuite] + // Replaced by Gluten versions that handle wrapped exceptions + .excludeByPrefix("merge cardinality check with") + enableSuite[GlutenDeltaBasedUpdateAsDeleteAndInsertTableSuite] + // FIXME: complex type result mismatch + .exclude("update nested struct fields") + .exclude("update char/varchar columns") + enableSuite[GlutenDeltaBasedUpdateTableSuite] + enableSuite[GlutenGroupBasedMergeIntoTableSuite] + // Replaced by Gluten versions that handle wrapped exceptions + .excludeByPrefix("merge cardinality check with") + enableSuite[GlutenFileSourceCustomMetadataStructSuite] + enableSuite[GlutenParquetFileMetadataStructRowIndexSuite] + enableSuite[GlutenTableLocationSuite] + enableSuite[GlutenRemoveRedundantWindowGroupLimitsSuite] + enableSuite[GlutenSQLCollectLimitExecSuite] + enableSuite[GlutenBatchEvalPythonExecSuite] + // Replaced with other tests that check for native operations + .exclude("Python UDF: push down deterministic FilterExec predicates") + .exclude("Nested Python UDF: push down deterministic FilterExec predicates") + .exclude("Python UDF: no push down on non-deterministic") + .exclude("Python UDF: push down on deterministic predicates after the first non-deterministic") + enableSuite[GlutenExtractPythonUDFsSuite] + // Replaced with test that check for native operations + .exclude("Python UDF should not break column pruning/filter pushdown -- Parquet V1") + .exclude("Chained Scalar Pandas UDFs should be combined to a single physical node") + .exclude("Mixed Batched Python UDFs and Pandas UDF should be separate physical node") + .exclude("Independent Batched Python UDFs and Scalar Pandas UDFs should be combined separately") + .exclude("Dependent Batched Python UDFs and Scalar Pandas UDFs should not be combined") + .exclude("Python UDF should not break column pruning/filter pushdown -- Parquet V2") + enableSuite[GlutenStreamingQuerySuite] + // requires test resources that don't exist in Gluten repo + .exclude("detect escaped path and report the migration guide") + .exclude("ignore the escaped path check when the flag is off") + .excludeByPrefix("SPARK-51187") + // Rewrite for the query plan check + .excludeByPrefix("SPARK-49905") + // TODO: fix on Spark-4.1 introduced by https://github.com/apache/spark/pull/52645 + .exclude("SPARK-53942: changing the number of stateless shuffle partitions via config") + .exclude("SPARK-53942: stateful shuffle partitions are retained from old checkpoint") + enableSuite[GlutenQueryExecutionSuite] + // Rewritten to set root logger level to INFO so that logs can be parsed + .exclude("Logging plan changes for execution") + // Rewrite for transformed plan + .exclude("dumping query execution info to a file - explainMode=formatted") + // TODO: fix in Spark-4.0 + .exclude("SPARK-47289: extended explain info") + // TODO: fix on Spark-4.1 introduced by https://github.com/apache/spark/pull/52157 + .exclude("SPARK-53413: Cleanup shuffle dependencies for commands") + override def getSQLQueryTestSettings: SQLQueryTestSettings = VeloxSQLQueryTestSettings +} +// scalastyle:on line.size.limit diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/GlutenSortShuffleSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/GlutenSortShuffleSuite.scala new file mode 100644 index 000000000000..70579c886248 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/GlutenSortShuffleSuite.scala @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark + +import org.apache.spark.sql.GlutenTestsBaseTrait + +class GlutenSortShuffleSuite extends SortShuffleSuite with GlutenTestsBaseTrait { + override def beforeAll(): Unit = { + super.beforeAll() + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenApproxCountDistinctForIntervalsQuerySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenApproxCountDistinctForIntervalsQuerySuite.scala new file mode 100644 index 000000000000..86ef1238965f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenApproxCountDistinctForIntervalsQuerySuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenApproxCountDistinctForIntervalsQuerySuite + extends ApproxCountDistinctForIntervalsQuerySuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenApproximatePercentileQuerySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenApproximatePercentileQuerySuite.scala new file mode 100644 index 000000000000..8fb40f1a20ec --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenApproximatePercentileQuerySuite.scala @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenApproximatePercentileQuerySuite + extends ApproximatePercentileQuerySuite + with GlutenSQLTestsTrait { + + override def testFile(fileName: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + fileName + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenBitmapExpressionsQuerySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenBitmapExpressionsQuerySuite.scala new file mode 100644 index 000000000000..e07821857a50 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenBitmapExpressionsQuerySuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenBitmapExpressionsQuerySuite + extends BitmapExpressionsQuerySuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenBloomFilterAggregateQuerySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenBloomFilterAggregateQuerySuite.scala new file mode 100644 index 000000000000..3eb59d8feca9 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenBloomFilterAggregateQuerySuite.scala @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.HashAggregateExecBaseTransformer + +import org.apache.spark.SparkConf +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.internal.SQLConf + +class GlutenBloomFilterAggregateQuerySuite + extends BloomFilterAggregateQuerySuite + with GlutenSQLTestsTrait + with AdaptiveSparkPlanHelper { + import testImplicits._ + + val veloxBloomFilterMaxNumBits = 4194304L + + testGluten("Test bloom_filter_agg with big RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS") { + val table = "bloom_filter_test" + withSQLConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS.key -> "5000000") { + val numEstimatedItems = 5000000L + val sqlString = s""" + |SELECT every(might_contain( + | (SELECT bloom_filter_agg(col, + | cast($numEstimatedItems as long), + | cast($veloxBloomFilterMaxNumBits as long)) + | FROM $table), + | col)) positive_membership_test + |FROM $table + """.stripMargin + withTempView(table) { + (Seq(Long.MinValue, 0, Long.MaxValue) ++ (1L to 200000L)) + .toDF("col") + .createOrReplaceTempView(table) + checkAnswer(spark.sql(sqlString), Row(true)) + } + } + } + + testGluten("Test that might_contain on bloom_filter_agg with empty input") { + checkAnswer( + spark.sql("""SELECT might_contain((select bloom_filter_agg(cast(id as long)) + | from range(1, 1)), cast(123 as long))""".stripMargin), + Row(null) + ) + + checkAnswer( + spark.sql("""SELECT might_contain((select bloom_filter_agg(cast(id as long)) + | from range(1, 1)), null)""".stripMargin), + Row(null)) + } + + testGluten("Test bloom_filter_agg filter fallback") { + val table = "bloom_filter_test" + val numEstimatedItems = 5000000L + val sqlString = s""" + |SELECT col positive_membership_test + |FROM $table + |WHERE might_contain( + | (SELECT bloom_filter_agg(col, + | cast($numEstimatedItems as long), + | cast($veloxBloomFilterMaxNumBits as long)) + | FROM $table), col) + """.stripMargin + withTempView(table) { + (Seq(Long.MinValue, 0, Long.MaxValue) ++ (1L to 200000L)) + .toDF("col") + .createOrReplaceTempView(table) + withSQLConf( + GlutenConfig.COLUMNAR_PROJECT_ENABLED.key -> "false" + ) { + val df = spark.sql(sqlString) + df.collect + assert( + collectWithSubqueries(df.queryExecution.executedPlan) { + case h if h.isInstanceOf[HashAggregateExecBaseTransformer] => h + }.size == 2, + df.queryExecution.executedPlan + ) + } + if (BackendsApiManager.getSettings.requireBloomFilterAggMightContainJointFallback()) { + withSQLConf( + GlutenConfig.COLUMNAR_FILTER_ENABLED.key -> "false" + ) { + val df = spark.sql(sqlString) + df.collect + assert( + collectWithSubqueries(df.queryExecution.executedPlan) { + case h if h.isInstanceOf[HashAggregateExecBaseTransformer] => h + }.size == 2, + df.queryExecution.executedPlan + ) + } + } + } + } + + testGluten("Test bloom_filter_agg agg fallback") { + val table = "bloom_filter_test" + val numEstimatedItems = 5000000L + val sqlString = s""" + |SELECT col positive_membership_test + |FROM $table + |WHERE might_contain( + | (SELECT bloom_filter_agg(col, + | cast($numEstimatedItems as long), + | cast($veloxBloomFilterMaxNumBits as long)) + | FROM $table), col) + """.stripMargin + + withTempView(table) { + (Seq(Long.MinValue, 0, Long.MaxValue) ++ (1L to 200000L)) + .toDF("col") + .createOrReplaceTempView(table) + withSQLConf( + GlutenConfig.COLUMNAR_HASHAGG_ENABLED.key -> "false" + ) { + val df = spark.sql(sqlString) + df.collect + assert( + collectWithSubqueries(df.queryExecution.executedPlan) { + case h if h.isInstanceOf[HashAggregateExecBaseTransformer] => h + }.isEmpty, + df.queryExecution.executedPlan + ) + } + } + } +} + +class GlutenBloomFilterAggregateQuerySuiteCGOff extends GlutenBloomFilterAggregateQuerySuite { + override def sparkConf: SparkConf = { + super.sparkConf + .set("spark.sql.codegen.wholeStage", "false") + .set("spark.sql.codegen.factoryMode", "NO_CODEGEN") + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCTEHintSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCTEHintSuite.scala new file mode 100644 index 000000000000..8005bffc310d --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCTEHintSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenCTEHintSuite extends CTEHintSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCTEInlineSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCTEInlineSuite.scala new file mode 100644 index 000000000000..3a05eda7119f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCTEInlineSuite.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.spark.SparkConf + +class GlutenCTEInlineSuiteAEOff extends CTEInlineSuiteAEOff with GlutenSQLTestsTrait { + override def sparkConf: SparkConf = + super.sparkConf + .set("spark.gluten.sql.columnar.backend.ch.enable.coalesce.project.union", "false") + +} + +class GlutenCTEInlineSuiteAEOn extends CTEInlineSuiteAEOn with GlutenSQLTestsTrait { + override def sparkConf: SparkConf = + super.sparkConf + .set("spark.gluten.sql.columnar.backend.ch.enable.coalesce.project.union", "false") + +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala new file mode 100644 index 000000000000..0afabae6e5fd --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.gluten.config.GlutenConfig + +import org.apache.spark.SparkConf +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.columnar.InMemoryRelation +import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike +import org.apache.spark.sql.internal.SQLConf + +class GlutenCachedTableSuite + extends CachedTableSuite + with GlutenSQLTestsTrait + with AdaptiveSparkPlanHelper { + import testImplicits._ + // for temporarily disable the columnar table cache globally. + sys.props.put(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED.key, "true") + override def sparkConf: SparkConf = { + super.sparkConf.set("spark.sql.shuffle.partitions", "5") + super.sparkConf.set(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED.key, "true") + } + + testGluten("InMemoryRelation statistics") { + sql("CACHE TABLE testData") + spark.table("testData").queryExecution.withCachedData.collect { + case cached: InMemoryRelation => + assert(cached.stats.sizeInBytes === 1132) + } + } + + def verifyNumExchanges(df: DataFrame, expected: Int): Unit = { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + df.collect() + } + assert(collect(df.queryExecution.executedPlan) { + case e: ShuffleExchangeLike => e + }.size == expected) + } + + testGluten("A cached table preserves the partitioning and ordering of its cached SparkPlan") { + // Distribute the tables into non-matching number of partitions. Need to shuffle one side. + withTempView("t1", "t2") { + testData.repartition(6, $"key").createOrReplaceTempView("t1") + testData2.repartition(3, $"a").createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") + val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a") + verifyNumExchanges(query, 1) + assert(stripAQEPlan(query.queryExecution.executedPlan).outputPartitioning.numPartitions === 6) + checkAnswer( + query, + testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) + uncacheTable("t1") + uncacheTable("t2") + } + + // One side of join is not partitioned in the desired way. Need to shuffle one side. + withTempView("t1", "t2") { + testData.repartition(6, $"value").createOrReplaceTempView("t1") + testData2.repartition(6, $"a").createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") + + val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a") + verifyNumExchanges(query, 1) + assert(stripAQEPlan(query.queryExecution.executedPlan).outputPartitioning.numPartitions === 6) + checkAnswer( + query, + testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) + uncacheTable("t1") + uncacheTable("t2") + } + + withTempView("t1", "t2") { + testData.repartition(6, $"value").createOrReplaceTempView("t1") + testData2.repartition(12, $"a").createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") + + val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a") + verifyNumExchanges(query, 1) + assert( + stripAQEPlan(query.queryExecution.executedPlan).outputPartitioning.numPartitions === 12) + checkAnswer( + query, + testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) + uncacheTable("t1") + uncacheTable("t2") + } + + // One side of join is not partitioned in the desired way. We'll only shuffle this side. + withTempView("t1", "t2") { + testData.repartition(6, $"value").createOrReplaceTempView("t1") + testData2.repartition(3, $"a").createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") + + val query = sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a") + verifyNumExchanges(query, 1) + checkAnswer( + query, + testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) + uncacheTable("t1") + uncacheTable("t2") + } + + // repartition's column ordering is different from group by column ordering. + // But they use the same set of columns. + withTempView("t1") { + testData.repartition(6, $"value", $"key").createOrReplaceTempView("t1") + spark.catalog.cacheTable("t1") + + val query = sql("SELECT value, key from t1 group by key, value") + verifyNumExchanges(query, 0) + checkAnswer(query, testData.distinct().select($"value", $"key")) + uncacheTable("t1") + } + + // repartition's column ordering is different from join condition's column ordering. + // We will still shuffle because hashcodes of a row depend on the column ordering. + // If we do not shuffle, we may actually partition two tables in totally two different way. + // See PartitioningSuite for more details. + withTempView("t1", "t2") { + val df1 = testData + df1.repartition(6, $"value", $"key").createOrReplaceTempView("t1") + val df2 = testData2.select($"a", $"b".cast("string")) + df2.repartition(6, $"a", $"b").createOrReplaceTempView("t2") + spark.catalog.cacheTable("t1") + spark.catalog.cacheTable("t2") + + val query = + sql("SELECT key, value, a, b FROM t1 t1 JOIN t2 t2 ON t1.key = t2.a and t1.value = t2.b") + verifyNumExchanges(query, 1) + assert(stripAQEPlan(query.queryExecution.executedPlan).outputPartitioning.numPartitions === 6) + checkAnswer( + query, + df1.join(df2, $"key" === $"a" && $"value" === $"b").select($"key", $"value", $"a", $"b")) + uncacheTable("t1") + uncacheTable("t2") + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala new file mode 100644 index 000000000000..ed40918de6bc --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.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 + +import org.apache.spark.{SparkException, SparkRuntimeException, SparkThrowable} + +trait GlutenCharVarcharTestSuite extends CharVarcharTestSuite with GlutenSQLTestsTrait { + protected val ERROR_MESSAGE = + "Exceeds char/varchar type length limitation: 5" + + protected val VELOX_ERROR_MESSAGE = + "Exceeds allowed length limitation: 5" + + override def assertLengthCheckFailure(func: () => Unit): Unit = { + val e = intercept[SparkThrowable](func()) + e match { + // Spark throws exception + case _: SparkRuntimeException => + checkError( + exception = e, + condition = "EXCEED_LIMIT_LENGTH", + parameters = Map("limit" -> "5") + ) + // Gluten throws exception. but sometimes, Spark exception is wrapped in GlutenException. + case e: SparkException => + assert(e.getMessage.contains(VELOX_ERROR_MESSAGE) || e.getMessage.contains(ERROR_MESSAGE)) + case _ => throw new RuntimeException(s"Unexpected exception: $e") + } + } +} + +class GlutenFileSourceCharVarcharTestSuite + extends FileSourceCharVarcharTestSuite + with GlutenCharVarcharTestSuite { + private def testTableWrite(f: String => Unit): Unit = { + withTable("t")(f("char")) + withTable("t")(f("varchar")) + } + + testGluten("length check for input string values: nested in array") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c ARRAY<$typeName(5)>) USING $format") + sql("INSERT INTO t VALUES (array(null))") + checkAnswer(spark.table("t"), Row(Seq(null))) + val e = intercept[SparkException] { + sql("INSERT INTO t VALUES (array('a', '123456'))") + } + assert(e.getMessage.contains(VELOX_ERROR_MESSAGE)) + } + } +} + +class GlutenDSV2CharVarcharTestSuite + extends DSV2CharVarcharTestSuite + with GlutenCharVarcharTestSuite {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala new file mode 100644 index 000000000000..6ee95026fdfb --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.gluten.config.GlutenConfig + +import org.apache.spark.SparkRuntimeException +import org.apache.spark.sql.execution.ProjectExec +import org.apache.spark.sql.functions.{assert_true, expr, input_file_name, lit} + +class GlutenColumnExpressionSuite extends ColumnExpressionSuite with GlutenSQLTestsTrait { + import testImplicits._ + + // TODO: fix in Spark-4.0 + // ignore("raise_error") { + // val strDf = Seq(("hello")).toDF("a") + + // val e1 = intercept[SparkRuntimeException] { + // strDf.select(raise_error(lit(null.asInstanceOf[String]))).collect() + // } + // assert(e1.getCause.isInstanceOf[RuntimeException]) + + // val e2 = intercept[SparkRuntimeException] { + // strDf.select(raise_error($"a")).collect() + // } + // assert(e2.getCause.isInstanceOf[RuntimeException]) + // assert(e2.getCause.getMessage.contains("hello")) + // } + + // TODO: fix in Spark-4.0 + ignoreGluten("assert_true") { + // assert_true(condition, errMsgCol) + val booleanDf = Seq((true), (false)).toDF("cond") + checkAnswer( + booleanDf.filter("cond = true").select(assert_true($"cond")), + Row(null) :: Nil + ) + val e1 = intercept[SparkRuntimeException] { + booleanDf.select(assert_true($"cond", lit(null.asInstanceOf[String]))).collect() + } + assert(e1.getCause.isInstanceOf[RuntimeException]) + + val nullDf = Seq(("first row", None), ("second row", Some(true))).toDF("n", "cond") + checkAnswer( + nullDf.filter("cond = true").select(assert_true($"cond", $"cond")), + Row(null) :: Nil + ) + val e2 = intercept[SparkRuntimeException] { + nullDf.select(assert_true($"cond", $"n")).collect() + } + assert(e2.getCause.isInstanceOf[RuntimeException]) + assert(e2.getCause.getMessage.contains("first row")) + + // assert_true(condition) + val intDf = Seq((0, 1)).toDF("a", "b") + checkAnswer(intDf.select(assert_true($"a" < $"b")), Row(null) :: Nil) + val e3 = intercept[SparkRuntimeException] { + intDf.select(assert_true($"a" > $"b")).collect() + } + assert(e3.getCause.isInstanceOf[RuntimeException]) + assert(e3.getCause.getMessage.contains("'('a > 'b)' is not true!")) + } + + testGluten( + "input_file_name, input_file_block_start and input_file_block_length " + + "should fall back if scan falls back") { + withSQLConf((GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key, "false")) { + withTempPath { + dir => + val data = sparkContext.parallelize(0 to 10).toDF("id") + data.write.parquet(dir.getCanonicalPath) + + val q = + spark.read + .parquet(dir.getCanonicalPath) + .select( + input_file_name(), + expr("input_file_block_start()"), + expr("input_file_block_length()")) + val firstRow = q.head() + assert(firstRow.getString(0).contains(dir.toURI.getPath)) + assert(firstRow.getLong(1) == 0) + assert(firstRow.getLong(2) > 0) + val project = q.queryExecution.executedPlan.collect { case p: ProjectExec => p } + assert(project.size == 1) + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenComplexTypesSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenComplexTypesSuite.scala new file mode 100644 index 000000000000..835b7ecfd88b --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenComplexTypesSuite.scala @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenComplexTypesSuite extends ComplexTypesSuite with GlutenSQLTestsTrait { + + override def beforeAll(): Unit = { + super.beforeAll() + spark + .range(10) + .selectExpr( + "(id % 2 = 0) as bool", + "cast(id as BYTE) as i8", + "cast(id as SHORT) as i16", + "cast(id as FLOAT) as fp32", + "cast(id as DOUBLE) as fp64", + "cast(id as DECIMAL(4, 2)) as dec", + "cast(cast(id as BYTE) as BINARY) as vbin", + "binary(id) as vbin1", + "map_from_arrays(array(id),array(id+2)) as map", + "array(id, id+1, id+2) as list", + "struct(cast(id as LONG) as a, cast(id+1 as STRING) as b) as struct" + ) + .write + .saveAsTable("tab_types") + } + + override def afterAll(): Unit = { + try { + spark.sql("DROP TABLE IF EXISTS tab_types") + } finally { + super.afterAll() + } + } + + testGluten("types bool/byte/short/float/double/decimal/binary/map/array/struct") { + val df = spark + .table("tab_types") + .selectExpr( + "bool", + "i8", + "i16", + "fp32", + "fp64", + "dec", + "vbin", + "length(vbin)", + "vbin1", + "length(vbin1)", + "struct", + "struct.a", + "list", + "map" + ) + .sort("i8") + .limit(1) + + checkAnswer( + df, + Seq( + Row( + true, + 0.toByte, + 0.toShort, + 0.toFloat, + 0.toDouble, + BigDecimal(0), + Array.fill[Byte](1)(0.toByte), + 1.toInt, + Array.fill[Byte](8)(0.toByte), + 8.toInt, + Row(0.toLong, "1"), + 0.toLong, + Array(0, 1, 2), + Map(0 -> 2) + )) + ) + + checkNamedStruct(df.queryExecution.optimizedPlan, expectedCount = 0) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenConfigBehaviorSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenConfigBehaviorSuite.scala new file mode 100644 index 000000000000..c1984a5e22dd --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenConfigBehaviorSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenConfigBehaviorSuite extends ConfigBehaviorSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCountMinSketchAggQuerySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCountMinSketchAggQuerySuite.scala new file mode 100644 index 000000000000..182464c0a5ee --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCountMinSketchAggQuerySuite.scala @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +/** End-to-end test suite for count_min_sketch. */ +class GlutenCountMinSketchAggQuerySuite + extends CountMinSketchAggQuerySuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCsvFunctionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCsvFunctionsSuite.scala new file mode 100644 index 000000000000..0550fef442ff --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCsvFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenCsvFunctionsSuite extends CsvFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameAggregateSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameAggregateSuite.scala new file mode 100644 index 000000000000..2f3777caa174 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameAggregateSuite.scala @@ -0,0 +1,283 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.HashAggregateExecBaseTransformer + +import org.apache.spark.sql.execution.WholeStageCodegenExec +import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} +import org.apache.spark.sql.expressions.Aggregator +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf + +import java.lang.{Long => JLong} + +import scala.util.Random + +class GlutenDataFrameAggregateSuite extends DataFrameAggregateSuite with GlutenSQLTestsTrait { + + import testImplicits._ + + // blackTestNameList is defined in ClickHouseNotSupport + + testGluten("count") { + // agg with no input col + assert(testData2.count() === testData2.rdd.map(_ => 1).count()) + + checkAnswer( + testData2.agg(count($"a"), sum_distinct($"a")), // non-partial + Row(6, 6.0)) + } + + testGluten("null count") { + checkAnswer(testData3.groupBy($"a").agg(count($"b")), Seq(Row(1, 0), Row(2, 1))) + + checkAnswer(testData3.groupBy($"a").agg(count($"a" + $"b")), Seq(Row(1, 0), Row(2, 1))) + + checkAnswer( + testData3 + .agg(count($"a"), count($"b"), count(lit(1)), count_distinct($"a"), count_distinct($"b")), + Row(2, 1, 2, 2, 1)) + + // [wishlist] does not support sum distinct +// checkAnswer( +// testData3.agg(count($"b"), count_distinct($"b"), sum_distinct($"b")), // non-partial +// Row(1, 1, 2) +// ) + } + + testGluten("groupBy") { + checkAnswer(testData2.groupBy("a").agg(sum($"b")), Seq(Row(1, 3), Row(2, 3), Row(3, 3))) + checkAnswer(testData2.groupBy("a").agg(sum($"b").as("totB")).agg(sum($"totB")), Row(9)) + checkAnswer(testData2.groupBy("a").agg(count("*")), Row(1, 2) :: Row(2, 2) :: Row(3, 2) :: Nil) + checkAnswer( + testData2.groupBy("a").agg(Map("*" -> "count")), + Row(1, 2) :: Row(2, 2) :: Row(3, 2) :: Nil) + checkAnswer( + testData2.groupBy("a").agg(Map("b" -> "sum")), + Row(1, 3) :: Row(2, 3) :: Row(3, 3) :: Nil) + + val df1 = Seq(("a", 1, 0, "b"), ("b", 2, 4, "c"), ("a", 2, 3, "d")) + .toDF("key", "value1", "value2", "rest") + + checkAnswer(df1.groupBy("key").min(), df1.groupBy("key").min("value1", "value2").collect()) + checkAnswer(df1.groupBy("key").min("value2"), Seq(Row("a", 0), Row("b", 4))) + + // [wishlist] does not support decimal +// checkAnswer( +// decimalData.groupBy("a").agg(sum("b")), +// Seq(Row(new java.math.BigDecimal(1), new java.math.BigDecimal(3)), +// Row(new java.math.BigDecimal(2), new java.math.BigDecimal(3)), +// Row(new java.math.BigDecimal(3), new java.math.BigDecimal(3))) +// ) +// +// val decimalDataWithNulls = spark.sparkContext.parallelize( +// DecimalData(1, 1) :: +// DecimalData(1, null) :: +// DecimalData(2, 1) :: +// DecimalData(2, null) :: +// DecimalData(3, 1) :: +// DecimalData(3, 2) :: +// DecimalData(null, 2) :: Nil).toDF() +// checkAnswer( +// decimalDataWithNulls.groupBy("a").agg(sum("b")), +// Seq(Row(new java.math.BigDecimal(1), new java.math.BigDecimal(1)), +// Row(new java.math.BigDecimal(2), new java.math.BigDecimal(1)), +// Row(new java.math.BigDecimal(3), new java.math.BigDecimal(3)), +// Row(null, new java.math.BigDecimal(2))) +// ) + } + + testGluten("average") { + + checkAnswer(testData2.agg(avg($"a"), mean($"a")), Row(2.0, 2.0)) + + checkAnswer( + testData2.agg(avg($"a"), sum_distinct($"a")), // non-partial and test deprecated version + Row(2.0, 6.0) :: Nil) + + // [wishlist] does not support decimal +// checkAnswer( +// decimalData.agg(avg($"a")), +// Row(new java.math.BigDecimal(2))) +// +// checkAnswer( +// decimalData.agg(avg($"a"), sum_distinct($"a")), // non-partial +// Row(new java.math.BigDecimal(2), new java.math.BigDecimal(6)) :: Nil) +// +// checkAnswer( +// decimalData.agg(avg($"a" cast DecimalType(10, 2))), +// Row(new java.math.BigDecimal(2))) +// // non-partial +// checkAnswer( +// decimalData.agg( +// avg($"a" cast DecimalType(10, 2)), sum_distinct($"a" cast DecimalType(10, 2))), +// Row(new java.math.BigDecimal(2), new java.math.BigDecimal(6)) :: Nil) + } + + ignoreGluten("SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate") { + withTempView("view") { + Seq( + ("mithunr", Float.NaN), + ("mithunr", Float.NaN), + ("mithunr", Float.NaN), + ("abellina", 1.0f), + ("abellina", 2.0f)).toDF("uid", "score").createOrReplaceTempView("view") + + val df = spark.sql("select uid, count(distinct score) from view group by 1 order by 1 asc") + checkAnswer(df, Row("abellina", 2) :: Row("mithunr", 1) :: Nil) + } + } + + testGluten("variance") { + checkAnswer( + testData2.agg(var_samp($"a"), var_pop($"a"), variance($"a")), + Row(0.8, 2.0 / 3.0, 0.8)) + checkAnswer(testData2.agg(var_samp("a"), var_pop("a"), variance("a")), Row(0.8, 2.0 / 3.0, 0.8)) + } + + testGluten("aggregation with filter") { + Seq( + ("mithunr", 12.3f, 5.0f, true, 9.4f), + ("mithunr", 15.5f, 4.0f, false, 19.9f), + ("mithunr", 19.8f, 3.0f, false, 35.6f), + ("abellina", 20.1f, 2.0f, true, 98.0f), + ("abellina", 20.1f, 1.0f, true, 0.5f), + ("abellina", 23.6f, 2.0f, true, 3.9f) + ) + .toDF("uid", "time", "score", "pass", "rate") + .createOrReplaceTempView("view") + var df = spark.sql("select count(score) filter (where pass) from view group by time") + checkAnswer(df, Row(1) :: Row(0) :: Row(0) :: Row(2) :: Row(1) :: Nil) + + df = spark.sql("select count(score) filter (where pass) from view") + checkAnswer(df, Row(4) :: Nil) + + df = spark.sql("select count(score) filter (where rate > 20) from view group by time") + checkAnswer(df, Row(0) :: Row(0) :: Row(1) :: Row(1) :: Row(0) :: Nil) + + df = spark.sql("select count(score) filter (where rate > 20) from view") + checkAnswer(df, Row(2) :: Nil) + } + + testGluten("extend with cast expression") { + checkAnswer( + decimalData.agg( + sum($"a".cast("double")), + avg($"b".cast("double")), + count_distinct($"a"), + count_distinct($"b")), + Row(12.0, 1.5, 3, 2)) + } + + // This test is applicable to velox backend. For CH backend, the replacement is disabled. + testGluten("use gluten hash agg to replace vanilla spark sort agg") { + + withSQLConf((GlutenConfig.COLUMNAR_FORCE_HASHAGG_ENABLED.key, "false")) { + Seq("A", "B", "C", "D").toDF("col1").createOrReplaceTempView("t1") + // SortAggregateExec is expected to be used for string type input. + val df = spark.sql("select max(col1) from t1") + checkAnswer(df, Row("D") :: Nil) + assert(find(df.queryExecution.executedPlan)(_.isInstanceOf[SortAggregateExec]).isDefined) + } + + withSQLConf((GlutenConfig.COLUMNAR_FORCE_HASHAGG_ENABLED.key, "true")) { + Seq("A", "B", "C", "D").toDF("col1").createOrReplaceTempView("t1") + val df = spark.sql("select max(col1) from t1") + checkAnswer(df, Row("D") :: Nil) + // Sort agg is expected to be replaced by gluten's hash agg. + assert( + find(df.queryExecution.executedPlan)( + _.isInstanceOf[HashAggregateExecBaseTransformer]).isDefined) + } + } + + testGluten("mixed supported and unsupported aggregate functions") { + withUserDefinedFunction(("udaf_sum", true)) { + spark.udf.register( + "udaf_sum", + udaf(new Aggregator[JLong, JLong, JLong] { + override def zero: JLong = 0 + override def reduce(b: JLong, a: JLong): JLong = a + b + override def merge(b1: JLong, b2: JLong): JLong = b1 + b2 + override def finish(reduction: JLong): JLong = reduction + override def bufferEncoder: Encoder[JLong] = Encoders.LONG + override def outputEncoder: Encoder[JLong] = Encoders.LONG + }) + ) + + val df = spark.sql("SELECT a, udaf_sum(b), max(b) FROM testData2 group by a") + checkAnswer(df, Row(1, 3, 2) :: Row(2, 3, 2) :: Row(3, 3, 2) :: Nil) + } + } + + // Ported from spark DataFrameAggregateSuite only with plan check changed. + private def assertNoExceptions(c: Column): Unit = { + for ( + (wholeStage, useObjectHashAgg) <- + Seq((true, true), (true, false), (false, true), (false, false)) + ) { + withSQLConf( + (SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, wholeStage.toString), + (SQLConf.USE_OBJECT_HASH_AGG.key, useObjectHashAgg.toString)) { + + val df = Seq(("1", 1), ("1", 2), ("2", 3), ("2", 4)).toDF("x", "y") + + // test case for HashAggregate + val hashAggDF = df.groupBy("x").agg(c, sum("y")) + hashAggDF.collect() + val hashAggPlan = hashAggDF.queryExecution.executedPlan + if (wholeStage) { + assert(find(hashAggPlan) { + case WholeStageCodegenExec(_: HashAggregateExec) => true + // If offloaded, spark whole stage codegen takes no effect and a gluten hash agg is + // expected to be used. + case _: HashAggregateExecBaseTransformer => true + case _ => false + }.isDefined) + } else { + assert( + stripAQEPlan(hashAggPlan).isInstanceOf[HashAggregateExec] || + stripAQEPlan(hashAggPlan).find { + case _: HashAggregateExecBaseTransformer => true + case _ => false + }.isDefined) + } + + // test case for ObjectHashAggregate and SortAggregate + val objHashAggOrSortAggDF = df.groupBy("x").agg(c, collect_list("y")) + objHashAggOrSortAggDF.collect() + assert(stripAQEPlan(objHashAggOrSortAggDF.queryExecution.executedPlan).find { + case _: HashAggregateExecBaseTransformer => true + case _ => false + }.isDefined) + } + } + } + + testGluten( + "SPARK-19471: AggregationIterator does not initialize the generated" + + " result projection before using it") { + Seq( + monotonically_increasing_id(), + spark_partition_id(), + rand(Random.nextLong()), + randn(Random.nextLong()) + ).foreach(assertNoExceptions) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameAsOfJoinSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameAsOfJoinSuite.scala new file mode 100644 index 000000000000..9367fab17f2b --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameAsOfJoinSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDataFrameAsOfJoinSuite extends DataFrameAsOfJoinSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameComplexTypeSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameComplexTypeSuite.scala new file mode 100644 index 000000000000..7464968cba51 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameComplexTypeSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDataFrameComplexTypeSuite extends DataFrameComplexTypeSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameFunctionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameFunctionsSuite.scala new file mode 100644 index 000000000000..49f6052b201f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameFunctionsSuite.scala @@ -0,0 +1,281 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.spark.SparkException +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{IntegerType, MapType, StringType, StructField, StructType} + +class GlutenDataFrameFunctionsSuite extends DataFrameFunctionsSuite with GlutenSQLTestsTrait { + import testImplicits._ + + testGluten("map_zip_with function - map of primitive types") { + val df = Seq( + (Map(8 -> 6L, 3 -> 5L, 6 -> 2L), Map[Integer, Integer]((6, 4), (8, 2), (3, 2))), + (Map(10 -> 6L, 8 -> 3L), Map[Integer, Integer]((8, 4), (4, null))), + (Map.empty[Int, Long], Map[Integer, Integer]((5, 1))), + (Map(5 -> 1L), null) + ).toDF("m1", "m2") + + GlutenQueryTestUtil.sameRows( + df.selectExpr("map_zip_with(m1, m2, (k, v1, v2) -> k == v1 + v2)").collect.toSeq, + Seq( + Row(Map(8 -> true, 3 -> false, 6 -> true)), + Row(Map(10 -> null, 8 -> false, 4 -> null)), + Row(Map(5 -> null)), + Row(null)), + false + ) + + GlutenQueryTestUtil.sameRows( + df.select(map_zip_with(df("m1"), df("m2"), (k, v1, v2) => k === v1 + v2)).collect.toSeq, + Seq( + Row(Map(8 -> true, 3 -> false, 6 -> true)), + Row(Map(10 -> null, 8 -> false, 4 -> null)), + Row(Map(5 -> null)), + Row(null)), + false + ) + } + + testGluten("map_concat function") { + val df1 = Seq( + (Map[Int, Int](1 -> 100, 2 -> 200), Map[Int, Int](3 -> 300, 4 -> 400)), + (Map[Int, Int](1 -> 100, 2 -> 200), Map[Int, Int](3 -> 300, 1 -> 400)), + (null, Map[Int, Int](3 -> 300, 4 -> 400)) + ).toDF("map1", "map2") + + val expected1a = Seq( + Row(Map(1 -> 100, 2 -> 200, 3 -> 300, 4 -> 400)), + Row(Map(1 -> 400, 2 -> 200, 3 -> 300)), + Row(null) + ) + + intercept[SparkException](df1.selectExpr("map_concat(map1, map2)").collect()) + intercept[SparkException](df1.select(map_concat($"map1", $"map2")).collect()) + withSQLConf(SQLConf.MAP_KEY_DEDUP_POLICY.key -> SQLConf.MapKeyDedupPolicy.LAST_WIN.toString) { + checkAnswer(df1.selectExpr("map_concat(map1, map2)"), expected1a) + checkAnswer(df1.select(map_concat($"map1", $"map2")), expected1a) + } + + val expected1b = Seq( + Row(Map(1 -> 100, 2 -> 200)), + Row(Map(1 -> 100, 2 -> 200)), + Row(null) + ) + + checkAnswer(df1.selectExpr("map_concat(map1)"), expected1b) + checkAnswer(df1.select(map_concat($"map1")), expected1b) + + val df2 = Seq( + ( + Map[Array[Int], Int](Array(1) -> 100, Array(2) -> 200), + Map[String, Int]("3" -> 300, "4" -> 400) + ) + ).toDF("map1", "map2") + + val expected2 = Seq(Row(Map())) + + checkAnswer(df2.selectExpr("map_concat()"), expected2) + checkAnswer(df2.select(map_concat()), expected2) + + val df3 = { + val schema = StructType( + StructField("map1", MapType(StringType, IntegerType, true), false) :: + StructField("map2", MapType(StringType, IntegerType, false), false) :: Nil + ) + val data = Seq( + Row(Map[String, Any]("a" -> 1, "b" -> null), Map[String, Any]("c" -> 3, "d" -> 4)), + Row(Map[String, Any]("a" -> 1, "b" -> 2), Map[String, Any]("c" -> 3, "d" -> 4)) + ) + spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + } + + val expected3 = Seq( + Row(Map[String, Any]("a" -> 1, "b" -> null, "c" -> 3, "d" -> 4)), + Row(Map[String, Any]("a" -> 1, "b" -> 2, "c" -> 3, "d" -> 4)) + ) + + checkAnswer(df3.selectExpr("map_concat(map1, map2)"), expected3) + checkAnswer(df3.select(map_concat($"map1", $"map2")), expected3) + + checkError( + exception = intercept[AnalysisException] { + df2.selectExpr("map_concat(map1, map2)").collect() + }, + condition = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + sqlState = None, + parameters = Map( + "sqlExpr" -> "\"map_concat(map1, map2)\"", + "dataType" -> "(\"MAP, INT>\" or \"MAP\")", + "functionName" -> "`map_concat`"), + context = ExpectedContext(fragment = "map_concat(map1, map2)", start = 0, stop = 21) + ) + + checkError( + exception = intercept[AnalysisException] { + df2.select(map_concat($"map1", $"map2")).collect() + }, + condition = "DATATYPE_MISMATCH.DATA_DIFF_TYPES", + sqlState = None, + parameters = Map( + "sqlExpr" -> "\"map_concat(map1, map2)\"", + "dataType" -> "(\"MAP, INT>\" or \"MAP\")", + "functionName" -> "`map_concat`"), + context = + ExpectedContext(fragment = "map_concat", callSitePattern = getCurrentClassCallSitePattern) + ) + + checkError( + exception = intercept[AnalysisException] { + df2.selectExpr("map_concat(map1, 12)").collect() + }, + condition = "DATATYPE_MISMATCH.MAP_CONCAT_DIFF_TYPES", + sqlState = None, + parameters = Map( + "sqlExpr" -> "\"map_concat(map1, 12)\"", + "dataType" -> "[\"MAP, INT>\", \"INT\"]", + "functionName" -> "`map_concat`"), + context = ExpectedContext(fragment = "map_concat(map1, 12)", start = 0, stop = 19) + ) + + checkError( + exception = intercept[AnalysisException] { + df2.select(map_concat($"map1", lit(12))).collect() + }, + condition = "DATATYPE_MISMATCH.MAP_CONCAT_DIFF_TYPES", + sqlState = None, + parameters = Map( + "sqlExpr" -> "\"map_concat(map1, 12)\"", + "dataType" -> "[\"MAP, INT>\", \"INT\"]", + "functionName" -> "`map_concat`"), + context = + ExpectedContext(fragment = "map_concat", callSitePattern = getCurrentClassCallSitePattern) + ) + } + + testGluten("transform keys function - primitive data types") { + val dfExample1 = Seq( + Map[Int, Int](1 -> 1, 9 -> 9, 8 -> 8, 7 -> 7) + ).toDF("i") + + val dfExample2 = Seq( + Map[Int, Double](1 -> 1.0, 2 -> 1.40, 3 -> 1.70) + ).toDF("j") + + val dfExample3 = Seq( + Map[Int, Boolean](25 -> true, 26 -> false) + ).toDF("x") + + val dfExample4 = Seq( + Map[Array[Int], Boolean](Array(1, 2) -> false) + ).toDF("y") + + def testMapOfPrimitiveTypesCombination(): Unit = { + checkAnswer( + dfExample1.selectExpr("transform_keys(i, (k, v) -> k + v)"), + Seq(Row(Map(2 -> 1, 18 -> 9, 16 -> 8, 14 -> 7)))) + + checkAnswer( + dfExample1.select(transform_keys(col("i"), (k, v) => k + v)), + Seq(Row(Map(2 -> 1, 18 -> 9, 16 -> 8, 14 -> 7)))) + + checkAnswer( + dfExample2.selectExpr( + "transform_keys(j, " + + "(k, v) -> map_from_arrays(ARRAY(1, 2, 3), ARRAY('one', 'two', 'three'))[k])"), + Seq(Row(Map("one" -> 1.0, "two" -> 1.4, "three" -> 1.7))) + ) + + checkAnswer( + dfExample2.select( + transform_keys( + col("j"), + (k, v) => + element_at( + map_from_arrays( + array(lit(1), lit(2), lit(3)), + array(lit("one"), lit("two"), lit("three")) + ), + k + ) + ) + ), + Seq(Row(Map("one" -> 1.0, "two" -> 1.4, "three" -> 1.7))) + ) + + checkAnswer( + dfExample2.selectExpr("transform_keys(j, (k, v) -> CAST(v * 2 AS BIGINT) + k)"), + Seq(Row(Map(3 -> 1.0, 4 -> 1.4, 6 -> 1.7)))) + + checkAnswer( + dfExample2.select(transform_keys(col("j"), (k, v) => (v * 2).cast("bigint") + k)), + Seq(Row(Map(3 -> 1.0, 4 -> 1.4, 6 -> 1.7)))) + + checkAnswer( + dfExample2.selectExpr("transform_keys(j, (k, v) -> k + v)"), + Seq(Row(Map(2.0 -> 1.0, 3.4 -> 1.4, 4.7 -> 1.7)))) + + checkAnswer( + dfExample2.select(transform_keys(col("j"), (k, v) => k + v)), + Seq(Row(Map(2.0 -> 1.0, 3.4 -> 1.4, 4.7 -> 1.7)))) + + intercept[SparkException] { + dfExample3.selectExpr("transform_keys(x, (k, v) -> k % 2 = 0 OR v)").collect() + } + intercept[SparkException] { + dfExample3.select(transform_keys(col("x"), (k, v) => k % 2 === 0 || v)).collect() + } + withSQLConf(SQLConf.MAP_KEY_DEDUP_POLICY.key -> SQLConf.MapKeyDedupPolicy.LAST_WIN.toString) { + checkAnswer( + dfExample3.selectExpr("transform_keys(x, (k, v) -> k % 2 = 0 OR v)"), + Seq(Row(Map(true -> true, true -> false)))) + + checkAnswer( + dfExample3.select(transform_keys(col("x"), (k, v) => k % 2 === 0 || v)), + Seq(Row(Map(true -> true, true -> false)))) + } + + checkAnswer( + dfExample3.selectExpr("transform_keys(x, (k, v) -> if(v, 2 * k, 3 * k))"), + Seq(Row(Map(50 -> true, 78 -> false)))) + + checkAnswer( + dfExample3.select(transform_keys(col("x"), (k, v) => when(v, k * 2).otherwise(k * 3))), + Seq(Row(Map(50 -> true, 78 -> false)))) + + checkAnswer( + dfExample4.selectExpr("transform_keys(y, (k, v) -> array_contains(k, 3) AND v)"), + Seq(Row(Map(false -> false)))) + + checkAnswer( + dfExample4.select(transform_keys(col("y"), (k, v) => array_contains(k, lit(3)) && v)), + Seq(Row(Map(false -> false)))) + } + + // Test with local relation, the Project will be evaluated without codegen + testMapOfPrimitiveTypesCombination() + dfExample1.cache() + dfExample2.cache() + dfExample3.cache() + dfExample4.cache() + // Test with cached relation, the Project will be evaluated with codegen + testMapOfPrimitiveTypesCombination() + } + +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameHintSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameHintSuite.scala new file mode 100644 index 000000000000..663a6111b0d0 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameHintSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDataFrameHintSuite extends DataFrameHintSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameImplicitsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameImplicitsSuite.scala new file mode 100644 index 000000000000..2a6e367bc08a --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameImplicitsSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDataFrameImplicitsSuite extends DataFrameImplicitsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameJoinSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameJoinSuite.scala new file mode 100644 index 000000000000..6581d7f2d88d --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameJoinSuite.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDataFrameJoinSuite extends DataFrameJoinSuite with GlutenSQLTestsTrait { + + override def testNameBlackList: Seq[String] = Seq( + "Supports multi-part names for broadcast hint resolution" + ) +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameNaFunctionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameNaFunctionsSuite.scala new file mode 100644 index 000000000000..424087c8de89 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameNaFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDataFrameNaFunctionsSuite extends DataFrameNaFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFramePivotSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFramePivotSuite.scala new file mode 100644 index 000000000000..fe53c68401d1 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFramePivotSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDataFramePivotSuite extends DataFramePivotSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameRangeSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameRangeSuite.scala new file mode 100644 index 000000000000..e8a424de5be1 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameRangeSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDataFrameRangeSuite extends DataFrameRangeSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameSelfJoinSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameSelfJoinSuite.scala new file mode 100644 index 000000000000..61cc4bc4c080 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameSelfJoinSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDataFrameSelfJoinSuite extends DataFrameSelfJoinSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameSessionWindowingSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameSessionWindowingSuite.scala new file mode 100644 index 000000000000..d76d8b21cdcf --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameSessionWindowingSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDataFrameSessionWindowingSuite + extends DataFrameSessionWindowingSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameSetOperationsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameSetOperationsSuite.scala new file mode 100644 index 000000000000..fe7958b67773 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameSetOperationsSuite.scala @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.spark.SparkConf + +class GlutenDataFrameSetOperationsSuite + extends DataFrameSetOperationsSuite + with GlutenSQLTestsTrait { + override def sparkConf: SparkConf = + super.sparkConf + .set("spark.gluten.sql.columnar.backend.ch.enable.coalesce.project.union", "false") + .set("spark.gluten.sql.columnar.backend.ch.enable.coalesce.aggregation.union", "false") + +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameStatSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameStatSuite.scala new file mode 100644 index 000000000000..bab8e9b83cb2 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameStatSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDataFrameStatSuite extends DataFrameStatSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameSuite.scala new file mode 100644 index 000000000000..069f308fdca0 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameSuite.scala @@ -0,0 +1,434 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.gluten.execution.ProjectExecTransformer + +import org.apache.spark.SparkException +import org.apache.spark.sql.execution.ColumnarShuffleExchangeExec +import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.exchange.{ReusedExchangeExec, ShuffleExchangeExec} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestData.TestData2 +import org.apache.spark.sql.types.StringType + +import java.io.ByteArrayOutputStream +import java.nio.charset.StandardCharsets + +import scala.util.Random + +class GlutenDataFrameSuite extends DataFrameSuite with GlutenSQLTestsTrait { + + testGluten("repartitionByRange") { + val partitionNum = 10 + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> partitionNum.toString) { + import testImplicits._ + val data1d = Random.shuffle(0.to(partitionNum - 1)) + val data2d = data1d.map(i => (i, data1d.size - i)) + + checkAnswer( + data1d + .toDF("val") + .repartitionByRange(data1d.size, $"val".asc) + .select(spark_partition_id().as("id"), $"val"), + data1d.map(i => Row(i, i))) + + checkAnswer( + data1d + .toDF("val") + .repartitionByRange(data1d.size, $"val".desc) + .select(spark_partition_id().as("id"), $"val"), + data1d.map(i => Row(i, data1d.size - 1 - i))) + + checkAnswer( + data1d + .toDF("val") + .repartitionByRange(data1d.size, lit(42)) + .select(spark_partition_id().as("id"), $"val"), + data1d.map(i => Row(0, i))) + + checkAnswer( + data1d + .toDF("val") + .repartitionByRange(data1d.size, lit(null), $"val".asc, rand()) + .select(spark_partition_id().as("id"), $"val"), + data1d.map(i => Row(i, i))) + + // .repartitionByRange() assumes .asc by default if no explicit sort order is specified + checkAnswer( + data2d + .toDF("a", "b") + .repartitionByRange(data2d.size, $"a".desc, $"b") + .select(spark_partition_id().as("id"), $"a", $"b"), + data2d + .toDF("a", "b") + .repartitionByRange(data2d.size, $"a".desc, $"b".asc) + .select(spark_partition_id().as("id"), $"a", $"b") + ) + + // at least one partition-by expression must be specified + intercept[IllegalArgumentException] { + data1d.toDF("val").repartitionByRange(data1d.size) + } + intercept[IllegalArgumentException] { + data1d.toDF("val").repartitionByRange(data1d.size, Seq.empty: _*) + } + } + } + + testGluten("distributeBy and localSort") { + import testImplicits._ + val data = spark.sparkContext.parallelize((1 to 100).map(i => TestData2(i % 10, i))).toDF() + + /** partitionNum = 1 */ + var partitionNum = 1 + val original = testData.repartition(partitionNum) + assert(original.rdd.partitions.length == partitionNum) + + // Distribute into one partition and order by. This partition should contain all the values. + val df6 = data.repartition(partitionNum, $"a").sortWithinPartitions("b") + // Walk each partition and verify that it is sorted ascending and not globally sorted. + df6.rdd.foreachPartition { + p => + var previousValue: Int = -1 + var allSequential: Boolean = true + p.foreach { + r => + val v: Int = r.getInt(1) + if (previousValue != -1) { + if (previousValue > v) throw new SparkException("Partition is not ordered.") + if (v - 1 != previousValue) allSequential = false + } + previousValue = v + } + if (!allSequential) { + throw new SparkException("Partition should contain all sequential values") + } + } + + /** partitionNum = 5 */ + partitionNum = 5 + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> partitionNum.toString) { + val df = original.repartition(partitionNum, $"key") + assert(df.rdd.partitions.length == partitionNum) + checkAnswer(original.select(), df.select()) + + // Distribute and order by. + val df4 = data.repartition(partitionNum, $"a").sortWithinPartitions($"b".desc) + // Walk each partition and verify that it is sorted descending and does not contain all + // the values. + df4.rdd.foreachPartition { + p => + // Skip empty partition + if (p.hasNext) { + var previousValue: Int = -1 + var allSequential: Boolean = true + p.foreach { + r => + val v: Int = r.getInt(1) + if (previousValue != -1) { + if (previousValue < v) throw new SparkException("Partition is not ordered.") + if (v + 1 != previousValue) allSequential = false + } + previousValue = v + } + if (allSequential) throw new SparkException("Partition should not be globally ordered") + } + } + } + + /** partitionNum = 10 */ + partitionNum = 10 + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> partitionNum.toString) { + val df2 = original.repartition(partitionNum, $"key") + assert(df2.rdd.partitions.length == partitionNum) + checkAnswer(original.select(), df2.select()) + } + + // Group by the column we are distributed by. This should generate a plan with no exchange + // between the aggregates + val df3 = testData.repartition($"key").groupBy("key").count() + verifyNonExchangingAgg(df3) + verifyNonExchangingAgg( + testData + .repartition($"key", $"value") + .groupBy("key", "value") + .count()) + + // Grouping by just the first distributeBy expr, need to exchange. + verifyExchangingAgg( + testData + .repartition($"key", $"value") + .groupBy("key") + .count()) + + /** partitionNum = 2 */ + partitionNum = 2 + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> partitionNum.toString) { + // Distribute and order by with multiple order bys + val df5 = data.repartition(partitionNum, $"a").sortWithinPartitions($"b".asc, $"a".asc) + // Walk each partition and verify that it is sorted ascending + df5.rdd.foreachPartition { + p => + var previousValue: Int = -1 + var allSequential: Boolean = true + p.foreach { + r => + val v: Int = r.getInt(1) + if (previousValue != -1) { + if (previousValue > v) throw new SparkException("Partition is not ordered.") + if (v - 1 != previousValue) allSequential = false + } + previousValue = v + } + if (allSequential) throw new SparkException("Partition should not be all sequential") + } + } + } + + testGluten("reuse exchange") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "2") { + val df = spark.range(100).toDF() + val join = df.join(df, "id") + val plan = join.queryExecution.executedPlan + checkAnswer(join, df) + assert(collect(join.queryExecution.executedPlan) { + // replace ShuffleExchangeExec + case e: ColumnarShuffleExchangeExec => true + }.size === 1) + assert(collect(join.queryExecution.executedPlan) { + case e: ReusedExchangeExec => true + }.size === 1) + val broadcasted = broadcast(join) + val join2 = join.join(broadcasted, "id").join(broadcasted, "id") + checkAnswer(join2, df) + assert(collect(join2.queryExecution.executedPlan) { + // replace ShuffleExchangeExec + case e: ColumnarShuffleExchangeExec => true + }.size == 1) + assert(collect(join2.queryExecution.executedPlan) { + case e: ReusedExchangeExec => true + }.size == 4) + } + } + + // TODO: fix in spark-4.0 + /** Failed to check WholeStageCodegenExec, so we rewrite the UT. */ + // testGluten("SPARK-22520: support code generation for large CaseWhen") { + // import org.apache.spark.sql.catalyst.dsl.expressions.StringToAttributeConversionHelper + // val N = 30 + // var expr1 = when(equalizer($"id", lit(0)), 0) + // var expr2 = when(equalizer($"id", lit(0)), 10) + // (1 to N).foreach { + // i => + // expr1 = expr1.when(equalizer($"id", lit(i)), -i) + // expr2 = expr2.when(equalizer($"id", lit(i + 10)), i) + // } + // val df = spark.range(1).select(expr1, expr2.otherwise(0)) + // checkAnswer(df, Row(0, 10) :: Nil) + // // We check WholeStageTransformer instead of WholeStageCodegenExec + // assert(df.queryExecution.executedPlan.find(_.isInstanceOf[WholeStageTransformer]).isDefined) + // } + import testImplicits._ + + private lazy val person2: DataFrame = Seq( + ("Bob", 16, 176), + ("Alice", 32, 164), + ("David", 60, 192), + ("Amy", 24, 180)).toDF("name", "age", "height") + + testGluten("describe") { + val describeResult = Seq( + Row("count", "4", "4", "4"), + Row("mean", null, "33.0", "178.0"), + Row("stddev", null, "19.148542155126762", "11.547005383792516"), + Row("min", "Alice", "16", "164"), + Row("max", "David", "60", "192") + ) + + val emptyDescribeResult = Seq( + Row("count", "0", "0", "0"), + Row("mean", null, null, null), + Row("stddev", null, null, null), + Row("min", null, null, null), + Row("max", null, null, null)) + + val aggResult = Seq( + Row("4", "33.0", "19.148542155126762", "16", "60") + ) + + def getSchemaAsSeq(df: DataFrame): Seq[String] = df.schema.map(_.name) + + Seq("true", "false").foreach { + ansiEnabled => + withSQLConf(SQLConf.ANSI_ENABLED.key -> ansiEnabled) { + val describeAllCols = person2.describe() + assert(getSchemaAsSeq(describeAllCols) === Seq("summary", "name", "age", "height")) + checkAnswer(describeAllCols, describeResult) + // All aggregate value should have been cast to string + describeAllCols.collect().foreach { + row => + row.toSeq.foreach { + value => + if (value != null) { + assert( + value.isInstanceOf[String], + "expected string but found " + value.getClass) + } + } + } + + val describeOneCol = person2.describe("age") + assert(getSchemaAsSeq(describeOneCol) === Seq("summary", "age")) + val aggOneCol = person2.agg( + count("age").cast(StringType), + avg("age").cast(StringType), + stddev_samp("age").cast(StringType), + min("age").cast(StringType), + max("age").cast(StringType)) + checkAnswer(aggOneCol, aggResult) + + val describeNoCol = person2.select().describe() + assert(getSchemaAsSeq(describeNoCol) === Seq("summary")) + checkAnswer(describeNoCol, describeResult.map { case Row(s, _, _, _) => Row(s) }) + + val emptyDescription = person2.limit(0).describe() + assert(getSchemaAsSeq(emptyDescription) === Seq("summary", "name", "age", "height")) + checkAnswer(emptyDescription, emptyDescribeResult) + } + } + } + + testGluten("Allow leading/trailing whitespace in string before casting") { + withSQLConf("spark.gluten.velox.castFromVarcharAddTrimNode" -> "true") { + def checkResult(df: DataFrame, expectedResult: Seq[Row]): Unit = { + checkAnswer(df, expectedResult) + assert( + find(df.queryExecution.executedPlan)(_.isInstanceOf[ProjectExecTransformer]).isDefined) + } + + // scalastyle:off nonascii + Seq( + " 123", + "123 ", + " 123 ", + "\u2000123\n\n\n", + "123\r\r\r", + "123\f\f\f", + "123\u000C", + "123\u0000") + .toDF("col1") + .createOrReplaceTempView("t1") + // scalastyle:on nonascii + val expectedIntResult = Row(123) :: Row(123) :: + Row(123) :: Row(123) :: Row(123) :: Row(123) :: Row(123) :: Row(123) :: Nil + var df = spark.sql("select cast(col1 as int) from t1") + checkResult(df, expectedIntResult) + df = spark.sql("select cast(col1 as long) from t1") + checkResult(df, expectedIntResult) + + Seq(" 123.5", "123.5 ", " 123.5 ", "123.5\n\n\n", "123.5\r\r\r", "123.5\f\f\f", "123.5\u000C") + .toDF("col1") + .createOrReplaceTempView("t1") + val expectedFloatResult = Row(123.5) :: Row(123.5) :: + Row(123.5) :: Row(123.5) :: Row(123.5) :: Row(123.5) :: Row(123.5) :: Nil + df = spark.sql("select cast(col1 as float) from t1") + checkResult(df, expectedFloatResult) + df = spark.sql("select cast(col1 as double) from t1") + checkResult(df, expectedFloatResult) + + // scalastyle:off nonascii + val rawData = + Seq(" abc", "abc ", " abc ", "\u2000abc\n\n\n", "abc\r\r\r", "abc\f\f\f", "abc\u000C") + // scalastyle:on nonascii + rawData.toDF("col1").createOrReplaceTempView("t1") + val expectedBinaryResult = rawData.map(d => Row(d.getBytes(StandardCharsets.UTF_8))).seq + df = spark.sql("select cast(col1 as binary) from t1") + checkResult(df, expectedBinaryResult) + } + } + + testGluten("SPARK-27439: Explain result should match collected result after view change") { + withTempView("test", "test2", "tmp") { + spark.range(10).createOrReplaceTempView("test") + spark.range(5).createOrReplaceTempView("test2") + spark.sql("select * from test").createOrReplaceTempView("tmp") + val df = spark.sql("select * from tmp") + spark.sql("select * from test2").createOrReplaceTempView("tmp") + + val captured = new ByteArrayOutputStream() + Console.withOut(captured) { + df.explain(extended = true) + } + checkAnswer(df, spark.range(10).toDF) + val output = captured.toString + assert(output.contains("""== Parsed Logical Plan == + |'Project [*] + |+- 'UnresolvedRelation [tmp]""".stripMargin)) + assert(output.contains("""== Physical Plan == + |*(1) ColumnarToRow + |+- ColumnarRange 0, 10, 1, 2, 10""".stripMargin)) + } + } + + // TODO: fix in spark-4.0 + // private def withExpr(newExpr: Expression): Column = new Column(newExpr) + + // def equalizer(expr: Expression, other: Any): Column = withExpr { + // val right = lit(other).expr + // if (expr == right) { + // logWarning( + // s"Constructing trivially true equals predicate, '$expr = $right'. " + + // "Perhaps you need to use aliases.") + // } + // EqualTo(expr, right) + // } + + private def verifyNonExchangingAgg(df: DataFrame): Unit = { + var atFirstAgg: Boolean = false + df.queryExecution.executedPlan.foreach { + case agg: HashAggregateExec => + atFirstAgg = !atFirstAgg + case _ => + if (atFirstAgg) { + fail("Should not have operators between the two aggregations") + } + } + } + + private def verifyExchangingAgg(df: DataFrame): Unit = { + var atFirstAgg: Boolean = false + df.queryExecution.executedPlan.foreach { + case _: HashAggregateExec => + if (atFirstAgg) { + fail("Should not have back to back Aggregates") + } + atFirstAgg = true + case _: ShuffleExchangeExec => atFirstAgg = false + case _ => + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameTimeWindowingSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameTimeWindowingSuite.scala new file mode 100644 index 000000000000..f2833a357cd2 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameTimeWindowingSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDataFrameTimeWindowingSuite + extends DataFrameTimeWindowingSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameToSchemaSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameToSchemaSuite.scala new file mode 100644 index 000000000000..d578b92c4c8a --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameToSchemaSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDataFrameToSchemaSuite extends DataFrameToSchemaSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameTungstenSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameTungstenSuite.scala new file mode 100644 index 000000000000..0e555c8eac6f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameTungstenSuite.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.spark.sql.types._ + +class GlutenDataFrameTungstenSuite extends DataFrameTungstenSuite with GlutenSQLTestsTrait { + + testGluten("Map type with struct type as key") { + val kv = Map(Row(1, 2L) -> Seq("v")) + val data = sparkContext.parallelize(Seq(Row(1, kv))) + val schema = new StructType() + .add("a", IntegerType) + .add( + "b", + MapType(new StructType().add("k1", IntegerType).add("k2", LongType), ArrayType(StringType))) + val df = spark.createDataFrame(data, schema) + assert(df.select("b").first() === Row(kv)) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameWindowFramesSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameWindowFramesSuite.scala new file mode 100644 index 000000000000..3ba990d2eea6 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameWindowFramesSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDataFrameWindowFramesSuite + extends DataFrameWindowFramesSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameWindowFunctionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameWindowFunctionsSuite.scala new file mode 100644 index 000000000000..978685db87a0 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameWindowFunctionsSuite.scala @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.gluten.execution.WindowExecTransformer + +import org.apache.spark.SparkConf +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning +import org.apache.spark.sql.execution.ColumnarShuffleExchangeExec +import org.apache.spark.sql.execution.exchange.ENSURE_REQUIREMENTS +import org.apache.spark.sql.expressions.Window +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf + +class GlutenDataFrameWindowFunctionsSuite + extends DataFrameWindowFunctionsSuite + with GlutenSQLTestsTrait { + + import testImplicits._ + + override def sparkConf: SparkConf = { + super.sparkConf + // avoid single partition + .set("spark.sql.shuffle.partitions", "2") + } + + testGluten("covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window") { + withSQLConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "true") { + val df = Seq( + ("a", "p1", 10.0, 20.0), + ("b", "p1", 20.0, 10.0), + ("c", "p2", 20.0, 20.0), + ("d", "p2", 20.0, 20.0), + ("e", "p3", 0.0, 0.0), + ("f", "p3", 6.0, 12.0), + ("g", "p3", 6.0, 12.0), + ("h", "p3", 8.0, 16.0) + ).toDF("key", "partitionId", "value1", "value2") + checkAnswer( + df.select( + $"key", + covar_samp("value1", "value2").over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_samp("value1").over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + variance("value1").over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_samp("value1").over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev("value1").over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)) + ), + Seq( + Row("a", -50.0, 50.0, 50.0, 7.0710678118654755, 7.0710678118654755), + Row("b", -50.0, 50.0, 50.0, 7.0710678118654755, 7.0710678118654755), + Row("c", 0.0, 0.0, 0.0, 0.0, 0.0), + Row("d", 0.0, 0.0, 0.0, 0.0, 0.0), + Row("e", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("f", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("g", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("h", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544) + ) + ) + } + } + + testGluten("corr, covar_pop, stddev_pop functions in specific window") { + withSQLConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "true") { + val df = Seq( + ("a", "p1", 10.0, 20.0), + ("b", "p1", 20.0, 10.0), + ("c", "p2", 20.0, 20.0), + ("d", "p2", 20.0, 20.0), + ("e", "p3", 0.0, 0.0), + ("f", "p3", 6.0, 12.0), + ("g", "p3", 6.0, 12.0), + ("h", "p3", 8.0, 16.0) + ).toDF("key", "partitionId", "value1", "value2") + checkAnswer( + df.select( + $"key", + corr("value1", "value2").over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + covar_pop("value1", "value2") + .over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_pop("value1") + .over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_pop("value1") + .over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_pop("value2") + .over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_pop("value2") + .over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)) + ), + + // As stddev_pop(expr) = sqrt(var_pop(expr)) + // the "stddev_pop" column can be calculated from the "var_pop" column. + // + // As corr(expr1, expr2) = covar_pop(expr1, expr2) / (stddev_pop(expr1) * stddev_pop(expr2)) + // the "corr" column can be calculated from the "covar_pop" and the two "stddev_pop" columns + Seq( + Row("a", -1.0, -25.0, 25.0, 5.0, 25.0, 5.0), + Row("b", -1.0, -25.0, 25.0, 5.0, 25.0, 5.0), + Row("c", null, 0.0, 0.0, 0.0, 0.0, 0.0), + Row("d", null, 0.0, 0.0, 0.0, 0.0, 0.0), + Row("e", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("f", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("g", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("h", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0) + ) + ) + } + } + + testGluten( + "SPARK-38237: require all cluster keys for child required distribution for window query") { + def partitionExpressionsColumns(expressions: Seq[Expression]): Seq[String] = { + expressions.flatMap { case ref: AttributeReference => Some(ref.name) } + } + + def isShuffleExecByRequirement( + plan: ColumnarShuffleExchangeExec, + desiredClusterColumns: Seq[String]): Boolean = plan match { + case ColumnarShuffleExchangeExec(op: HashPartitioning, _, ENSURE_REQUIREMENTS, _, _) => + partitionExpressionsColumns(op.expressions) === desiredClusterColumns + case _ => false + } + + val df = Seq(("a", 1, 1), ("a", 2, 2), ("b", 1, 3), ("b", 1, 4)).toDF("key1", "key2", "value") + val windowSpec = Window.partitionBy("key1", "key2").orderBy("value") + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_DISTRIBUTION.key -> "true") { + + val windowed = df + // repartition by subset of window partitionBy keys which satisfies ClusteredDistribution + .repartition($"key1") + .select(lead($"key1", 1).over(windowSpec), lead($"value", 1).over(windowSpec)) + + checkAnswer(windowed, Seq(Row("b", 4), Row(null, null), Row(null, null), Row(null, null))) + + val shuffleByRequirement = windowed.queryExecution.executedPlan.exists { + case w: WindowExecTransformer => + w.child.exists { + case s: ColumnarShuffleExchangeExec => + isShuffleExecByRequirement(s, Seq("key1", "key2")) + case _ => false + } + case _ => false + } + + assert(shuffleByRequirement, "Can't find desired shuffle node from the query plan") + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameWriterV2Suite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameWriterV2Suite.scala new file mode 100644 index 000000000000..ddae3139d06b --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDataFrameWriterV2Suite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDataFrameWriterV2Suite extends DataFrameWriterV2Suite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetAggregatorSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetAggregatorSuite.scala new file mode 100644 index 000000000000..8a9a6b5756e9 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetAggregatorSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDatasetAggregatorSuite extends DatasetAggregatorSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetCacheSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetCacheSuite.scala new file mode 100644 index 000000000000..848560192722 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetCacheSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDatasetCacheSuite extends DatasetCacheSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetOptimizationSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetOptimizationSuite.scala new file mode 100644 index 000000000000..a9d1bd29cead --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetOptimizationSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDatasetOptimizationSuite extends DatasetOptimizationSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetPrimitiveSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetPrimitiveSuite.scala new file mode 100644 index 000000000000..c7463dcef75f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetPrimitiveSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDatasetPrimitiveSuite extends DatasetPrimitiveSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetSerializerRegistratorSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetSerializerRegistratorSuite.scala new file mode 100644 index 000000000000..6749227ed79d --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetSerializerRegistratorSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDatasetSerializerRegistratorSuite + extends DatasetSerializerRegistratorSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetSuite.scala new file mode 100644 index 000000000000..a8e73cee5a81 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.spark.sql.execution.ColumnarShuffleExchangeExec + +class GlutenDatasetSuite extends DatasetSuite with GlutenSQLTestsTrait { + import testImplicits._ + + testGluten("dropDuplicates: columns with same column name") { + val ds1 = Seq(("a", 1), ("a", 2), ("b", 1), ("a", 1)).toDS() + val ds2 = Seq(("a", 1), ("a", 2), ("b", 1), ("a", 1)).toDS() + // The dataset joined has two columns of the same name "_2". + val joined = ds1.join(ds2, "_1").select(ds1("_2").as[Int], ds2("_2").as[Int]) + // Using the checkDatasetUnorderly method to sort the result in Gluten. + checkDatasetUnorderly(joined.dropDuplicates(), (1, 2), (1, 1), (2, 1), (2, 2)) + } + + testGluten("groupBy.as") { + val df1 = Seq(DoubleData(1, "one"), DoubleData(2, "two"), DoubleData(3, "three")) + .toDS() + .repartition($"id") + .sortWithinPartitions("id") + val df2 = Seq(DoubleData(5, "one"), DoubleData(1, "two"), DoubleData(3, "three")) + .toDS() + .repartition($"id") + .sortWithinPartitions("id") + + val df3 = df1 + .groupBy("id") + .as[Int, DoubleData] + .cogroup(df2.groupBy("id").as[Int, DoubleData]) { + case (key, data1, data2) => + if (key == 1) { + Iterator(DoubleData(key, (data1 ++ data2).foldLeft("")((cur, next) => cur + next.val1))) + } else Iterator.empty + } + checkDataset(df3, DoubleData(1, "onetwo")) + + // Assert that no extra shuffle introduced by cogroup. + val exchanges = collect(df3.queryExecution.executedPlan) { + case h: ColumnarShuffleExchangeExec => h + } + // Assert the number of ColumnarShuffleExchangeExec + // instead of ShuffleExchangeExec in Gluten. + assert(exchanges.size == 2) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetUnpivotSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetUnpivotSuite.scala new file mode 100644 index 000000000000..e3ba780530fd --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDatasetUnpivotSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDatasetUnpivotSuite extends DatasetUnpivotSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDateFunctionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDateFunctionsSuite.scala new file mode 100644 index 000000000000..082f06641b5a --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDateFunctionsSuite.scala @@ -0,0 +1,300 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf + +import java.sql.{Date, Timestamp} +import java.time.{LocalDateTime, ZoneId} +import java.util.concurrent.TimeUnit + +class GlutenDateFunctionsSuite extends DateFunctionsSuite with GlutenSQLTestsTrait { + import testImplicits._ + + private def secs(millis: Long): Long = TimeUnit.MILLISECONDS.toSeconds(millis) + + testGluten("unix_timestamp") { + Seq("corrected", "legacy").foreach { + legacyParserPolicy => + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> ZoneId.systemDefault().toString) { + val date1 = Date.valueOf("2015-07-24") + val date2 = Date.valueOf("2015-07-25") + val ts1 = Timestamp.valueOf("2015-07-24 10:00:00.3") + val ts2 = Timestamp.valueOf("2015-07-25 02:02:02.2") + val ntzTs1 = LocalDateTime.parse("2015-07-24T10:00:00.3") + val ntzTs2 = LocalDateTime.parse("2015-07-25T02:02:02.2") + val s1 = "2015/07/24 10:00:00.5" + val s2 = "2015/07/25 02:02:02.6" + val ss1 = "2015-07-24 10:00:00" + val ss2 = "2015-07-25 02:02:02" + val fmt = "yyyy/MM/dd HH:mm:ss.S" + val df = Seq((date1, ts1, ntzTs1, s1, ss1), (date2, ts2, ntzTs2, s2, ss2)).toDF( + "d", + "ts", + "ntzTs", + "s", + "ss") + checkAnswer( + df.select(unix_timestamp(col("ts"))), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.select(unix_timestamp(col("ss"))), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.select(unix_timestamp(col("ntzTs"))), + Seq( + Row(secs(DateTimeUtils.microsToMillis(DateTimeUtils.localDateTimeToMicros(ntzTs1)))), + Row(secs(DateTimeUtils.microsToMillis(DateTimeUtils.localDateTimeToMicros(ntzTs2)))) + ) + ) + checkAnswer( + df.select(unix_timestamp(col("d"), fmt)), + Seq(Row(secs(date1.getTime)), Row(secs(date2.getTime)))) + checkAnswer( + df.select(unix_timestamp(col("s"), fmt)), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr("unix_timestamp(ts)"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr("unix_timestamp(ss)"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr("unix_timestamp(ntzTs)"), + Seq( + Row(secs(DateTimeUtils.microsToMillis(DateTimeUtils.localDateTimeToMicros(ntzTs1)))), + Row(secs(DateTimeUtils.microsToMillis(DateTimeUtils.localDateTimeToMicros(ntzTs2)))) + ) + ) + checkAnswer( + df.selectExpr(s"unix_timestamp(d, '$fmt')"), + Seq(Row(secs(date1.getTime)), Row(secs(date2.getTime)))) + checkAnswer( + df.selectExpr(s"unix_timestamp(s, '$fmt')"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + + val x1 = "2015-07-24 10:00:00" + val x2 = "2015-25-07 02:02:02" + val x3 = "2015-07-24 25:02:02" + val x4 = "2015-24-07 26:02:02" + val ts3 = Timestamp.valueOf("2015-07-24 02:25:02") + val ts4 = Timestamp.valueOf("2015-07-24 00:10:00") + + val df1 = Seq(x1, x2, x3, x4).toDF("x") + checkAnswer( + df1.select(unix_timestamp(col("x"))), + Seq(Row(secs(ts1.getTime)), Row(null), Row(null), Row(null))) + checkAnswer( + df1.selectExpr("unix_timestamp(x)"), + Seq(Row(secs(ts1.getTime)), Row(null), Row(null), Row(null))) + checkAnswer( + df1.select(unix_timestamp(col("x"), "yyyy-dd-MM HH:mm:ss")), + Seq(Row(null), Row(secs(ts2.getTime)), Row(null), Row(null))) + checkAnswer( + df1.selectExpr(s"unix_timestamp(x, 'yyyy-MM-dd mm:HH:ss')"), + Seq(Row(secs(ts4.getTime)), Row(null), Row(secs(ts3.getTime)), Row(null))) + + // invalid format + val invalid = df1.selectExpr(s"unix_timestamp(x, 'yyyy-MM-dd aa:HH:ss')") + checkAnswer(invalid, Seq(Row(null), Row(null), Row(null), Row(null))) + + // February + val y1 = "2016-02-29" + val y2 = "2017-02-29" + val ts5 = Timestamp.valueOf("2016-02-29 00:00:00") + val df2 = Seq(y1, y2).toDF("y") + checkAnswer( + df2.select(unix_timestamp(col("y"), "yyyy-MM-dd")), + Seq(Row(secs(ts5.getTime)), Row(null))) + + // Test unix_timestamp(timestamp, format) - format should be ignored + checkAnswer( + df.select(unix_timestamp(col("ts"), "yyyy-MM-dd")), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr("unix_timestamp(ts, 'invalid-format')"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + + val now = sql("select unix_timestamp()").collect().head.getLong(0) + checkAnswer( + sql(s"select timestamp_seconds($now)"), + Row(new java.util.Date(TimeUnit.SECONDS.toMillis(now)))) + } + } + } + + testGluten("to_unix_timestamp") { + Seq("corrected", "legacy").foreach { + legacyParserPolicy => + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> ZoneId.systemDefault().toString + ) { + val date1 = Date.valueOf("2015-07-24") + val date2 = Date.valueOf("2015-07-25") + val ts1 = Timestamp.valueOf("2015-07-24 10:00:00.3") + val ts2 = Timestamp.valueOf("2015-07-25 02:02:02.2") + val s1 = "2015/07/24 10:00:00.5" + val s2 = "2015/07/25 02:02:02.6" + val ss1 = "2015-07-24 10:00:00" + val ss2 = "2015-07-25 02:02:02" + val fmt = "yyyy/MM/dd HH:mm:ss.S" + val df = Seq((date1, ts1, s1, ss1), (date2, ts2, s2, ss2)).toDF("d", "ts", "s", "ss") + checkAnswer( + df.selectExpr("to_unix_timestamp(ts)"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr("to_unix_timestamp(ss)"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr(s"to_unix_timestamp(d, '$fmt')"), + Seq(Row(secs(date1.getTime)), Row(secs(date2.getTime)))) + checkAnswer( + df.selectExpr(s"to_unix_timestamp(s, '$fmt')"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + + val x1 = "2015-07-24 10:00:00" + val x2 = "2015-25-07 02:02:02" + val x3 = "2015-07-24 25:02:02" + val x4 = "2015-24-07 26:02:02" + val ts3 = Timestamp.valueOf("2015-07-24 02:25:02") + val ts4 = Timestamp.valueOf("2015-07-24 00:10:00") + + val df1 = Seq(x1, x2, x3, x4).toDF("x") + checkAnswer( + df1.selectExpr("to_unix_timestamp(x)"), + Seq(Row(secs(ts1.getTime)), Row(null), Row(null), Row(null))) + checkAnswer( + df1.selectExpr(s"to_unix_timestamp(x, 'yyyy-MM-dd mm:HH:ss')"), + Seq(Row(secs(ts4.getTime)), Row(null), Row(secs(ts3.getTime)), Row(null))) + + // February + val y1 = "2016-02-29" + val y2 = "2017-02-29" + val ts5 = Timestamp.valueOf("2016-02-29 00:00:00") + val df2 = Seq(y1, y2).toDF("y") + checkAnswer( + df2.select(unix_timestamp(col("y"), "yyyy-MM-dd")), + Seq(Row(secs(ts5.getTime)), Row(null))) + + // Test to_unix_timestamp(timestamp, format) - format should be ignored + checkAnswer( + df.selectExpr("to_unix_timestamp(ts, 'yyyy-MM-dd')"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr("to_unix_timestamp(ts, 'invalid-format')"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + + val invalid = df1.selectExpr(s"to_unix_timestamp(x, 'yyyy-MM-dd bb:HH:ss')") + checkAnswer(invalid, Seq(Row(null), Row(null), Row(null), Row(null))) + } + } + } + + testGluten("function to_date") { + val d1 = Date.valueOf("2015-07-22") + val d2 = Date.valueOf("2015-07-01") + val d3 = Date.valueOf("2014-12-31") + val t1 = Timestamp.valueOf("2015-07-22 10:00:00") + val t2 = Timestamp.valueOf("2014-12-31 23:59:59") + val t3 = Timestamp.valueOf("2014-12-31 23:59:59") + val s1 = "2015-07-22 10:00:00" + val s2 = "2014-12-31" + val s3 = "2014-31-12" + val df = Seq((d1, t1, s1), (d2, t2, s2), (d3, t3, s3)).toDF("d", "t", "s") + + checkAnswer( + df.select(to_date(col("t"))), + Seq( + Row(Date.valueOf("2015-07-22")), + Row(Date.valueOf("2014-12-31")), + Row(Date.valueOf("2014-12-31")))) + checkAnswer( + df.select(to_date(col("d"))), + Seq( + Row(Date.valueOf("2015-07-22")), + Row(Date.valueOf("2015-07-01")), + Row(Date.valueOf("2014-12-31")))) + checkAnswer( + df.select(to_date(col("s"))), + Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")), Row(null))) + + checkAnswer( + df.selectExpr("to_date(t)"), + Seq( + Row(Date.valueOf("2015-07-22")), + Row(Date.valueOf("2014-12-31")), + Row(Date.valueOf("2014-12-31")))) + checkAnswer( + df.selectExpr("to_date(d)"), + Seq( + Row(Date.valueOf("2015-07-22")), + Row(Date.valueOf("2015-07-01")), + Row(Date.valueOf("2014-12-31")))) + checkAnswer( + df.selectExpr("to_date(s)"), + Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")), Row(null))) + + // now with format + checkAnswer( + df.select(to_date(col("t"), "yyyy-MM-dd")), + Seq( + Row(Date.valueOf("2015-07-22")), + Row(Date.valueOf("2014-12-31")), + Row(Date.valueOf("2014-12-31")))) + checkAnswer( + df.select(to_date(col("d"), "yyyy-MM-dd")), + Seq( + Row(Date.valueOf("2015-07-22")), + Row(Date.valueOf("2015-07-01")), + Row(Date.valueOf("2014-12-31")))) + val confKey = SQLConf.LEGACY_TIME_PARSER_POLICY.key + withSQLConf(confKey -> "corrected") { + checkAnswer( + df.select(to_date(col("s"), "yyyy-MM-dd")), + Seq(Row(null), Row(Date.valueOf("2014-12-31")), Row(null))) + } + // legacyParserPolicy is not respected by Gluten. + // withSQLConf(confKey -> "exception") { + // checkExceptionMessage(df.select(to_date(col("s"), "yyyy-MM-dd"))) + // } + + // now switch format + checkAnswer( + df.select(to_date(col("s"), "yyyy-dd-MM")), + Seq(Row(null), Row(null), Row(Date.valueOf("2014-12-31")))) + + // invalid format + checkAnswer(df.select(to_date(col("s"), "yyyy-hh-MM")), Seq(Row(null), Row(null), Row(null))) + // velox getTimestamp function does not throw exception when format is "yyyy-dd-aa". + // val e = + // intercept[SparkUpgradeException](df.select(to_date(col("s"), "yyyy-dd-aa")).collect()) + // assert(e.getCause.isInstanceOf[IllegalArgumentException]) + // assert( + // e.getMessage.contains("You may get a different result due to the upgrading to Spark")) + + // February + val x1 = "2016-02-29" + val x2 = "2017-02-29" + val df1 = Seq(x1, x2).toDF("x") + checkAnswer(df1.select(to_date(col("x"))), Row(Date.valueOf("2016-02-29")) :: Row(null) :: Nil) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDeprecatedAPISuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDeprecatedAPISuite.scala new file mode 100644 index 000000000000..b6428773f1d0 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDeprecatedAPISuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenDeprecatedAPISuite extends DeprecatedAPISuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDynamicPartitionPruningSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDynamicPartitionPruningSuite.scala new file mode 100644 index 000000000000..d2222aa8a61f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenDynamicPartitionPruningSuite.scala @@ -0,0 +1,713 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.{BatchScanExecTransformer, FileSourceScanExecTransformer, FilterExecTransformerBase} + +import org.apache.spark.SparkConf +import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} +import org.apache.spark.sql.catalyst.plans.ExistenceJoin +import org.apache.spark.sql.connector.catalog.InMemoryTableCatalog +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeLike, ReusedExchangeExec} +import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.streaming.runtime.{MemoryStream, StreamingQueryWrapper} +import org.apache.spark.sql.internal.SQLConf + +abstract class GlutenDynamicPartitionPruningSuiteBase + extends DynamicPartitionPruningSuiteBase + with GlutenSQLTestsTrait { + + import testImplicits._ + + override def beforeAll(): Unit = { + prepareWorkDir() + super.beforeAll() + spark.sparkContext.setLogLevel("WARN") + } + + override def testNameBlackList: Seq[String] = Seq( + // overwritten with different plan + "SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec", + "Make sure dynamic pruning works on uncorrelated queries" + ) + + // === Following cases override super class's cases === + + ignoreGluten("DPP should not be rewritten as an existential join") { + // ignored: BroadcastHashJoinExec is from Vanilla Spark + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "1.5", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false" + ) { + val df = sql(s""" + |SELECT * FROM product p WHERE p.store_id NOT IN + | (SELECT f.store_id FROM fact_sk f JOIN dim_store d ON + | f.store_id = d.store_id + | WHERE d.state_province = 'NL' + | ) + """.stripMargin) + + val found = df.queryExecution.executedPlan.find { + case _ @BroadcastHashJoinExec(_, _, _: ExistenceJoin, _, _, _, _, _) => true + case _ => false + } + + assert(found.isEmpty) + } + } + + testGluten("no partition pruning when the build side is a stream") { + withTable("fact") { + val input = MemoryStream[Int] + val stream = input.toDF.select($"value".as("one"), ($"value" * 3).as("code")) + spark + .range(100) + .select($"id", ($"id" + 1).as("one"), ($"id" + 2).as("two"), ($"id" + 3).as("three")) + .write + .partitionBy("one") + .format(tableFormat) + .mode("overwrite") + .saveAsTable("fact") + val table = sql("SELECT * from fact f") + + // join a partitioned table with a stream + val joined = table.join(stream, Seq("one")).where("code > 40") + val query = joined.writeStream.format("memory").queryName("test").start() + input.addData(1, 10, 20, 40, 50) + try { + query.processAllAvailable() + } finally { + query.stop() + } + // search dynamic pruning predicates on the executed plan + val plan = query.asInstanceOf[StreamingQueryWrapper].streamingQuery.lastExecution.executedPlan + val ret = plan.find { + case s: FileSourceScanExecTransformer => + s.partitionFilters.exists { + case _: DynamicPruningExpression => true + case _ => false + } + case s: FileSourceScanExec => + s.partitionFilters.exists { + case _: DynamicPruningExpression => true + case _ => false + } + case _ => false + } + assert(ret.isDefined == false) + } + } + + testGluten("Make sure dynamic pruning works on uncorrelated queries") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + val df = sql(""" + |SELECT d.store_id, + | SUM(f.units_sold), + | (SELECT SUM(f.units_sold) + | FROM fact_stats f JOIN dim_stats d ON d.store_id = f.store_id + | WHERE d.country = 'US') AS total_prod + |FROM fact_stats f JOIN dim_stats d ON d.store_id = f.store_id + |WHERE d.country = 'US' + |GROUP BY 1 + """.stripMargin) + checkAnswer(df, Row(4, 50, 70) :: Row(5, 10, 70) :: Row(6, 10, 70) :: Nil) + + val plan = df.queryExecution.executedPlan + val countSubqueryBroadcasts = + collectWithSubqueries(plan) { + case _: SubqueryBroadcastExec => 1 + case _: ColumnarSubqueryBroadcastExec => 1 + }.sum + + val countReusedSubqueryBroadcasts = + collectWithSubqueries(plan) { + case ReusedSubqueryExec(_: SubqueryBroadcastExec) => 1 + case ReusedSubqueryExec(_: ColumnarSubqueryBroadcastExec) => 1 + }.sum + + assert(countSubqueryBroadcasts == 1) + assert(countReusedSubqueryBroadcasts == 1) + } + } + + testGluten( + "SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + + "canonicalization and exchange reuse") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.V2_BUCKETING_ENABLED.key -> "false") { + val df = sql(""" WITH view1 as ( + | SELECT f.store_id FROM fact_stats f WHERE f.units_sold = 70 + | ) + | + | SELECT * FROM view1 v1 join view1 v2 WHERE v1.store_id = v2.store_id + """.stripMargin) + + checkPartitionPruningPredicate(df, false, false) + val reuseExchangeNodes = collect(df.queryExecution.executedPlan) { + case se: ReusedExchangeExec => se + } + assert( + reuseExchangeNodes.size == 1, + "Expected plan to contain 1 ReusedExchangeExec " + + s"nodes. Found ${reuseExchangeNodes.size}") + + checkAnswer(df, Row(15, 15) :: Nil) + } + } + } + + testGluten("SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec") { + withTable("duplicate_keys") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { + Seq[(Int, String)]((1, "NL"), (1, "NL"), (3, "US"), (3, "US"), (3, "US")) + .toDF("store_id", "country") + .write + .format(tableFormat) + .saveAsTable("duplicate_keys") + + val df = sql(""" + |SELECT date_id, product_id FROM fact_sk f + |JOIN duplicate_keys s + |ON f.store_id = s.store_id WHERE s.country = 'US' AND date_id > 1050 + """.stripMargin) + + checkPartitionPruningPredicate(df, withSubquery = false, withBroadcast = true) + + val subqueryBroadcastExecs = collectWithSubqueries(df.queryExecution.executedPlan) { + case s: ColumnarSubqueryBroadcastExec => s + } + assert(subqueryBroadcastExecs.size === 1) + subqueryBroadcastExecs.foreach { + subqueryBroadcastExec => + assert(subqueryBroadcastExec.metrics("numOutputRows").value === 1) + } + + checkAnswer(df, Row(1060, 2) :: Row(1060, 2) :: Row(1060, 2) :: Nil) + } + } + } + + // === Following methods override super class's methods === + + override protected def collectDynamicPruningExpressions(plan: SparkPlan): Seq[Expression] = { + flatMap(plan) { + case s: FileSourceScanExecTransformer => + s.partitionFilters.collect { case d: DynamicPruningExpression => d.child } + case s: FileSourceScanExec => + s.partitionFilters.collect { case d: DynamicPruningExpression => d.child } + case s: BatchScanExecTransformer => + s.runtimeFilters.collect { case d: DynamicPruningExpression => d.child } + case s: BatchScanExec => + s.runtimeFilters.collect { case d: DynamicPruningExpression => d.child } + case _ => Nil + } + } + + override def checkPartitionPruningPredicate( + df: DataFrame, + withSubquery: Boolean, + withBroadcast: Boolean): Unit = { + df.collect() + + val plan = df.queryExecution.executedPlan + val dpExprs = collectDynamicPruningExpressions(plan) + val hasSubquery = dpExprs.exists { + case InSubqueryExec(_, _: SubqueryExec, _, _, _, _) => true + case _ => false + } + val subqueryBroadcast = dpExprs.collect { + case InSubqueryExec(_, b: SubqueryBroadcastExec, _, _, _, _) => b + case InSubqueryExec(_, b: ColumnarSubqueryBroadcastExec, _, _, _, _) => b + } + + val hasFilter = if (withSubquery) "Should" else "Shouldn't" + assert( + hasSubquery == withSubquery, + s"$hasFilter trigger DPP with a subquery duplicate:\n${df.queryExecution}") + val hasBroadcast = if (withBroadcast) "Should" else "Shouldn't" + assert( + subqueryBroadcast.nonEmpty == withBroadcast, + s"$hasBroadcast trigger DPP with a reused broadcast exchange:\n${df.queryExecution}") + + subqueryBroadcast.foreach { + s => + s.child match { + case _: ReusedExchangeExec => // reuse check ok. + case BroadcastQueryStageExec(_, _: ReusedExchangeExec, _) => // reuse check ok. + case b: BroadcastExchangeLike => + val hasReuse = plan.find { + case ReusedExchangeExec(_, e) => e eq b + case _ => false + }.isDefined + assert(hasReuse, s"$s\nshould have been reused in\n$plan") + case a: AdaptiveSparkPlanExec => + val broadcastQueryStage = collectFirst(a) { case b: BroadcastQueryStageExec => b } + val broadcastPlan = broadcastQueryStage.get.broadcast + val hasReuse = find(plan) { + case ReusedExchangeExec(_, e) => e eq broadcastPlan + case b: BroadcastExchangeLike => b eq broadcastPlan + case _ => false + }.isDefined + assert(hasReuse, s"$s\nshould have been reused in\n$plan") + case _ => + fail(s"Invalid child node found in\n$s") + } + } + + val isMainQueryAdaptive = plan.isInstanceOf[AdaptiveSparkPlanExec] + subqueriesAll(plan).filterNot(subqueryBroadcast.contains).foreach { + s => + val subquery = s match { + case r: ReusedSubqueryExec => r.child + case o => o + } + assert( + subquery.find(_.isInstanceOf[AdaptiveSparkPlanExec]).isDefined == isMainQueryAdaptive) + } + } + + override def checkDistinctSubqueries(df: DataFrame, n: Int): Unit = { + df.collect() + + val buf = collectDynamicPruningExpressions(df.queryExecution.executedPlan).collect { + case InSubqueryExec(_, b: SubqueryBroadcastExec, _, _, _, _) => + b.indices + case InSubqueryExec(_, b: ColumnarSubqueryBroadcastExec, _, _, _, _) => + b.indices + } + assert(buf.distinct.size == n) + } + + override def checkUnpushedFilters(df: DataFrame): Boolean = { + find(df.queryExecution.executedPlan) { + case FilterExec(condition, _) => + splitConjunctivePredicates(condition).exists { + case _: DynamicPruningExpression => true + case _ => false + } + case transformer: FilterExecTransformerBase => + splitConjunctivePredicates(transformer.cond).exists { + case _: DynamicPruningExpression => true + case _ => false + } + case FilterTransformer(condition, _) => + splitConjunctivePredicates(condition).exists { + case _: DynamicPruningExpression => true + case _ => false + } + case _ => false + }.isDefined + } + + object FilterTransformer { + def unapply(plan: SparkPlan): Option[(Expression, SparkPlan)] = { + plan match { + case transformer: FilterExecTransformerBase => + Some((transformer.cond, transformer.input)) + case _ => None + } + } + } +} + +abstract class GlutenDynamicPartitionPruningV1Suite extends GlutenDynamicPartitionPruningSuiteBase { + + import testImplicits._ + + /** Check the static scan metrics with and without DPP */ + testGluten("static scan metrics", DisableAdaptiveExecution("DPP in AQE must reuse broadcast")) { + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false" + ) { + withTable("fact", "dim") { + val numPartitions = 10 + + spark + .range(10) + .map(x => Tuple3(x, x + 1, 0)) + .toDF("did", "d1", "d2") + .write + .format(tableFormat) + .mode("overwrite") + .saveAsTable("dim") + + spark + .range(100) + .map(x => Tuple2(x, x % numPartitions)) + .toDF("f1", "fid") + .write + .partitionBy("fid") + .format(tableFormat) + .mode("overwrite") + .saveAsTable("fact") + + def getFactScan(plan: SparkPlan): SparkPlan = { + val scanOption = + find(plan) { + case s: FileSourceScanExec => + s.output.exists(_.find(_.argString(maxFields = 100).contains("fid")).isDefined) + case s: FileSourceScanExecTransformer => + s.output.exists(_.find(_.argString(maxFields = 100).contains("fid")).isDefined) + case s: BatchScanExec => + // we use f1 col for v2 tables due to schema pruning + s.output.exists(_.find(_.argString(maxFields = 100).contains("f1")).isDefined) + case s: BatchScanExecTransformer => + // we use f1 col for v2 tables due to schema pruning + s.output.exists(_.find(_.argString(maxFields = 100).contains("f1")).isDefined) + case _ => false + } + assert(scanOption.isDefined) + scanOption.get + } + + def getDriverMetrics(plan: SparkPlan, key: String): Option[SQLMetric] = plan match { + case fs: FileSourceScanExec => fs.driverMetrics.get(key) + case fs: FileSourceScanExecTransformer => fs.driverMetrics.get(key) + case _ => None + } + + // No dynamic partition pruning, so no static metrics + // All files in fact table are scanned + val df1 = sql("SELECT sum(f1) FROM fact") + df1.collect() + val scan1 = getFactScan(df1.queryExecution.executedPlan) + assert(!scan1.metrics.contains("staticFilesNum")) + assert(!scan1.metrics.contains("staticFilesSize")) + + val allFilesNum: Long = getDriverMetrics(scan1, "numFiles").map(_.value).getOrElse(-1) + val allFilesSize: Long = getDriverMetrics(scan1, "filesSize").map(_.value).getOrElse(-1) + val allPartitions: Long = + getDriverMetrics(scan1, "numPartitions").map(_.value).getOrElse(-1) + assert(allPartitions === numPartitions) + val pruningTimeVal1: Long = + getDriverMetrics(scan1, "pruningTime").map(_.value).getOrElse(-1) + assert(pruningTimeVal1 === 0) + + // No dynamic partition pruning, so no static metrics + // Only files from fid = 5 partition are scanned + val df2 = sql("SELECT sum(f1) FROM fact WHERE fid = 5") + df2.collect() + val scan2 = getFactScan(df2.queryExecution.executedPlan) + assert(!scan2.metrics.contains("staticFilesNum")) + assert(!scan2.metrics.contains("staticFilesSize")) + val partFilesNum: Long = getDriverMetrics(scan2, "numFiles").map(_.value).getOrElse(-1) + val partFilesSize: Long = getDriverMetrics(scan2, "filesSize").map(_.value).getOrElse(-1) + val partPartitions: Long = + getDriverMetrics(scan2, "numPartitions").map(_.value).getOrElse(-1) + assert(0 < partFilesNum && partFilesNum < allFilesNum) + assert(0 < partFilesSize && partFilesSize < allFilesSize) + assert(partPartitions === 1) + val pruningTimeVal2: Long = + getDriverMetrics(scan2, "pruningTime").map(_.value).getOrElse(-1) + assert(pruningTimeVal2 === 0) + + // Dynamic partition pruning is used + // Static metrics are as-if reading the whole fact table + // "Regular" metrics are as-if reading only the "fid = 5" partition + val df3 = sql("SELECT sum(f1) FROM fact, dim WHERE fid = did AND d1 = 6") + df3.collect() + val scan3 = getFactScan(df3.queryExecution.executedPlan) + val staticFilesNumVal: Long = + getDriverMetrics(scan3, "staticFilesNum").map(_.value).getOrElse(-1) + val staticFilesSizeVal: Long = + getDriverMetrics(scan3, "staticFilesSize").map(_.value).getOrElse(-1) + val numFilesVal: Long = getDriverMetrics(scan3, "numFiles").map(_.value).getOrElse(-1) + val filesSizeVal: Long = getDriverMetrics(scan3, "filesSize").map(_.value).getOrElse(-1) + val numPartitionsVal: Long = + getDriverMetrics(scan3, "numPartitions").map(_.value).getOrElse(-1) + val pruningTimeVal3: Long = + getDriverMetrics(scan3, "pruningTime").map(_.value).getOrElse(-1) + assert(staticFilesNumVal == allFilesNum) + assert(staticFilesSizeVal == allFilesSize) + assert(numFilesVal == partFilesNum) + assert(filesSizeVal == partFilesSize) + assert(numPartitionsVal === 1) + assert(pruningTimeVal3 > -1) + } + } + } +} + +class GlutenDynamicPartitionPruningV1SuiteAEOff + extends GlutenDynamicPartitionPruningV1Suite + with DisableAdaptiveExecutionSuite { + + import testImplicits._ + + testGluten( + "override static scan metrics", + DisableAdaptiveExecution("DPP in AQE must reuse broadcast")) { + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + // "spark.gluten.enabled" -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false" + ) { + withTable("fact", "dim") { + val numPartitions = 10 + + spark + .range(10) + .map(x => Tuple3(x, x + 1, 0)) + .toDF("did", "d1", "d2") + .write + .format(tableFormat) + .mode("overwrite") + .saveAsTable("dim") + + spark + .range(100) + .map(x => Tuple2(x, x % numPartitions)) + .toDF("f1", "fid") + .write + .partitionBy("fid") + .format(tableFormat) + .mode("overwrite") + .saveAsTable("fact") + + def getFactScan(plan: SparkPlan): SparkPlan = { + val scanOption = + find(plan) { + case s: FileSourceScanExecTransformer => + s.output.exists(_.find(_.argString(maxFields = 100).contains("fid")).isDefined) + case s: FileSourceScanExec => + s.output.exists(_.find(_.argString(maxFields = 100).contains("fid")).isDefined) + case s: BatchScanExecTransformer => + // we use f1 col for v2 tables due to schema pruning + s.output.exists(_.find(_.argString(maxFields = 100).contains("f1")).isDefined) + case s: BatchScanExec => + // we use f1 col for v2 tables due to schema pruning + s.output.exists(_.find(_.argString(maxFields = 100).contains("f1")).isDefined) + case _ => false + } + assert(scanOption.isDefined) + scanOption.get + } + + def getDriverMetrics(plan: SparkPlan, key: String): Option[SQLMetric] = plan match { + case fs: FileSourceScanExec => fs.driverMetrics.get(key) + case fs: FileSourceScanExecTransformer => fs.driverMetrics.get(key) + case _ => None + } + + // No dynamic partition pruning, so no static metrics + // All files in fact table are scanned + val df1 = sql("SELECT sum(f1) FROM fact") + df1.collect() + val scan1 = getFactScan(df1.queryExecution.executedPlan) + assert(!scan1.metrics.contains("staticFilesNum")) + assert(!scan1.metrics.contains("staticFilesSize")) + val allFilesNum: Long = getDriverMetrics(scan1, "numFiles").map(_.value).getOrElse(-1) + val allFilesSize: Long = getDriverMetrics(scan1, "filesSize").map(_.value).getOrElse(-1) + val allPartitions: Long = + getDriverMetrics(scan1, "numPartitions").map(_.value).getOrElse(-1) + assert(allPartitions === numPartitions) + val pruningTimeVal1: Long = + getDriverMetrics(scan1, "pruningTime").map(_.value).getOrElse(-1) + assert(pruningTimeVal1 === 0) + + // No dynamic partition pruning, so no static metrics + // Only files from fid = 5 partition are scanned + val df2 = sql("SELECT sum(f1) FROM fact WHERE fid = 5") + df2.collect() + val scan2 = getFactScan(df2.queryExecution.executedPlan) + assert(!scan2.metrics.contains("staticFilesNum")) + assert(!scan2.metrics.contains("staticFilesSize")) + val partFilesNum: Long = getDriverMetrics(scan2, "numFiles").map(_.value).getOrElse(-1) + val partFilesSize: Long = getDriverMetrics(scan2, "filesSize").map(_.value).getOrElse(-1) + val partPartitions: Long = + getDriverMetrics(scan2, "numPartitions").map(_.value).getOrElse(-1) + assert(0 < partFilesNum && partFilesNum < allFilesNum) + assert(0 < partFilesSize && partFilesSize < allFilesSize) + assert(partPartitions === 1) + val pruningTimeVal2: Long = + getDriverMetrics(scan2, "pruningTime").map(_.value).getOrElse(-1) + assert(pruningTimeVal2 === 0) + + // Dynamic partition pruning is used + // Static metrics are as-if reading the whole fact table + // "Regular" metrics are as-if reading only the "fid = 5" partition + val df3 = sql("SELECT sum(f1) FROM fact, dim WHERE fid = did AND d1 = 6") + df3.collect() + val scan3 = getFactScan(df3.queryExecution.executedPlan) + val staticFilesNumVal: Long = + getDriverMetrics(scan3, "staticFilesNum").map(_.value).getOrElse(-1) + val staticFilesSizeVal: Long = + getDriverMetrics(scan3, "staticFilesSize").map(_.value).getOrElse(-1) + val numFilesVal: Long = getDriverMetrics(scan3, "numFiles").map(_.value).getOrElse(-1) + val filesSizeVal: Long = getDriverMetrics(scan3, "filesSize").map(_.value).getOrElse(-1) + val numPartitionsVal: Long = + getDriverMetrics(scan3, "numPartitions").map(_.value).getOrElse(-1) + val pruningTimeVal3: Long = + getDriverMetrics(scan3, "pruningTime").map(_.value).getOrElse(-1) + assert(staticFilesNumVal == allFilesNum) + assert(staticFilesSizeVal == allFilesSize) + assert(numFilesVal == partFilesNum) + assert(filesSizeVal == partFilesSize) + assert(numPartitionsVal === 1) + assert(pruningTimeVal3 > -1) + } + } + } +} + +class GlutenDynamicPartitionPruningV1SuiteAEOn + extends GlutenDynamicPartitionPruningV1Suite + with EnableAdaptiveExecutionSuite { + + testGluten("SPARK-39447: Avoid AssertionError in AdaptiveSparkPlanExec.doExecuteBroadcast") { + val df = sql(""" + |WITH empty_result AS ( + | SELECT * FROM fact_stats WHERE product_id < 0 + |) + |SELECT * + |FROM (SELECT /*+ SHUFFLE_MERGE(fact_sk) */ empty_result.store_id + | FROM fact_sk + | JOIN empty_result + | ON fact_sk.product_id = empty_result.product_id) t2 + | JOIN empty_result + | ON t2.store_id = empty_result.store_id + """.stripMargin) + + checkPartitionPruningPredicate(df, false, false) + checkAnswer(df, Nil) + } + + testGluten( + "SPARK-37995: PlanAdaptiveDynamicPruningFilters should use prepareExecutedPlan " + + "rather than createSparkPlan to re-plan subquery") { + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false" + ) { + val df = sql(""" + |SELECT f.date_id, f.store_id FROM fact_sk f + |JOIN dim_store s ON f.store_id = s.store_id AND s.country = 'NL' + |WHERE s.state_province != (SELECT max(state_province) FROM dim_stats) + """.stripMargin) + + checkPartitionPruningPredicate(df, true, false) + checkAnswer(df, Row(1000, 1) :: Row(1010, 2) :: Row(1020, 2) :: Nil) + } + } + + testGluten("Filter with DynamicPruningExpression") { + withTable("fact_stats_non_partition") { + spark + .table("fact_stats") + .write + .format(tableFormat) + .saveAsTable("fact_stats_non_partition") + + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false") { + val df = sql(""" + |SELECT f.date_id, f.product_id, f.units_sold, f.store_id FROM ( + | select * from fact_stats + | union all + | select * from fact_stats_non_partition + |) f + |JOIN dim_stats s + |ON f.store_id = s.store_id WHERE s.country = 'DE' + """.stripMargin) + checkAnswer( + df, + Row(1030, 2, 10, 3) :: + Row(1040, 2, 50, 3) :: + Row(1050, 2, 50, 3) :: + Row(1060, 2, 50, 3) :: + Row(1030, 2, 10, 3) :: + Row(1040, 2, 50, 3) :: + Row(1050, 2, 50, 3) :: + Row(1060, 2, 50, 3) :: Nil + ) + val filters = collect(df.queryExecution.executedPlan) { case f: FilterExec => f } + assert(filters.isEmpty) + val filterTransformerWithDPPs = collect(df.queryExecution.executedPlan) { + case f: FilterExecTransformerBase + if f.cond.exists(_.isInstanceOf[DynamicPruningExpression]) => + f + } + assert(filterTransformerWithDPPs.nonEmpty) + } + } + } +} + +abstract class GlutenDynamicPartitionPruningV2Suite extends GlutenDynamicPartitionPruningSuiteBase { + override protected def runAnalyzeColumnCommands: Boolean = false + + override protected def initState(): Unit = { + spark.conf.set("spark.sql.catalog.testcat", classOf[InMemoryTableCatalog].getName) + spark.conf.set("spark.sql.defaultCatalog", "testcat") + } +} + +class GlutenDynamicPartitionPruningV2SuiteAEOff + extends GlutenDynamicPartitionPruningV2Suite + with DisableAdaptiveExecutionSuite + +class GlutenDynamicPartitionPruningV2SuiteAEOn + extends GlutenDynamicPartitionPruningV2Suite + with EnableAdaptiveExecutionSuite + +// Test DPP with file scan disabled by user for some reason, which can also mock the situation +// that scan is not transformable. +class GlutenDynamicPartitionPruningV1SuiteAEOnDisableScan + extends GlutenDynamicPartitionPruningV1SuiteAEOn { + override def sparkConf: SparkConf = { + super.sparkConf.set(GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key, "false") + } +} + +// Same as above except AQE is off. +class GlutenDynamicPartitionPruningV1SuiteAEOffDisableScan + extends GlutenDynamicPartitionPruningV2SuiteAEOff { + override def sparkConf: SparkConf = { + super.sparkConf.set(GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key, "false") + } +} + +// Test DPP with batch scan disabled by user for some reason, which can also mock the situation +// that scan is not transformable. +class GlutenDynamicPartitionPruningV2SuiteAEOnDisableScan + extends GlutenDynamicPartitionPruningV2SuiteAEOn { + override def sparkConf: SparkConf = { + super.sparkConf.set(GlutenConfig.COLUMNAR_BATCHSCAN_ENABLED.key, "false") + } +} + +// Same as above except AQE is off. +class GlutenDynamicPartitionPruningV2SuiteAEOffDisableScan + extends GlutenDynamicPartitionPruningV2SuiteAEOff { + override def sparkConf: SparkConf = { + super.sparkConf.set(GlutenConfig.COLUMNAR_BATCHSCAN_ENABLED.key, "false") + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenEmptyInSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenEmptyInSuite.scala new file mode 100644 index 000000000000..ede561cbd6b1 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenEmptyInSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenEmptyInSuite extends EmptyInSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenExpressionsSchemaSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenExpressionsSchemaSuite.scala new file mode 100644 index 000000000000..0dd285c7426a --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenExpressionsSchemaSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenExpressionsSchemaSuite extends ExpressionsSchemaSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenExtraStrategiesSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenExtraStrategiesSuite.scala new file mode 100644 index 000000000000..3c3b438f3cf0 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenExtraStrategiesSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenExtraStrategiesSuite extends ExtraStrategiesSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenFileBasedDataSourceSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenFileBasedDataSourceSuite.scala new file mode 100644 index 000000000000..9ae4a0f063f4 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenFileBasedDataSourceSuite.scala @@ -0,0 +1,275 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.BatchScanExecTransformer + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} +import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST +import org.apache.spark.sql.catalyst.expressions.{GreaterThan, Literal} +import org.apache.spark.sql.execution.FileSourceScanLike +import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, FileScan} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.functions.rand +import org.apache.spark.sql.internal.SQLConf + +import org.apache.hadoop.fs.Path + +import java.io.FileNotFoundException + +import scala.collection.mutable + +class GlutenFileBasedDataSourceSuite extends FileBasedDataSourceSuite with GlutenSQLTestsTrait { + import testImplicits._ + + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") + .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") + } + + // test data path is jar path, so failed, test code is same with spark + testGluten("Option recursiveFileLookup: disable partition inferring") { + val dataPath = getWorkspaceFilePath( + "sql", + "core", + "src", + "test", + "resources").toString + "/" + "test-data/text-partitioned" + + val df = spark.read + .format("binaryFile") + .option("recursiveFileLookup", true) + .load(dataPath) + + assert(!df.columns.contains("year"), "Expect partition inferring disabled") + val fileList = df.select("path").collect().map(_.getString(0)) + + val expectedFileList = Array( + dataPath + "/year=2014/data.txt", + dataPath + "/year=2015/data.txt" + ).map(path => "file:" + new Path(path).toString) + + assert(fileList.toSet === expectedFileList.toSet) + } + + testGluten("Spark native readers should respect spark.sql.caseSensitive - parquet") { + withTempDir { + dir => + val format = "parquet" + val tableName = s"spark_25132_${format}_native" + val tableDir = dir.getCanonicalPath + s"/$tableName" + withTable(tableName) { + val end = 5 + val data = spark.range(end).selectExpr("id as A", "id * 2 as b", "id * 3 as B") + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + data.write.format(format).mode("overwrite").save(tableDir) + } + sql(s"CREATE TABLE $tableName (a LONG, b LONG) USING $format LOCATION '$tableDir'") + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + checkAnswer(sql(s"select a from $tableName"), data.select("A")) + checkAnswer(sql(s"select A from $tableName"), data.select("A")) + + // TODO: gluten can catch exception in executor side, but cannot catch SparkException + // in Driver side + // RuntimeException is triggered at executor side, which is then wrapped as + // SparkException at driver side + // val e1 = intercept[SparkException] { + // sql(s"select b from $tableName").collect() + // } + // + // assert( + // e1.getCause.isInstanceOf[RuntimeException] && + // e1.getMessage.contains( + // """Found duplicate field(s) b in case-insensitive mode """)) + // val e2 = intercept[SparkException] { + // sql(s"select B from $tableName").collect() + // } + // assert( + // e2.getCause.isInstanceOf[RuntimeException] && + // e2.getMessage.contains( + // """Found duplicate field(s) b in case-insensitive mode""")) + } + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + checkAnswer(sql(s"select a from $tableName"), (0 until end).map(_ => Row(null))) + checkAnswer(sql(s"select b from $tableName"), data.select("b")) + } + } + } + } + + testGluten("SPARK-22790,SPARK-27668: spark.sql.sources.compressionFactor takes effect") { + Seq(1.0, 0.5).foreach { + compressionFactor => + withSQLConf( + SQLConf.FILE_COMPRESSION_FACTOR.key -> compressionFactor.toString, + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "350") { + withTempPath { + workDir => + // the file size is 504 bytes + val workDirPath = workDir.getAbsolutePath + val data1 = Seq(100, 200, 300, 400).toDF("count") + data1.write.orc(workDirPath + "/data1") + val df1FromFile = spark.read.orc(workDirPath + "/data1") + val data2 = Seq(100, 200, 300, 400).toDF("count") + data2.write.orc(workDirPath + "/data2") + val df2FromFile = spark.read.orc(workDirPath + "/data2") + val joinedDF = df1FromFile.join(df2FromFile, Seq("count")) + if (compressionFactor == 0.5) { + val bJoinExec = collect(joinedDF.queryExecution.executedPlan) { + case bJoin: BroadcastHashJoinExec => bJoin + } + assert(bJoinExec.nonEmpty) + val smJoinExec = collect(joinedDF.queryExecution.executedPlan) { + case smJoin: SortMergeJoinExec => smJoin + } + assert(smJoinExec.isEmpty) + } else { + // compressionFactor is 1.0 + val bJoinExec = collect(joinedDF.queryExecution.executedPlan) { + case bJoin: BroadcastHashJoinExec => bJoin + } + assert(bJoinExec.isEmpty) + val smJoinExec = collect(joinedDF.queryExecution.executedPlan) { + case smJoin: SortMergeJoinExec => smJoin + } + assert(smJoinExec.nonEmpty) + } + } + } + } + } + + testGluten("SPARK-25237 compute correct input metrics in FileScanRDD") { + // TODO: Test CSV V2 as well after it implements [[SupportsReportStatistics]]. + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "csv") { + withTempPath { + p => + val path = p.getAbsolutePath + spark.range(1000).repartition(1).write.csv(path) + val bytesReads = new mutable.ArrayBuffer[Long]() + val bytesReadListener = new SparkListener() { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + bytesReads += taskEnd.taskMetrics.inputMetrics.bytesRead + } + } + sparkContext.addSparkListener(bytesReadListener) + try { + spark.read.csv(path).limit(1).collect() + sparkContext.listenerBus.waitUntilEmpty() + // plan is different, so metric is different + assert(bytesReads.sum === 7864) + } finally { + sparkContext.removeSparkListener(bytesReadListener) + } + } + } + } + + testGluten("SPARK-41017: filter pushdown with nondeterministic predicates") { + withTempPath { + path => + val pathStr = path.getCanonicalPath + spark.range(10).write.parquet(pathStr) + Seq("parquet", "").foreach { + useV1SourceList => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> useV1SourceList) { + val scan = spark.read.parquet(pathStr) + val df = scan.where(rand() > 0.5 && $"id" > 5) + val filters = df.queryExecution.executedPlan.collect { + case f: FileSourceScanLike => f.dataFilters + case b: BatchScanExec => b.scan.asInstanceOf[FileScan].dataFilters + case b: BatchScanExecTransformer => b.scan.asInstanceOf[FileScan].dataFilters + }.flatten + assert(filters.contains(GreaterThan(scan.logicalPlan.output.head, Literal(5L)))) + } + } + } + } + + Seq("orc", "parquet").foreach { + format => + testQuietly(GLUTEN_TEST + s"Enabling/disabling ignoreMissingFiles using $format") { + def testIgnoreMissingFiles(options: Map[String, String]): Unit = { + withTempDir { + dir => + val basePath = dir.getCanonicalPath + + Seq("0").toDF("a").write.format(format).save(new Path(basePath, "second").toString) + Seq("1").toDF("a").write.format(format).save(new Path(basePath, "fourth").toString) + + val firstPath = new Path(basePath, "first") + val thirdPath = new Path(basePath, "third") + val fs = thirdPath.getFileSystem(spark.sessionState.newHadoopConf()) + Seq("2").toDF("a").write.format(format).save(firstPath.toString) + Seq("3").toDF("a").write.format(format).save(thirdPath.toString) + val files = Seq(firstPath, thirdPath).flatMap { + p => fs.listStatus(p).filter(_.isFile).map(_.getPath) + } + + val df = spark.read + .options(options) + .format(format) + .load( + new Path(basePath, "first").toString, + new Path(basePath, "second").toString, + new Path(basePath, "third").toString, + new Path(basePath, "fourth").toString) + + // Make sure all data files are deleted and can't be opened. + files.foreach(f => fs.delete(f, false)) + assert(fs.delete(thirdPath, true)) + for (f <- files) { + intercept[FileNotFoundException](fs.open(f)) + } + + checkAnswer(df, Seq(Row("0"), Row("1"))) + } + } + + // Test set ignoreMissingFiles via SQL Conf + // Rewrite this test as error msg is different from velox and data Source reader options + // is not supported. + for { + (ignore, options, sqlConf) <- Seq( + // Set via SQL Conf: leave options empty + ("true", Map.empty[String, String], "true"), + ("false", Map.empty[String, String], "false") + ) + sources <- Seq("", format) + } { + withSQLConf( + SQLConf.USE_V1_SOURCE_LIST.key -> sources, + SQLConf.IGNORE_MISSING_FILES.key -> sqlConf) { + if (ignore.toBoolean) { + testIgnoreMissingFiles(options) + } else { + val exception = intercept[SparkException] { + testIgnoreMissingFiles(options) + } + assert(exception.getMessage().contains("No such file or directory")) + } + } + } + } + } + +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenFileScanSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenFileScanSuite.scala new file mode 100644 index 000000000000..d5885afaee9c --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenFileScanSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenFileScanSuite extends FileScanSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenGeneratorFunctionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenGeneratorFunctionSuite.scala new file mode 100644 index 000000000000..b3d51e802985 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenGeneratorFunctionSuite.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.spark.sql.internal.SQLConf + +class GlutenGeneratorFunctionSuite extends GeneratorFunctionSuite with GlutenSQLTestsTrait { + testGluten("SPARK-45171: Handle evaluated nondeterministic expression") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { + val df = sql("select explode(array(rand(0)))") + checkAnswer(df, Row(0.5488135024422883)) + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenInjectRuntimeFilterSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenInjectRuntimeFilterSuite.scala new file mode 100644 index 000000000000..26e11f84387a --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenInjectRuntimeFilterSuite.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.spark.sql.internal.SQLConf + +class GlutenInjectRuntimeFilterSuite extends InjectRuntimeFilterSuite with GlutenSQLTestsBaseTrait { + + testGluten("GLUTEN-9849: bloom filter applied to partition filter") { + withSQLConf( + SQLConf.RUNTIME_BLOOM_FILTER_APPLICATION_SIDE_SCAN_SIZE_THRESHOLD.key -> "3000", + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "false", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "2000" + ) { + assertRewroteWithBloomFilter( + "select * from bf5part join bf2 on " + + "bf5part.f5 = bf2.c2 where bf2.a2 = 67") + } + } + +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenIntervalFunctionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenIntervalFunctionsSuite.scala new file mode 100644 index 000000000000..0a354a1fc39e --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenIntervalFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenIntervalFunctionsSuite extends IntervalFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenJoinSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenJoinSuite.scala new file mode 100644 index 000000000000..5ef4056201ed --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenJoinSuite.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenJoinSuite extends JoinSuite with GlutenSQLTestsTrait { + + override def testNameBlackList: Seq[String] = Seq( + // Below tests are to verify operators, just skip. + "join operator selection", + "broadcasted hash join operator selection", + "broadcasted hash outer join operator selection", + "broadcasted existence join operator selection", + "SPARK-28323: PythonUDF should be able to use in join condition", + "SPARK-28345: PythonUDF predicate should be able to pushdown to join", + "cross join with broadcast", + "test SortMergeJoin output ordering", + "SPARK-22445 Respect stream-side child's needCopyResult in BroadcastHashJoin", + "SPARK-32330: Preserve shuffled hash join build side partitioning", + "SPARK-32383: Preserve hash join (BHJ and SHJ) stream side ordering", + "SPARK-32399: Full outer shuffled hash join", + "SPARK-32649: Optimize BHJ/SHJ inner/semi join with empty hashed relation", + "SPARK-34593: Preserve broadcast nested loop join partitioning and ordering", + "SPARK-35984: Config to force applying shuffled hash join", + "test SortMergeJoin (with spill)", + "SPARK-36612: Support left outer join build left or right" + + " outer join build right in shuffled hash join", + // NaN is not supported currently, just skip. + "NaN and -0.0 in join keys" + ) + + testGluten("test case sensitive for BHJ") { + spark.sql("create table t_bhj(a int, b int, C int) using parquet") + spark.sql("insert overwrite t_bhj select id as a, (id+1) as b, (id+2) as c from range(3)") + val sql = + """ + |select /*+ BROADCAST(t1) */ t0.a, t0.b + |from t_bhj as t0 join t_bhj as t1 on t0.a = t1.a and t0.b = t1.b and t0.c = t1.c + |group by t0.a, t0.b + |order by t0.a, t0.b + |""".stripMargin + checkAnswer(spark.sql(sql), Seq(Row(0, 1), Row(1, 2), Row(2, 3))) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenJsonFunctionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenJsonFunctionsSuite.scala new file mode 100644 index 000000000000..51602f144bb7 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenJsonFunctionsSuite.scala @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.gluten.execution.GlutenPlan + +class GlutenJsonFunctionsSuite extends JsonFunctionsSuite with GlutenSQLTestsTrait { + import testImplicits._ + + testGluten("SPARK-42782: Hive compatibility check for get_json_object ") { + val book0 = "{\"author\":\"Nigel Rees\",\"title\":\"Sayings of the Century\"" + + ",\"category\":\"reference\",\"price\":8.95}" + val backet0 = "[1,2,{\"b\":\"y\",\"a\":\"x\"}]" + val backet = "[" + backet0 + ",[3,4],[5,6]]" + val backetFlat = backet0.substring(0, backet0.length() - 1) + ",3,4,5,6]" + + val book = "[" + book0 + ",{\"author\":\"Herman Melville\",\"title\":\"Moby Dick\"," + + "\"category\":\"fiction\",\"price\":8.99" + + ",\"isbn\":\"0-553-21311-3\"},{\"author\":\"J. R. R. Tolkien\"" + + ",\"title\":\"The Lord of the Rings\",\"category\":\"fiction\"" + + ",\"reader\":[{\"age\":25,\"name\":\"bob\"},{\"age\":26,\"name\":\"jack\"}]" + + ",\"price\":22.99,\"isbn\":\"0-395-19395-8\"}]" + + val json = "{\"store\":{\"fruit\":[{\"weight\":8,\"type\":\"apple\"}," + + "{\"weight\":9,\"type\":\"pear\"}],\"basket\":" + backet + ",\"book\":" + book + + ",\"bicycle\":{\"price\":19.95,\"color\":\"red\"}}" + + ",\"email\":\"amy@only_for_json_udf_test.net\"" + + ",\"owner\":\"amy\",\"zip code\":\"94025\",\"fb:testid\":\"1234\"}" + + // Basic test + runTest(json, "$.owner", "amy") + runTest(json, "$.store.bicycle", "{\"price\":19.95,\"color\":\"red\"}") + runTest(json, "$.store.book", book) + runTest(json, "$.store.book[0]", book0) + // runTest(json, "$.store.book[*]", book) - not supported in velox + runTest(json, "$.store.book[0].category", "reference") + // runTest(json, "$.store.book[*].category", + // "[\"reference\",\"fiction\",\"fiction\"]") - not supported in velox + // runTest(json, "$.store.book[*].reader[0].age", "25") - not supported in velox + // runTest(json, "$.store.book[*].reader[*].age", "[25,26]") - not supported in velox + runTest(json, "$.store.basket[0][1]", "2") + // runTest(json, "$.store.basket[*]", backet) - not supported in velox + // runTest(json, "$.store.basket[*][0]", "[1,3,5]") - not supported in velox + // runTest(json, "$.store.basket[0][*]", backet0) - not supported in velox + // runTest(json, "$.store.basket[*][*]", backetFlat) - not supported in velox + runTest(json, "$.store.basket[0][2].b", "y") + // runTest(json, "$.store.basket[0][*].b", "[\"y\"]") - not supported in velox + runTest(json, "$.non_exist_key", null) + runTest(json, "$.store.book[10]", null) + runTest(json, "$.store.book[0].non_exist_key", null) + // runTest(json, "$.store.basket[*].non_exist_key", null) - not supported in velox + // runTest(json, "$.store.basket[0][*].non_exist_key", null) - not supported in velox + // runTest(json, "$.store.basket[*][*].non_exist_key", null) - not supported in velox + runTest(json, "$.zip code", "94025") + runTest(json, "$.fb:testid", "1234") + // runTest("{\"a\":\"b\nc\"}", "$.a", "b\nc") - not supported in velox + + // Test root array + runTest("[1,2,3]", "$[0]", "1") + runTest("[1,2,3]", "$.[0]", null) // Not supported in spark and velox + runTest("[1,2,3]", "$.[1]", null) // Not supported in spark and velox + runTest("[1,2,3]", "$[1]", "2") + + runTest("[1,2,3]", "$[3]", null) + runTest("[1,2,3]", "$.[*]", null) // Not supported in spark and velox + // runTest("[1,2,3]", "$[*]", "[1,2,3]") - not supported in velox + // runTest("[1,2,3]", "$", "[1,2,3]") - not supported in velox + runTest("[{\"k1\":\"v1\"},{\"k2\":\"v2\"},{\"k3\":\"v3\"}]", "$[2]", "{\"k3\":\"v3\"}") + runTest("[{\"k1\":\"v1\"},{\"k2\":\"v2\"},{\"k3\":\"v3\"}]", "$[2].k3", "v3") + runTest("[{\"k1\":[{\"k11\":[1,2,3]}]}]", "$[0].k1[0].k11[1]", "2") + runTest("[{\"k1\":[{\"k11\":[1,2,3]}]}]", "$[0].k1[0].k11", "[1,2,3]") + runTest("[{\"k1\":[{\"k11\":[1,2,3]}]}]", "$[0].k1[0]", "{\"k11\":[1,2,3]}") + runTest("[{\"k1\":[{\"k11\":[1,2,3]}]}]", "$[0].k1", "[{\"k11\":[1,2,3]}]") + runTest("[{\"k1\":[{\"k11\":[1,2,3]}]}]", "$[0]", "{\"k1\":[{\"k11\":[1,2,3]}]}") + runTest("[[1,2,3],[4,5,6],[7,8,9]]", "$[1]", "[4,5,6]") + runTest("[[1,2,3],[4,5,6],[7,8,9]]", "$[1][0]", "4") + runTest("[\"a\",\"b\"]", "$[1]", "b") + runTest("[[\"a\",\"b\"]]", "$[0][1]", "b") + + runTest("[1,2,3]", "[0]", null) + runTest("[1,2,3]", "$0", null) + runTest("[1,2,3]", "0", null) + runTest("[1,2,3]", "$.", null) + + runTest("[1,2,3]", "$", "[1,2,3]") + runTest("{\"a\":4}", "$", "{\"a\":4}") + + def runTest(json: String, path: String, exp: String): Unit = { + checkAnswer(Seq(json).toDF().selectExpr(s"get_json_object(value, '$path')"), Row(exp)) + } + } + + testGluten("function get_json_object - Codegen Support") { + withTempView("GetJsonObjectTable") { + val data = Seq(("1", """{"f1": "value1", "f5": 5.23}""")).toDF("key", "jstring") + data.createOrReplaceTempView("GetJsonObjectTable") + val df = sql("SELECT key, get_json_object(jstring, '$.f1') FROM GetJsonObjectTable") + val plan = df.queryExecution.executedPlan + assert(plan.isInstanceOf[GlutenPlan]) + checkAnswer(df, Seq(Row("1", "value1"))) + } + } + testGluten("function get_json_object - path is null") { + val data = Seq(("""{"name": "alice", "age": 5}""", "")).toDF("a", "b") + val df = data.selectExpr("get_json_object(a, null)") + val plan = df.queryExecution.executedPlan + assert(plan.isInstanceOf[GlutenPlan]) + checkAnswer(df, Row(null)) + } + + testGluten("function get_json_object - json is null") { + val data = Seq(("""{"name": "alice", "age": 5}""", "")).toDF("a", "b") + val df = data.selectExpr("get_json_object(null, '$.name')") + val plan = df.queryExecution.executedPlan + assert(plan.isInstanceOf[GlutenPlan]) + checkAnswer(df, Row(null)) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenLateralColumnAliasSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenLateralColumnAliasSuite.scala new file mode 100644 index 000000000000..27cda1c323af --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenLateralColumnAliasSuite.scala @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +// TODO: fix in Spark-4.0 change back to LateralColumnAliasSuite +class GlutenLateralColumnAliasSuite extends LateralColumnAliasSuiteBase with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenMathFunctionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenMathFunctionsSuite.scala new file mode 100644 index 000000000000..ee39f0138504 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenMathFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenMathFunctionsSuite extends MathFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenMetadataCacheSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenMetadataCacheSuite.scala new file mode 100644 index 000000000000..d9fc6fd05e1b --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenMetadataCacheSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenMetadataCacheSuite extends MetadataCacheSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenMiscFunctionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenMiscFunctionsSuite.scala new file mode 100644 index 000000000000..a95d8a2b2e5c --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenMiscFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenMiscFunctionsSuite extends MiscFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenNestedDataSourceSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenNestedDataSourceSuite.scala new file mode 100644 index 000000000000..d139221f631a --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenNestedDataSourceSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenNestedDataSourceV1Suite extends NestedDataSourceV1Suite with GlutenSQLTestsTrait {} + +class GlutenNestedDataSourceV2Suite extends NestedDataSourceV2Suite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenParametersSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenParametersSuite.scala new file mode 100644 index 000000000000..0887a7416fd8 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenParametersSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenParametersSuite extends ParametersSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenProcessingTimeSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenProcessingTimeSuite.scala new file mode 100644 index 000000000000..f8ab9b16adf4 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenProcessingTimeSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenProcessingTimeSuite extends ProcessingTimeSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenProductAggSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenProductAggSuite.scala new file mode 100644 index 000000000000..9cb35efbfbd3 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenProductAggSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenProductAggSuite extends ProductAggSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenReplaceNullWithFalseInPredicateEndToEndSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenReplaceNullWithFalseInPredicateEndToEndSuite.scala new file mode 100644 index 000000000000..e345309ab114 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenReplaceNullWithFalseInPredicateEndToEndSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenReplaceNullWithFalseInPredicateEndToEndSuite + extends ReplaceNullWithFalseInPredicateEndToEndSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenResolveDefaultColumnsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenResolveDefaultColumnsSuite.scala new file mode 100644 index 000000000000..2d1570be26c5 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenResolveDefaultColumnsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenResolveDefaultColumnsSuite + extends ResolveDefaultColumnsSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenRuntimeNullChecksV2Writes.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenRuntimeNullChecksV2Writes.scala new file mode 100644 index 000000000000..abd997bea82e --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenRuntimeNullChecksV2Writes.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenRuntimeNullChecksV2Writes extends RuntimeNullChecksV2Writes with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSQLInsertTestSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSQLInsertTestSuite.scala new file mode 100644 index 000000000000..44a370cbc374 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSQLInsertTestSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenFileSourceSQLInsertTestSuite + extends FileSourceSQLInsertTestSuite + with GlutenSQLTestsTrait {} + +class GlutenDSV2SQLInsertTestSuite extends DSV2SQLInsertTestSuite with GlutenTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSQLQuerySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSQLQuerySuite.scala new file mode 100644 index 000000000000..c75569af259a --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSQLQuerySuite.scala @@ -0,0 +1,274 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.spark.SparkException +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.internal.SQLConf + +class GlutenSQLQuerySuite extends SQLQuerySuite with GlutenSQLTestsTrait { + import testImplicits._ + + testGluten("SPARK-28156: self-join should not miss cached view") { + withTable("table1") { + withView("table1_vw") { + withTempView("cachedview") { + val df = Seq.tabulate(5)(x => (x, x + 1, x + 2, x + 3)).toDF("a", "b", "c", "d") + df.write.mode("overwrite").format("parquet").saveAsTable("table1") + sql("drop view if exists table1_vw") + sql("create view table1_vw as select * from table1") + + val cachedView = sql("select a, b, c, d from table1_vw") + + cachedView.createOrReplaceTempView("cachedview") + cachedView.persist() + + val queryDf = sql(s"""select leftside.a, leftside.b + |from cachedview leftside + |join cachedview rightside + |on leftside.a = rightside.a + """.stripMargin) + + val inMemoryTableScan = collect(queryDf.queryExecution.executedPlan) { + case i: InMemoryTableScanExec => i + } + assert(inMemoryTableScan.size == 2) + checkAnswer(queryDf, Row(0, 1) :: Row(1, 2) :: Row(2, 3) :: Row(3, 4) :: Row(4, 5) :: Nil) + } + } + } + + } + + // Velox throw exception : An unsupported nested encoding was found. + ignore( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-33338: GROUP BY using literal map should not fail") { + withTable("t") { + withTempDir { + dir => + sql( + s"CREATE TABLE t USING PARQUET LOCATION '${dir.toURI}' AS SELECT map('k1', 'v1') m," + + s" 'k1' k") + Seq( + "SELECT map('k1', 'v1')[k] FROM t GROUP BY 1", + "SELECT map('k1', 'v1')[k] FROM t GROUP BY map('k1', 'v1')[k]", + "SELECT map('k1', 'v1')[k] a FROM t GROUP BY a" + ).foreach(statement => checkAnswer(sql(statement), Row("v1"))) + } + } + } + + testGluten("SPARK-33593: Vector reader got incorrect data with binary partition value") { + Seq("false").foreach( + value => { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> value) { + withTable("t1") { + sql("""CREATE TABLE t1(name STRING, id BINARY, part BINARY) + |USING PARQUET PARTITIONED BY (part)""".stripMargin) + sql("INSERT INTO t1 PARTITION(part = 'Spark SQL') VALUES('a', X'537061726B2053514C')") + checkAnswer( + sql("SELECT name, cast(id as string), cast(part as string) FROM t1"), + Row("a", "Spark SQL", "Spark SQL")) + } + } + + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> value) { + withTable("t2") { + sql("""CREATE TABLE t2(name STRING, id BINARY, part BINARY) + |USING PARQUET PARTITIONED BY (part)""".stripMargin) + sql("INSERT INTO t2 PARTITION(part = 'Spark SQL') VALUES('a', X'537061726B2053514C')") + checkAnswer( + sql("SELECT name, cast(id as string), cast(part as string) FROM t2"), + Row("a", "Spark SQL", "Spark SQL")) + } + } + }) + } + + testGluten( + "SPARK-33677: LikeSimplification should be skipped if pattern contains any escapeChar") { + withTempView("df") { + Seq("m@ca").toDF("s").createOrReplaceTempView("df") + + val e = intercept[SparkException] { + sql("SELECT s LIKE 'm%@ca' ESCAPE '%' FROM df").collect() + } + assert( + e.getMessage.contains( + "Escape character must be followed by '%', '_' or the escape character itself")) + + checkAnswer(sql("SELECT s LIKE 'm@@ca' ESCAPE '@' FROM df"), Row(true)) + } + } + + testGluten("the escape character is not allowed to end with") { + withTempView("df") { + Seq("jialiuping").toDF("a").createOrReplaceTempView("df") + + val e = intercept[SparkException] { + sql("SELECT a LIKE 'jialiuping%' ESCAPE '%' FROM df").collect() + } + assert( + e.getMessage.contains( + "Escape character must be followed by '%', '_' or the escape character itself")) + } + } + + ignoreGluten("StreamingQueryProgress.numInputRows should be correct") { + withTempDir { + dir => + val path = dir.toURI.getPath + val numRows = 20 + val df = spark.range(0, numRows) + df.write.mode("overwrite").format("parquet").save(path) + val q = spark.readStream + .format("parquet") + .schema(df.schema) + .load(path) + .writeStream + .format("memory") + .queryName("test") + .start() + q.processAllAvailable + val inputOutputPairs = q.recentProgress.map(p => (p.numInputRows, p.sink.numOutputRows)) + + // numInputRows and sink.numOutputRows must be the same + assert(inputOutputPairs.forall(x => x._1 == x._2)) + + // Sum of numInputRows must match the total number of rows of the input + assert(inputOutputPairs.map(_._1).sum == numRows) + } + } + + testGluten("SPARK-47939: Explain should work with parameterized queries") { + def checkQueryPlan(df: DataFrame, plan: String): Unit = assert( + df.collect() + .map(_.getString(0)) + .map(_.replaceAll("#[0-9]+", "#N")) + // Remove the backend keyword in c2r/r2c. + .map(_.replaceAll("[A-Za-z]*ColumnarToRow", "ColumnarToRow")) + .map(_.replaceAll("RowTo[A-Za-z]*Columnar", "RowToColumnar")) + === Array(plan.stripMargin) + ) + + checkQueryPlan( + spark.sql("explain select ?", Array(1)), + """== Physical Plan == + |ColumnarToRow + |+- ^(1) ProjectExecTransformer [1 AS 1#N] + | +- ^(1) InputIteratorTransformer[] + | +- RowToColumnar + | +- *(1) Scan OneRowRelation[] + | + |""" + ) + checkQueryPlan( + spark.sql("explain select :first", Map("first" -> 1)), + """== Physical Plan == + |ColumnarToRow + |+- ^(1) ProjectExecTransformer [1 AS 1#N] + | +- ^(1) InputIteratorTransformer[] + | +- RowToColumnar + | +- *(1) Scan OneRowRelation[] + | + |""" + ) + + checkQueryPlan( + spark.sql("explain explain explain select ?", Array(1)), + """== Physical Plan == + |Execute ExplainCommand + | +- ExplainCommand ExplainCommand 'PosParameterizedQuery [1], SimpleMode, SimpleMode + + |""" + ) + checkQueryPlan( + spark.sql("explain explain explain select :first", Map("first" -> 1)), + // scalastyle:off + """== Physical Plan == + |Execute ExplainCommand + | +- ExplainCommand ExplainCommand 'NameParameterizedQuery [first], [1], SimpleMode, SimpleMode + + |""" + // scalastyle:on + ) + + checkQueryPlan( + spark.sql("explain describe select ?", Array(1)), + """== Physical Plan == + |Execute DescribeQueryCommand + | +- DescribeQueryCommand select ? + + |""" + ) + checkQueryPlan( + spark.sql("explain describe select :first", Map("first" -> 1)), + """== Physical Plan == + |Execute DescribeQueryCommand + | +- DescribeQueryCommand select :first + + |""" + ) + + checkQueryPlan( + spark.sql("explain extended select * from values (?, ?) t(x, y)", Array(1, "a")), + """== Parsed Logical Plan == + |'PosParameterizedQuery [1, a] + |+- 'Project [*] + | +- 'SubqueryAlias t + | +- 'UnresolvedInlineTable [x, y], [[posparameter(39), posparameter(42)]] + + |== Analyzed Logical Plan == + |x: int, y: string + |Project [x#N, y#N] + |+- SubqueryAlias t + | +- LocalRelation [x#N, y#N] + + |== Optimized Logical Plan == + |LocalRelation [x#N, y#N] + + |== Physical Plan == + |LocalTableScan [x#N, y#N] + |""" + ) + checkQueryPlan( + spark.sql( + "explain extended select * from values (:first, :second) t(x, y)", + Map("first" -> 1, "second" -> "a") + ), + """== Parsed Logical Plan == + |'NameParameterizedQuery [first, second], [1, a] + |+- 'Project [*] + | +- 'SubqueryAlias t + | +- 'UnresolvedInlineTable [x, y], [[namedparameter(first), namedparameter(second)]] + + |== Analyzed Logical Plan == + |x: int, y: string + |Project [x#N, y#N] + |+- SubqueryAlias t + | +- LocalRelation [x#N, y#N] + + |== Optimized Logical Plan == + |LocalRelation [x#N, y#N] + + |== Physical Plan == + |LocalTableScan [x#N, y#N] + |""" + ) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala new file mode 100644 index 000000000000..8d1c08c73dff --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala @@ -0,0 +1,794 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.gluten.config.GlutenConfig +// import org.apache.gluten.exception.GlutenException +import org.apache.gluten.utils.{BackendTestSettings, BackendTestUtils} + +import org.apache.spark.{SparkConf, TestUtils} +// import org.apache.spark.ErrorMessageFormat.MINIMAL +// import org.apache.spark.SparkThrowableHelper.getMessage +import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator +import org.apache.spark.sql.catalyst.optimizer.{ConstantFolding, ConvertToLocalRelation, NullPropagation} +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_SECOND +import org.apache.spark.sql.catalyst.util.stringToFile +import org.apache.spark.sql.execution.WholeStageCodegenExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.TimestampTypes +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.tags.ExtendedSQLTest +import org.apache.spark.util.Utils + +import java.io.File +import java.net.URI +import java.nio.file.Files +import java.util.Locale + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +// import scala.util.control.NonFatal + +/** + * Originated from org.apache.spark.sql.SQLQueryTestSuite, with the following modifications: + * - Overwrite the generated golden files to remove failed queries. + * - Overwrite the generated golden files to update expected results for exception message + * mismatches, result order mismatches in non-order-sensitive queries, and minor precision scale + * mismatches. + * - Remove the AnalyzerTest as it's not within the scope of the Gluten project. + * + * NOTE: DO NOT simply copy-paste this file for supporting new Spark versions. SQLQueryTestSuite is + * actively modified in Spark, so compare the difference and apply the necessary changes. + */ +@ExtendedSQLTest +class GlutenSQLQueryTestSuite + extends QueryTest + with SharedSparkSession + with SQLHelper + with SQLQueryTestHelper { + + import IntegratedUDFTestUtils._ + + // ==== Start of modifications for Gluten. ==== + protected val baseResourcePath = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toFile + } + protected val testDataPath = new File(baseResourcePath, "test-data").getAbsolutePath + + protected val overwriteInputFilePath = new File( + BackendTestSettings.instance.getSQLQueryTestSettings.getResourceFilePath, + "inputs").getAbsolutePath + + protected val overwriteGoldenFilePath = new File( + BackendTestSettings.instance.getSQLQueryTestSettings.getResourceFilePath, + "results").getAbsolutePath + protected val analyzerGoldenFilePath = + new File(baseResourcePath, "analyzer-results").getAbsolutePath + + private val isCHBackend = BackendTestUtils.isCHBackendLoaded() + + // List of supported cases to run with a certain backend, in lower case. + private val supportedList: Set[String] = + BackendTestSettings.instance.getSQLQueryTestSettings.getSupportedSQLQueryTests ++ + BackendTestSettings.instance.getSQLQueryTestSettings.getOverwriteSQLQueryTests + + private val normalizeRegex = "#\\d+L?".r + private val nodeNumberRegex = "[\\^*]\\(\\d+\\)".r + private def normalizeIds(plan: String): String = { + val normalizedPlan = nodeNumberRegex.replaceAllIn(plan, "") + val map = new mutable.HashMap[String, String]() + normalizeRegex + .findAllMatchIn(normalizedPlan) + .map(_.toString) + .foreach(map.getOrElseUpdate(_, (map.size + 1).toString)) + normalizeRegex.replaceAllIn(normalizedPlan, regexMatch => s"#${map(regexMatch.toString)}") + } + + override protected def sparkConf: SparkConf = { + val conf = super.sparkConf + // Fewer shuffle partitions to speed up testing. + .set(SQLConf.SHUFFLE_PARTITIONS, 4) + // use Java 8 time API to handle negative years properly + .set(SQLConf.DATETIME_JAVA8API_ENABLED, true) + // SPARK-39564: don't print out serde to avoid introducing complicated and error-prone + // regex magic. + .set("spark.test.noSerdeInExplain", "true") + .setAppName("Gluten-UT") + .set("spark.driver.memory", "1G") + .set("spark.sql.adaptive.enabled", "true") + .set("spark.sql.files.maxPartitionBytes", "134217728") + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "1024MB") + .set("spark.plugins", "org.apache.gluten.GlutenPlugin") + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set(GlutenConfig.SMALL_FILE_THRESHOLD.key, "0") + + if (isCHBackend) { + conf + .set("spark.io.compression.codec", "LZ4") + .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .set(GlutenConfig.NATIVE_VALIDATION_ENABLED.key, "false") + .set("spark.sql.files.openCostInBytes", "134217728") + .set("spark.unsafe.exceptionOnMemoryLeak", "true") + } else { + conf + .set("spark.unsafe.exceptionOnMemoryLeak", "true") + // Avoid static evaluation for literal input by spark catalyst. + .set( + SQLConf.OPTIMIZER_EXCLUDED_RULES.key, + ConvertToLocalRelation.ruleName + + "," + ConstantFolding.ruleName + "," + NullPropagation.ruleName) + } + conf + } + + /** List of test cases to ignore, in lower cases. */ + protected def ignoreList: Set[String] = Set( + "ignored.sql", // Do NOT remove this one. It is here to test the ignore functionality. + "explain-aqe.sql", // Explain is different in Gluten. + "explain-cbo.sql", // Explain is different in Gluten. + "explain.sql" // Explain is different in Gluten. + ) ++ otherIgnoreList ++ + BackendTestSettings.instance.getSQLQueryTestSettings.getIgnoredSQLQueryTests + + /** + * Returns the desired file path for results, given the input file. This is implemented as a + * function because differente Suites extending this class may want their results files with + * different names or in different locations. + */ + protected def resultFileForInputFile(file: File): String = { + file.getAbsolutePath.replace(overwriteInputFilePath, overwriteGoldenFilePath) + ".out" + } + + protected lazy val listTestCases: Seq[TestCase] = { + listFilesRecursively(new File(overwriteInputFilePath)) + .flatMap { + file => + var resultFile = resultFileForInputFile(file) + var analyzerResultFile = + file.getAbsolutePath.replace(overwriteInputFilePath, overwriteGoldenFilePath) + ".out" + // JDK-4511638 changes 'toString' result of Float/Double + // JDK-8282081 changes DataTimeFormatter 'F' symbol + if (Utils.isJavaVersionAtLeast21) { + if (new File(resultFile + ".java21").exists()) resultFile += ".java21" + if (new File(analyzerResultFile + ".java21").exists()) analyzerResultFile += ".java21" + } + val absPath = file.getAbsolutePath + val testCaseName = absPath.stripPrefix(overwriteInputFilePath).stripPrefix(File.separator) + + // Create test cases of test types that depend on the input filename. + val newTestCases: Seq[TestCase] = + if ( + file.getAbsolutePath.startsWith( + s"$overwriteInputFilePath${File.separator}udf${File.separator}postgreSQL") + ) { + Seq(TestScalaUDF("udf"), TestPythonUDF("udf"), TestScalarPandasUDF("udf")).map { + udf => + UDFPgSQLTestCase(s"$testCaseName - ${udf.prettyName}", absPath, resultFile, udf) + } + } else if ( + file.getAbsolutePath.startsWith(s"$overwriteInputFilePath${File.separator}udf") + ) { + Seq(TestScalaUDF("udf"), TestPythonUDF("udf"), TestScalarPandasUDF("udf")).map { + udf => UDFTestCase(s"$testCaseName - ${udf.prettyName}", absPath, resultFile, udf) + } + } else if ( + file.getAbsolutePath.startsWith(s"$overwriteInputFilePath${File.separator}udaf") + ) { + Seq(TestGroupedAggPandasUDF("udaf")).map { + udf => UDAFTestCase(s"$testCaseName - ${udf.prettyName}", absPath, resultFile, udf) + } + } else if ( + file.getAbsolutePath.startsWith(s"$overwriteInputFilePath${File.separator}udtf") + ) { + Seq(TestUDTFSet(AllTestUDTFs)).map { + udtfSet => + UDTFSetTestCase(s"$testCaseName - Python UDTFs", absPath, resultFile, udtfSet) + } + } else if ( + file.getAbsolutePath.startsWith(s"$overwriteInputFilePath${File.separator}postgreSQL") + ) { + PgSQLTestCase(testCaseName, absPath, resultFile) :: Nil + } else if ( + file.getAbsolutePath.startsWith(s"$overwriteInputFilePath${File.separator}nonansi") + ) { + NonAnsiTestCase(testCaseName, absPath, resultFile) :: Nil + } else if ( + file.getAbsolutePath.startsWith( + s"$overwriteInputFilePath${File.separator}timestampNTZ") + ) { + TimestampNTZTestCase(testCaseName, absPath, resultFile) :: Nil + } else if ( + file.getAbsolutePath.startsWith(s"$overwriteInputFilePath${File.separator}cte.sql") + ) { + CTETestCase(testCaseName, absPath, resultFile) :: Nil + } else { + RegularTestCase(testCaseName, absPath, resultFile) :: Nil + } + // Also include a copy of each of the above test cases as an analyzer test. + newTestCases.flatMap { + test => + test match { + case _: UDAFTestCase => + // Skip creating analyzer test. + Seq(test) + case _ => + Seq( + test, + test.asAnalyzerTest( + newName = s"${test.name}_analyzer_test", + newResultFile = analyzerResultFile)) + } + } + } + .sortBy(_.name) + } + // ==== End of modifications for Gluten. ==== + + protected val inputFilePath = new File(baseResourcePath, "inputs").getAbsolutePath + protected val goldenFilePath = new File(baseResourcePath, "results").getAbsolutePath + + // SPARK-32106 Since we add SQL test 'transform.sql' will use `cat` command, + // here we need to ignore it. + private val otherIgnoreList = + if (TestUtils.testCommandAvailable("/bin/bash")) Nil else Set("transform.sql") + + // Create all the test cases. + listTestCases.foreach(createScalaTestCase) + + /** A single SQL query's output. */ + protected case class QueryOutput(sql: String, schema: String, output: String) { + override def toString: String = { + // We are explicitly not using multi-line string due to stripMargin removing "|" in output. + s"-- !query\n" + + sql + "\n" + + s"-- !query schema\n" + + schema + "\n" + + s"-- !query output\n" + + output + } + } + + protected def createScalaTestCase(testCase: TestCase): Unit = { + if ( + // Modified for Gluten to use exact name matching. + !supportedList.exists( + t => testCase.name.toLowerCase(Locale.ROOT) == t.toLowerCase(Locale.ROOT)) || + ignoreList.exists(t => testCase.name.toLowerCase(Locale.ROOT) == t.toLowerCase(Locale.ROOT)) + ) { + // Create a test case to ignore this case. + ignore(testCase.name) { /* Do nothing */ } + } else { + testCase match { + case udfTestCase: UDFTest + if udfTestCase.udf.isInstanceOf[TestPythonUDF] && !shouldTestPythonUDFs => + ignore( + s"${testCase.name} is skipped because " + + s"[$pythonExec] and/or pyspark were not available.") { + /* Do nothing */ + } + case udfTestCase: UDFTest + if udfTestCase.udf.isInstanceOf[TestScalarPandasUDF] && !shouldTestPandasUDFs => + ignore( + s"${testCase.name} is skipped because pyspark," + + s"pandas and/or pyarrow were not available in [$pythonExec].") { + /* Do nothing */ + } + case udfTestCase: UDFTest + if udfTestCase.udf.isInstanceOf[TestGroupedAggPandasUDF] && + !shouldTestPandasUDFs => + ignore( + s"${testCase.name} is skipped because pyspark," + + s"pandas and/or pyarrow were not available in [$pythonExec].") { + /* Do nothing */ + } + case _ => + // Create a test case to run this case. + test(testCase.name) { + runSqlTestCase(testCase, listTestCases) + } + } + } + } + + /** Run a test case. */ + protected def runSqlTestCase(testCase: TestCase, listTestCases: Seq[TestCase]): Unit = { + def splitWithSemicolon(seq: Seq[String]) = { + seq.mkString("\n").split("(?<=[^\\\\]);") + } + + def splitCommentsAndCodes(input: String) = input.split("\n").partition { + line => + val newLine = line.trim + newLine.startsWith("--") && !newLine.startsWith("--QUERY-DELIMITER") + } + + val input = Files.readString(new File(testCase.inputFile).toPath) + + val (comments, code) = splitCommentsAndCodes(input) + + // If `--IMPORT` found, load code from another test case file, then insert them + // into the head in this test. + val importedTestCaseName = comments.filter(_.startsWith("--IMPORT ")).map(_.substring(9)) + val importedCode = importedTestCaseName.flatMap { + testCaseName => + listTestCases.find(_.name == testCaseName).map { + testCase => + val input = Files.readString(new File(testCase.inputFile).toPath) + val (_, code) = splitCommentsAndCodes(input) + code + } + }.flatten + + val allCode = importedCode ++ code + val tempQueries = if (allCode.exists(_.trim.startsWith("--QUERY-DELIMITER"))) { + // Although the loop is heavy, only used for bracketed comments test. + val queries = new ArrayBuffer[String] + val otherCodes = new ArrayBuffer[String] + var tempStr = "" + var start = false + for (c <- allCode) { + if (c.trim.startsWith("--QUERY-DELIMITER-START")) { + start = true + queries ++= splitWithSemicolon(otherCodes.toSeq) + otherCodes.clear() + } else if (c.trim.startsWith("--QUERY-DELIMITER-END")) { + start = false + queries += s"\n${tempStr.stripSuffix(";")}" + tempStr = "" + } else if (start) { + tempStr += s"\n$c" + } else { + otherCodes += c + } + } + if (otherCodes.nonEmpty) { + queries ++= splitWithSemicolon(otherCodes.toSeq) + } + queries.toSeq + } else { + splitWithSemicolon(allCode).toSeq + } + + // List of SQL queries to run + val queries = tempQueries + .map(_.trim) + .filter(_ != "") + .toSeq + // Fix misplacement when comment is at the end of the query. + .map(_.split("\n").filterNot(_.startsWith("--")).mkString("\n")) + .map(_.trim) + .filter(_ != "") + + val settingLines = comments.filter(_.startsWith("--SET ")).map(_.substring(6)) + val settings = settingLines.flatMap(_.split(",").map { + kv => + val (conf, value) = kv.span(_ != '=') + conf.trim -> value.substring(1).trim + }) + + if (regenerateGoldenFiles) { + runQueries(queries, testCase, settings) + } else { + // A config dimension has multiple config sets, and a config set has multiple configs. + // - config dim: Seq[Seq[(String, String)]] + // - config set: Seq[(String, String)] + // - config: (String, String)) + // We need to do cartesian product for all the config dimensions, to get a list of + // config sets, and run the query once for each config set. + val configDimLines = comments.filter(_.startsWith("--CONFIG_DIM")).map(_.substring(12)) + val configDims = configDimLines.groupBy(_.takeWhile(_ != ' ')).mapValues { + lines => + lines + .map(_.dropWhile(_ != ' ').substring(1)) + .map(_.split(",") + .map { + kv => + val (conf, value) = kv.span(_ != '=') + conf.trim -> value.substring(1).trim + } + .toSeq) + .toSeq + } + + val configSets = configDims.values.foldLeft(Seq(Seq[(String, String)]())) { + (res, dim) => dim.flatMap(configSet => res.map(_ ++ configSet)) + } + + configSets.foreach { + configSet => + try { + runQueries(queries, testCase, settings ++ configSet) + } catch { + case e: Throwable => + val configs = configSet.map { case (k, v) => s"$k=$v" } + logError(s"Error using configs: ${configs.mkString(",")}") + throw e + } + } + } + } + + def hasNoDuplicateColumns(schema: String): Boolean = { + val columnAndTypes = schema.replaceFirst("^struct<", "").stripSuffix(">").split(",") + columnAndTypes.size == columnAndTypes.distinct.size + } + + def expandCTEQueryAndCompareResult( + session: SparkSession, + query: String, + output: ExecutionOutput): Unit = { + val triggerCreateViewTest = + try { + val logicalPlan: LogicalPlan = session.sessionState.sqlParser.parsePlan(query) + !logicalPlan.isInstanceOf[Command] && + output.schema.get != emptySchema && + hasNoDuplicateColumns(output.schema.get) + } catch { + case _: ParseException => return + } + + // For non-command query with CTE, compare the results of selecting from view created on the + // original query. + if (triggerCreateViewTest) { + val createView = s"CREATE temporary VIEW cte_view AS $query" + val selectFromView = "SELECT * FROM cte_view" + val dropViewIfExists = "DROP VIEW IF EXISTS cte_view" + session.sql(createView) + val (selectViewSchema, selectViewOutput) = + handleExceptions(getNormalizedQueryExecutionResult(session, selectFromView)) + // Compare results. + assertResult( + output.schema.get, + s"Schema did not match for CTE query and select from its view: \n$output") { + selectViewSchema + } + assertResult( + output.output, + s"Result did not match for CTE query and select from its view: \n${output.sql}") { + selectViewOutput.mkString("\n").replaceAll("\\s+$", "") + } + // Drop view. + session.sql(dropViewIfExists) + } + } + + protected def runQueries( + queries: Seq[String], + testCase: TestCase, + configSet: Seq[(String, String)]): Unit = { + // Create a local SparkSession to have stronger isolation between different test cases. + // This does not isolate catalog changes. + val localSparkSession = spark.newSession() + + testCase match { + case udfTestCase: SQLQueryTestSuite#UDFTest => + registerTestUDF(udfTestCase.udf, localSparkSession) + case udtfTestCase: SQLQueryTestSuite#UDTFSetTest => + registerTestUDTFs(udtfTestCase.udtfSet, localSparkSession) + case _ => + } + + testCase match { + case _: SQLQueryTestSuite#PgSQLTest => + // booleq/boolne used by boolean.sql + localSparkSession.udf.register("booleq", (b1: Boolean, b2: Boolean) => b1 == b2) + localSparkSession.udf.register("boolne", (b1: Boolean, b2: Boolean) => b1 != b2) + // vol used by boolean.sql and case.sql. + localSparkSession.udf.register("vol", (s: String) => s) + localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) + localSparkSession.conf.set(SQLConf.LEGACY_INTERVAL_ENABLED.key, true) + case _: SQLQueryTestSuite#NonAnsiTest => + localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, false) + case _: SQLQueryTestSuite#TimestampNTZTest => + localSparkSession.conf.set( + SQLConf.TIMESTAMP_TYPE.key, + TimestampTypes.TIMESTAMP_NTZ.toString) + case _ => + localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) + } + + if (configSet.nonEmpty) { + // Execute the list of set operation in order to add the desired configs + val setOperations = configSet.map { case (key, value) => s"set $key=$value" } + logInfo(s"Setting configs: ${setOperations.mkString(", ")}") + setOperations.foreach(localSparkSession.sql) + } + + // Run the SQL queries preparing them for comparison. + val outputs: Seq[QueryTestOutput] = queries.map { + sql => + testCase match { + case _: AnalyzerTest => + val (_, output) = + handleExceptions(getNormalizedQueryAnalysisResult(localSparkSession, sql)) + // We might need to do some query canonicalization in the future. + AnalyzerOutput( + sql = sql, + schema = None, + output = output.mkString("\n").replaceAll("\\s+$", "")) + case _ => + val (schema, output) = + handleExceptions(getNormalizedQueryExecutionResult(localSparkSession, sql)) + // We might need to do some query canonicalization in the future. + val executionOutput = ExecutionOutput( + sql = sql, + schema = Some(schema), + // GLUTEN-3559: Overwrite scalar-subquery-select.sql test + output = normalizeIds(output.mkString("\n").replaceAll("\\s+$", "")) + ) + if (testCase.isInstanceOf[CTETest]) { + expandCTEQueryAndCompareResult(localSparkSession, sql, executionOutput) + } + executionOutput + } + } + + if (regenerateGoldenFiles) { + // Again, we are explicitly not using multi-line string due to stripMargin removing "|". + val goldenOutput = { + s"-- Automatically generated by ${getClass.getSimpleName}\n" + + outputs.mkString("\n\n\n") + "\n" + } + val resultFile = new File(testCase.resultFile) + val parent = resultFile.getParentFile + if (!parent.exists()) { + assert(parent.mkdirs(), "Could not create directory: " + parent) + } + stringToFile(resultFile, goldenOutput) + } + + // This is a temporary workaround for SPARK-28894. The test names are truncated after + // the last dot due to a bug in SBT. This makes easier to debug via Jenkins test result + // report. See SPARK-28894. + // See also SPARK-29127. It is difficult to see the version information in the failed test + // cases so the version information related to Python was also added. + val clue = testCase match { + case udfTestCase: SQLQueryTestSuite#UDFTest + if udfTestCase.udf.isInstanceOf[TestPythonUDF] && shouldTestPythonUDFs => + s"${testCase.name}${System.lineSeparator()}Python: $pythonVer${System.lineSeparator()}" + case udfTestCase: SQLQueryTestSuite#UDFTest + if udfTestCase.udf.isInstanceOf[TestScalarPandasUDF] && shouldTestPandasUDFs => + s"${testCase.name}${System.lineSeparator()}" + + s"Python: $pythonVer Pandas: $pandasVer PyArrow: $pyarrowVer${System.lineSeparator()}" + case udfTestCase: SQLQueryTestSuite#UDFTest + if udfTestCase.udf.isInstanceOf[TestGroupedAggPandasUDF] && + shouldTestPandasUDFs => + s"${testCase.name}${System.lineSeparator()}" + + s"Python: $pythonVer Pandas: $pandasVer PyArrow: $pyarrowVer${System.lineSeparator()}" + case udtfTestCase: SQLQueryTestSuite#UDTFSetTest + if udtfTestCase.udtfSet.udtfs.forall(_.isInstanceOf[TestPythonUDTF]) && + shouldTestPythonUDFs => + s"${testCase.name}${System.lineSeparator()}Python: $pythonVer${System.lineSeparator()}" + case _ => + s"${testCase.name}${System.lineSeparator()}" + } + + withClue(clue) { + testCase match { + case _: AnalyzerTest => + readGoldenFileAndCompareResults(testCase.resultFile, outputs, AnalyzerOutput) + case _ => + readGoldenFileAndCompareResults(testCase.resultFile, outputs, ExecutionOutput) + } + } + } + + /** Load built-in test tables into the SparkSession. */ + private def createTestTables(session: SparkSession): Unit = { + import session.implicits._ + + // Before creating test tables, deletes orphan directories in warehouse dir + Seq("testdata", "arraydata", "mapdata", "aggtest", "onek", "tenk1").foreach { + dirName => + val f = new File(new URI(s"${conf.warehousePath}/$dirName")) + if (f.exists()) { + Utils.deleteRecursively(f) + } + } + + (1 to 100) + .map(i => (i, i.toString)) + .toDF("key", "value") + .repartition(1) + .write + .format("parquet") + .saveAsTable("testdata") + + ((Seq(1, 2, 3), Seq(Seq(1, 2, 3))) :: (Seq(2, 3, 4), Seq(Seq(2, 3, 4))) :: Nil) + .toDF("arraycol", "nestedarraycol") + .write + .format("parquet") + .saveAsTable("arraydata") + + (Tuple1(Map(1 -> "a1", 2 -> "b1", 3 -> "c1", 4 -> "d1", 5 -> "e1")) :: + Tuple1(Map(1 -> "a2", 2 -> "b2", 3 -> "c2", 4 -> "d2")) :: + Tuple1(Map(1 -> "a3", 2 -> "b3", 3 -> "c3")) :: + Tuple1(Map(1 -> "a4", 2 -> "b4")) :: + Tuple1(Map(1 -> "a5")) :: Nil) + .toDF("mapcol") + .write + .format("parquet") + .saveAsTable("mapdata") + + session.read + .format("csv") + .options(Map("delimiter" -> "\t", "header" -> "false")) + .schema("a int, b float") + .load(testDataPath + "/postgresql/agg.data") + .write + .format("parquet") + .saveAsTable("aggtest") + + session.read + .format("csv") + .options(Map("delimiter" -> "\t", "header" -> "false")) + .schema(""" + |unique1 int, + |unique2 int, + |two int, + |four int, + |ten int, + |twenty int, + |hundred int, + |thousand int, + |twothousand int, + |fivethous int, + |tenthous int, + |odd int, + |even int, + |stringu1 string, + |stringu2 string, + |string4 string + """.stripMargin) + .load(testDataPath + "/postgresql/onek.data") + .write + .format("parquet") + .saveAsTable("onek") + + session.read + .format("csv") + .options(Map("delimiter" -> "\t", "header" -> "false")) + .schema(""" + |unique1 int, + |unique2 int, + |two int, + |four int, + |ten int, + |twenty int, + |hundred int, + |thousand int, + |twothousand int, + |fivethous int, + |tenthous int, + |odd int, + |even int, + |stringu1 string, + |stringu2 string, + |string4 string + """.stripMargin) + .load(testDataPath + "/postgresql/tenk.data") + .write + .format("parquet") + .saveAsTable("tenk1") + } + + protected def removeTestTables(session: SparkSession): Unit = { + session.sql("DROP TABLE IF EXISTS testdata") + session.sql("DROP TABLE IF EXISTS arraydata") + session.sql("DROP TABLE IF EXISTS mapdata") + session.sql("DROP TABLE IF EXISTS aggtest") + session.sql("DROP TABLE IF EXISTS onek") + session.sql("DROP TABLE IF EXISTS tenk1") + } + + override def beforeAll(): Unit = { + super.beforeAll() + createTestTables(spark) + RuleExecutor.resetMetrics() + CodeGenerator.resetCompileTime() + WholeStageCodegenExec.resetCodeGenTime() + } + + override def afterAll(): Unit = { + try { + removeTestTables(spark) + + // For debugging dump some statistics about how much time was spent in various optimizer rules + logWarning(RuleExecutor.dumpTimeSpent()) + + val codeGenTime = WholeStageCodegenExec.codeGenTime.toDouble / NANOS_PER_SECOND + val compileTime = CodeGenerator.compileTime.toDouble / NANOS_PER_SECOND + val codegenInfo = + s""" + |=== Metrics of Whole-stage Codegen === + |Total code generation time: $codeGenTime seconds + |Total compile time: $compileTime seconds + """.stripMargin + logWarning(codegenInfo) + } finally { + super.afterAll() + } + } + + /** + * Consumes contents from a single golden file and compares the expected results against the + * output of running a query. + */ + def readGoldenFileAndCompareResults( + resultFile: String, + outputs: Seq[QueryTestOutput], + makeOutput: (String, Option[String], String) => QueryTestOutput): Unit = { + // Read back the golden file. + val expectedOutputs: Seq[QueryTestOutput] = { + val goldenOutput = Files.readString(new File(resultFile).toPath) + val segments = goldenOutput.split("-- !query.*\n") + + val numSegments = outputs.map(_.numSegments).sum + 1 + assert( + segments.size == numSegments, + s"Expected $numSegments blocks in result file but got " + + s"${segments.size}. Try regenerate the result files.") + var curSegment = 0 + outputs.map { + output => + val result = if (output.numSegments == 3) { + makeOutput( + segments(curSegment + 1).trim, // SQL + Some(segments(curSegment + 2).trim), // Schema + segments(curSegment + 3).replaceAll("\\s+$", "") + ) // Output + } else { + makeOutput( + segments(curSegment + 1).trim, // SQL + None, // Schema + segments(curSegment + 2).replaceAll("\\s+$", "") + ) // Output + } + curSegment += output.numSegments + result + } + } + + // Compare results. + assertResult(expectedOutputs.size, s"Number of queries should be ${expectedOutputs.size}") { + outputs.size + } + + outputs.zip(expectedOutputs).zipWithIndex.foreach { + case ((output, expected), i) => + assertResult(expected.sql, s"SQL query did not match for query #$i\n${expected.sql}") { + output.sql + } + assertResult( + expected.schema, + s"Schema did not match for query #$i\n${expected.sql}: $output") { + output.schema + } + assertResult( + expected.output, + s"Result did not match" + + s" for query #$i\n${expected.sql}") { + output.output + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenScalaReflectionRelationSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenScalaReflectionRelationSuite.scala new file mode 100644 index 000000000000..75bc845b5c8f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenScalaReflectionRelationSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenScalaReflectionRelationSuite + extends ScalaReflectionRelationSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSerializationSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSerializationSuite.scala new file mode 100644 index 000000000000..569de43a75ca --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSerializationSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenSerializationSuite extends SerializationSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala new file mode 100644 index 000000000000..ae9b3901afb9 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSparkSessionExtensionSuite.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.gluten.config.GlutenConfig + +class GlutenSparkSessionExtensionSuite + extends SparkSessionExtensionSuite + with GlutenTestsCommonTrait { + + testGluten("customColumnarOp") { + val extensions = DummyFilterColmnarHelper.create { + extensions => extensions.injectPlannerStrategy(_ => DummyFilterColumnarStrategy) + } + DummyFilterColmnarHelper.withSession(extensions) { + session => + try { + session.range(2).write.format("parquet").mode("overwrite").saveAsTable("a") + def testWithFallbackSettings(scanFallback: Boolean, aggFallback: Boolean): Unit = { + session.sessionState.conf + .setConfString(GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key, scanFallback.toString) + session.sessionState.conf + .setConfString(GlutenConfig.COLUMNAR_HASHAGG_ENABLED.key, aggFallback.toString) + val df = session.sql("SELECT max(id) FROM a") + val newDf = DummyFilterColmnarHelper.dfWithDummyFilterColumnar( + session, + df.queryExecution.optimizedPlan) + val result = newDf.collect + newDf.explain(true) + assert(result(0).getLong(0) == 1) + } + testWithFallbackSettings(true, true) + testWithFallbackSettings(true, false) + testWithFallbackSettings(false, true) + testWithFallbackSettings(false, false) + } finally { + session.sql(s"DROP TABLE IF EXISTS a") + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenStatisticsCollectionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenStatisticsCollectionSuite.scala new file mode 100644 index 000000000000..fab706380488 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenStatisticsCollectionSuite.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.spark.sql.catalyst.plans.logical.ColumnStat +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils +import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneUTC +import org.apache.spark.sql.functions.timestamp_seconds +import org.apache.spark.sql.types.{DataType, DateType, TimestampType} + +import java.util.TimeZone +import java.util.concurrent.TimeUnit + +class GlutenStatisticsCollectionSuite extends StatisticsCollectionSuite with GlutenSQLTestsTrait { + + import testImplicits._ + + testGluten("store and retrieve column stats in different time zones") { + // TODO: bug fix on TableScan. + // val (start, end) = (0, TimeUnit.DAYS.toSeconds(2)) + val (start, end) = (0, 200) + + def checkTimestampStats(t: DataType, srcTimeZone: TimeZone, dstTimeZone: TimeZone)( + checker: ColumnStat => Unit): Unit = { + val table = "time_table" + val column = "T" + val original = TimeZone.getDefault + try { + withTable(table) { + TimeZone.setDefault(srcTimeZone) + spark + .range(start, end) + .select(timestamp_seconds($"id").cast(t).as(column)) + .write + .saveAsTable(table) + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS $column") + + TimeZone.setDefault(dstTimeZone) + val stats = getCatalogTable(table).stats.get.colStats(column).toPlanStat(column, t) + checker(stats) + } + } finally { + TimeZone.setDefault(original) + } + } + + DateTimeTestUtils.outstandingZoneIds.foreach { + zid => + val timeZone = TimeZone.getTimeZone(zid) + checkTimestampStats(DateType, TimeZoneUTC, timeZone) { + stats => + assert(stats.min.get.asInstanceOf[Int] == TimeUnit.SECONDS.toDays(start)) + assert(stats.max.get.asInstanceOf[Int] == TimeUnit.SECONDS.toDays(end - 1)) + } + checkTimestampStats(TimestampType, TimeZoneUTC, timeZone) { + stats => + assert(stats.min.get.asInstanceOf[Long] == TimeUnit.SECONDS.toMicros(start)) + assert(stats.max.get.asInstanceOf[Long] == TimeUnit.SECONDS.toMicros(end - 1)) + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenStringFunctionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenStringFunctionsSuite.scala new file mode 100644 index 000000000000..3d82e214f031 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenStringFunctionsSuite.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.gluten.test.FallbackUtil + +import org.apache.spark.sql.catalyst.expressions.ExpressionEvalHelper +import org.apache.spark.sql.functions._ + +import org.junit.Assert + +class GlutenStringFunctionsSuite + extends StringFunctionsSuite + with GlutenSQLTestsTrait + with ExpressionEvalHelper { + + import testImplicits._ + + testGluten("string split function with no limit and regex pattern") { + val df1 = Seq(("aaAbbAcc4")).toDF("a").select(split($"a", "A")) + checkAnswer(df1, Row(Seq("aa", "bb", "cc4"))) + Assert.assertFalse(FallbackUtil.hasFallback(df1.queryExecution.executedPlan)) + + // scalastyle:off nonascii + val df2 = Seq(("test_gluten单测_")).toDF("a").select(split($"a", "_")) + checkAnswer(df2, Row(Seq("test", "gluten单测", ""))) + // scalastyle:on nonascii + Assert.assertFalse(FallbackUtil.hasFallback(df2.queryExecution.executedPlan)) + } + + testGluten("string split function with limit explicitly set to 0") { + val df1 = Seq(("aaAbbAcc4")).toDF("a").select(split($"a", "A", 0)) + checkAnswer(df1, Row(Seq("aa", "bb", "cc4"))) + Assert.assertFalse(FallbackUtil.hasFallback(df1.queryExecution.executedPlan)) + + // scalastyle:off nonascii + val df2 = Seq(("test_gluten单测_")).toDF("a").select(split($"a", "_", 0)) + checkAnswer(df2, Row(Seq("test", "gluten单测", ""))) + // scalastyle:on nonascii + Assert.assertFalse(FallbackUtil.hasFallback(df2.queryExecution.executedPlan)) + } + + testGluten("string split function with negative limit") { + val df1 = Seq(("aaAbbAcc4")).toDF("a").select(split($"a", "A", -1)) + checkAnswer(df1, Row(Seq("aa", "bb", "cc4"))) + Assert.assertFalse(FallbackUtil.hasFallback(df1.queryExecution.executedPlan)) + + // scalastyle:off nonascii + val df2 = Seq(("test_gluten单测_")).toDF("a").select(split($"a", "_", -2)) + checkAnswer(df2, Row(Seq("test", "gluten单测", ""))) + // scalastyle:on nonascii + Assert.assertFalse(FallbackUtil.hasFallback(df2.queryExecution.executedPlan)) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSubqueryHintPropagationSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSubqueryHintPropagationSuite.scala new file mode 100644 index 000000000000..323c5fbe1477 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSubqueryHintPropagationSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenSubqueryHintPropagationSuite + extends SubqueryHintPropagationSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSubquerySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSubquerySuite.scala new file mode 100644 index 000000000000..05dae6290214 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenSubquerySuite.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import org.apache.gluten.execution.{FileSourceScanExecTransformer, WholeStageTransformer} + +class GlutenSubquerySuite extends SubquerySuite with GlutenSQLTestsTrait { + + // Test Canceled: IntegratedUDFTestUtils.shouldTestPythonUDFs was false + override def testNameBlackList: Seq[String] = Seq( + "SPARK-28441: COUNT bug in WHERE clause (Filter) with PythonUDF", + "SPARK-28441: COUNT bug in SELECT clause (Project) with PythonUDF", + "SPARK-28441: COUNT bug in Aggregate with PythonUDF", + "SPARK-28441: COUNT bug negative examples with PythonUDF", + "SPARK-28441: COUNT bug in nested subquery with PythonUDF", + "SPARK-28441: COUNT bug with nasty predicate expr with PythonUDF", + "SPARK-28441: COUNT bug in HAVING clause (Filter) with PythonUDF", + "SPARK-28441: COUNT bug with attribute ref in subquery input and output with PythonUDF" + ) + + // === Following cases override super class's cases === + // TODO: fix in Spark-4.0 + ignoreGluten("SPARK-26893 Allow pushdown of partition pruning subquery filters to file source") { + withTable("a", "b") { + spark.range(4).selectExpr("id", "id % 2 AS p").write.partitionBy("p").saveAsTable("a") + spark.range(2).write.saveAsTable("b") + + // need to execute the query before we can examine fs.inputRDDs() + val df = sql("SELECT * FROM a WHERE p <= (SELECT MIN(id) FROM b)") + checkAnswer(df, Seq(Row(0, 0), Row(2, 0))) + assert(stripAQEPlan(df.queryExecution.executedPlan).collectFirst { + case t: WholeStageTransformer => t + } match { + case Some(WholeStageTransformer(fs: FileSourceScanExecTransformer, _)) => + fs.dynamicallySelectedPartitions.toPartitionArray + .exists(_.filePath.toString.contains("p=0")) + case _ => false + }) + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenTableOptionsConstantFoldingSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenTableOptionsConstantFoldingSuite.scala new file mode 100644 index 000000000000..35858433b406 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenTableOptionsConstantFoldingSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenTableOptionsConstantFoldingSuite + extends TableOptionsConstantFoldingSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenTypedImperativeAggregateSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenTypedImperativeAggregateSuite.scala new file mode 100644 index 000000000000..cff309cfce2b --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenTypedImperativeAggregateSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenTypedImperativeAggregateSuite + extends TypedImperativeAggregateSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenUnwrapCastInComparisonEndToEndSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenUnwrapCastInComparisonEndToEndSuite.scala new file mode 100644 index 000000000000..2d34e4de9723 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenUnwrapCastInComparisonEndToEndSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenUnwrapCastInComparisonEndToEndSuite + extends UnwrapCastInComparisonEndToEndSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenUrlFunctionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenUrlFunctionsSuite.scala new file mode 100644 index 000000000000..ae173ecd47f9 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenUrlFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenUrlFunctionsSuite extends UrlFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenXPathFunctionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenXPathFunctionsSuite.scala new file mode 100644 index 000000000000..918a96c49e30 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenXPathFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +class GlutenXPathFunctionsSuite extends XPathFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenArithmeticExpressionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenArithmeticExpressionSuite.scala new file mode 100644 index 000000000000..14079037518f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenArithmeticExpressionSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenArithmeticExpressionSuite extends ArithmeticExpressionSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenBitwiseExpressionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenBitwiseExpressionsSuite.scala new file mode 100644 index 000000000000..fd9827ddf502 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenBitwiseExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenBitwiseExpressionsSuite extends BitwiseExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenCastWithAnsiOffSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenCastWithAnsiOffSuite.scala new file mode 100644 index 000000000000..74c1b25ca294 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenCastWithAnsiOffSuite.scala @@ -0,0 +1,298 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{withDefaultTimeZone, ALL_TIMEZONES, UTC, UTC_OPT} +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{fromJavaTimestamp, millisToMicros, TimeZoneUTC} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.util.DebuggableThreadUtils + +import java.sql.{Date, Timestamp} +import java.util.{Calendar, TimeZone} + +class GlutenCastWithAnsiOffSuite extends CastWithAnsiOffSuite with GlutenTestsTrait { + override def beforeAll(): Unit = { + super.beforeAll() + // Need to explicitly set spark.sql.preserveCharVarcharTypeInfo=true for gluten's test + // framework. In Gluten, it overrides the checkEvaluation that invokes Spark's RowEncoder, + // which requires this configuration to be set. + // In Vanilla spark, the checkEvaluation method doesn't invoke RowEncoder. + conf.setConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO, true) + } + + override def cast(v: Any, targetType: DataType, timeZoneId: Option[String] = None): Cast = { + v match { + case lit: Expression => + logDebug(s"Cast from: ${lit.dataType.typeName}, to: ${targetType.typeName}") + Cast(lit, targetType, timeZoneId) + case _ => + val lit = Literal(v) + logDebug(s"Cast from: ${lit.dataType.typeName}, to: ${targetType.typeName}") + Cast(lit, targetType, timeZoneId) + } + } + + // Register UDT For test("SPARK-32828") + UDTRegistration.register(classOf[IExampleBaseType].getName, classOf[ExampleBaseTypeUDT].getName) + UDTRegistration.register(classOf[IExampleSubType].getName, classOf[ExampleSubTypeUDT].getName) + + testGluten("missing cases - from boolean") { + (DataTypeTestUtils.numericTypeWithoutDecimal ++ Set(BooleanType)).foreach { + t => + t match { + case BooleanType => + checkEvaluation(cast(cast(true, BooleanType), t), true) + checkEvaluation(cast(cast(false, BooleanType), t), false) + case _ => + checkEvaluation(cast(cast(true, BooleanType), t), 1) + checkEvaluation(cast(cast(false, BooleanType), t), 0) + } + } + } + + testGluten("missing cases - from byte") { + DataTypeTestUtils.numericTypeWithoutDecimal.foreach { + t => + checkEvaluation(cast(cast(0, ByteType), t), 0) + checkEvaluation(cast(cast(-1, ByteType), t), -1) + checkEvaluation(cast(cast(1, ByteType), t), 1) + } + } + + testGluten("missing cases - from short") { + DataTypeTestUtils.numericTypeWithoutDecimal.foreach { + t => + checkEvaluation(cast(cast(0, ShortType), t), 0) + checkEvaluation(cast(cast(-1, ShortType), t), -1) + checkEvaluation(cast(cast(1, ShortType), t), 1) + } + } + + testGluten("missing cases - date self check") { + val d = Date.valueOf("1970-01-01") + checkEvaluation(cast(d, DateType), d) + } + + testGluten("data type casting") { + val sd = "1970-01-01" + val d = Date.valueOf(sd) + val zts = sd + " 00:00:00" + val sts = sd + " 00:00:02" + val nts = sts + ".1" + val ts = withDefaultTimeZone(UTC)(Timestamp.valueOf(nts)) + + // SystemV timezones are a legacy way of specifying timezones in Unix-like OS. + // It is not supported by Velox. + for (tz <- ALL_TIMEZONES.filterNot(_.getId.contains("SystemV"))) { + withSQLConf( + SQLConf.SESSION_LOCAL_TIMEZONE.key -> tz.getId + ) { + val timeZoneId = Option(tz.getId) + var c = Calendar.getInstance(TimeZoneUTC) + c.set(2015, 2, 8, 2, 30, 0) + checkEvaluation( + cast( + cast(new Timestamp(c.getTimeInMillis), StringType, timeZoneId), + TimestampType, + timeZoneId), + millisToMicros(c.getTimeInMillis)) + c = Calendar.getInstance(TimeZoneUTC) + c.set(2015, 10, 1, 2, 30, 0) + checkEvaluation( + cast( + cast(new Timestamp(c.getTimeInMillis), StringType, timeZoneId), + TimestampType, + timeZoneId), + millisToMicros(c.getTimeInMillis)) + } + } + + checkEvaluation(cast("abdef", StringType), "abdef") + checkEvaluation(cast("12.65", DecimalType.SYSTEM_DEFAULT), Decimal(12.65)) + + checkEvaluation(cast(cast(sd, DateType), StringType), sd) + checkEvaluation(cast(cast(d, StringType), DateType), 0) + + withSQLConf( + SQLConf.SESSION_LOCAL_TIMEZONE.key -> UTC_OPT.get + ) { + checkEvaluation(cast(cast(nts, TimestampType, UTC_OPT), StringType, UTC_OPT), nts) + checkEvaluation( + cast(cast(ts, StringType, UTC_OPT), TimestampType, UTC_OPT), + fromJavaTimestamp(ts)) + + // all convert to string type to check + checkEvaluation( + cast(cast(cast(nts, TimestampType, UTC_OPT), DateType, UTC_OPT), StringType), + sd) + checkEvaluation( + cast(cast(cast(ts, DateType, UTC_OPT), TimestampType, UTC_OPT), StringType, UTC_OPT), + zts) + } + + checkEvaluation(cast(cast("abdef", BinaryType), StringType), "abdef") + + checkEvaluation( + cast( + cast(cast(cast(cast(cast("5", ByteType), ShortType), IntegerType), FloatType), DoubleType), + LongType), + 5.toLong) + + checkEvaluation(cast("23", DoubleType), 23d) + checkEvaluation(cast("23", IntegerType), 23) + checkEvaluation(cast("23", FloatType), 23f) + checkEvaluation(cast("23", DecimalType.USER_DEFAULT), Decimal(23)) + checkEvaluation(cast("23", ByteType), 23.toByte) + checkEvaluation(cast("23", ShortType), 23.toShort) + checkEvaluation(cast(123, IntegerType), 123) + + checkEvaluation(cast(Literal.create(null, IntegerType), ShortType), null) + } + + test("cast from boolean to timestamp") { + val tsTrue = new Timestamp(0) + tsTrue.setNanos(1000) + + val tsFalse = new Timestamp(0) + + checkEvaluation(cast(true, TimestampType), tsTrue) + + checkEvaluation(cast(false, TimestampType), tsFalse) + } + + testGluten("cast string to timestamp") { + DebuggableThreadUtils.parmap( + ALL_TIMEZONES + .filterNot(_.getId.contains("SystemV")) + .filterNot(_.getId.contains("Europe/Kyiv")) + .filterNot(_.getId.contains("America/Ciudad_Juarez")) + .filterNot(_.getId.contains("Antarctica/Vostok")) + .filterNot(_.getId.contains("Pacific/Kanton")) + .filterNot(_.getId.contains("Asia/Tehran")) + .filterNot(_.getId.contains("Iran")), + prefix = "CastSuiteBase-cast-string-to-timestamp", + maxThreads = 1 + ) { + zid => + withSQLConf( + SQLConf.SESSION_LOCAL_TIMEZONE.key -> zid.getId + ) { + def checkCastStringToTimestamp(str: String, expected: Timestamp): Unit = { + checkEvaluation(cast(Literal(str), TimestampType, Option(zid.getId)), expected) + } + + val tz = TimeZone.getTimeZone(zid) + var c = Calendar.getInstance(tz) + c.set(2015, 0, 1, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015", new Timestamp(c.getTimeInMillis)) + c = Calendar.getInstance(tz) + c.set(2015, 2, 1, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03", new Timestamp(c.getTimeInMillis)) + c = Calendar.getInstance(tz) + c.set(2015, 2, 18, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18 ", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(tz) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18 12:03:17", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18T12:03:17", new Timestamp(c.getTimeInMillis)) + + // If the string value includes timezone string, it represents the timestamp string + // in the timezone regardless of the timeZoneId parameter. + c = Calendar.getInstance(TimeZone.getTimeZone(UTC)) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18T12:03:17Z", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18 12:03:17Z", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + // Unsupported timezone format for Velox backend. + // checkCastStringToTimestamp("2015-03-18T12:03:17-1:0", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18T12:03:17-01:00", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18T12:03:17+07:30", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:03")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + // Unsupported timezone format for Velox backend. + // checkCastStringToTimestamp("2015-03-18T12:03:17+7:3", + // new Timestamp(c.getTimeInMillis)) + + // tests for the string including milliseconds. + c = Calendar.getInstance(tz) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + checkCastStringToTimestamp("2015-03-18 12:03:17.123", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18T12:03:17.123", new Timestamp(c.getTimeInMillis)) + + // If the string value includes timezone string, it represents the timestamp string + // in the timezone regardless of the timeZoneId parameter. + c = Calendar.getInstance(TimeZone.getTimeZone(UTC)) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 456) + checkCastStringToTimestamp("2015-03-18T12:03:17.456Z", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18 12:03:17.456Z", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + // Unsupported timezone format for Velox backend. + // checkCastStringToTimestamp("2015-03-18T12:03:17.123-1:0", + // new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp( + "2015-03-18T12:03:17.123-01:00", + new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + checkCastStringToTimestamp( + "2015-03-18T12:03:17.123+07:30", + new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:03")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + // Unsupported timezone format for Velox backend. + // checkCastStringToTimestamp("2015-03-18T12:03:17.123+7:3", + // new Timestamp(c.getTimeInMillis)) + } + } + } + + testGluten("cast decimal to timestamp") { + val tz = TimeZone.getTimeZone(TimeZone.getDefault.getID) + val c = Calendar.getInstance(tz) + c.set(2015, 0, 1, 0, 0, 0) + c.set(Calendar.MILLISECOND, 123) + val d = Decimal(c.getTimeInMillis.toDouble / 1000) + checkEvaluation(cast(d, TimestampType), new Timestamp(c.getTimeInMillis)) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenCollectionExpressionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenCollectionExpressionsSuite.scala new file mode 100644 index 000000000000..ca6ef9df0f48 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenCollectionExpressionsSuite.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait +import org.apache.spark.sql.types._ + +import scala.util.Random + +class GlutenCollectionExpressionsSuite extends CollectionExpressionsSuite with GlutenTestsTrait { + testGluten("Shuffle") { + // Primitive-type elements + val ai0 = Literal.create(Seq(1, 2, 3, 4, 5), ArrayType(IntegerType, containsNull = false)) + val ai1 = Literal.create(Seq(1, 2, 3), ArrayType(IntegerType, containsNull = false)) + val ai2 = Literal.create(Seq(null, 1, null, 3), ArrayType(IntegerType, containsNull = true)) + val ai3 = Literal.create(Seq(2, null, 4, null), ArrayType(IntegerType, containsNull = true)) + val ai4 = Literal.create(Seq(null, null, null), ArrayType(IntegerType, containsNull = true)) + val ai5 = Literal.create(Seq(1), ArrayType(IntegerType, containsNull = false)) + val ai6 = Literal.create(Seq.empty, ArrayType(IntegerType, containsNull = false)) + val ai7 = Literal.create(null, ArrayType(IntegerType, containsNull = true)) + + checkEvaluation(Shuffle(ai0, Some(0)), Array(2, 1, 5, 4, 3)) + checkEvaluation(Shuffle(ai1, Some(0)), Array(2, 1, 3)) + checkEvaluation(Shuffle(ai2, Some(0)), Array(1, null, null, 3)) + checkEvaluation(Shuffle(ai3, Some(0)), Array(null, 2, 4, null)) + checkEvaluation(Shuffle(ai4, Some(0)), Array(null, null, null)) + checkEvaluation(Shuffle(ai5, Some(0)), Array(1)) + checkEvaluation(Shuffle(ai6, Some(0)), Array.empty) + checkEvaluation(Shuffle(ai7, Some(0)), null) + + // Non-primitive-type elements + val as0 = Literal.create(Seq("a", "b", "c", "d"), ArrayType(StringType, containsNull = false)) + val as1 = Literal.create(Seq("a", "b", "c"), ArrayType(StringType, containsNull = false)) + val as2 = Literal.create(Seq(null, "a", null, "c"), ArrayType(StringType, containsNull = true)) + val as3 = Literal.create(Seq("b", null, "d", null), ArrayType(StringType, containsNull = true)) + val as4 = Literal.create(Seq(null, null, null), ArrayType(StringType, containsNull = true)) + val as5 = Literal.create(Seq("a"), ArrayType(StringType, containsNull = false)) + val as6 = Literal.create(Seq.empty, ArrayType(StringType, containsNull = false)) + val as7 = Literal.create(null, ArrayType(StringType, containsNull = true)) + val aa = + Literal.create(Seq(Seq("a", "b"), Seq("c", "d"), Seq("e")), ArrayType(ArrayType(StringType))) + + checkEvaluation(Shuffle(as0, Some(0)), Array("b", "a", "c", "d")) + checkEvaluation(Shuffle(as1, Some(0)), Array("b", "a", "c")) + checkEvaluation(Shuffle(as2, Some(0)), Array("a", null, null, "c")) + checkEvaluation(Shuffle(as3, Some(0)), Array(null, "b", "d", null)) + checkEvaluation(Shuffle(as4, Some(0)), Array(null, null, null)) + checkEvaluation(Shuffle(as5, Some(0)), Array("a")) + checkEvaluation(Shuffle(as6, Some(0)), Array.empty) + checkEvaluation(Shuffle(as7, Some(0)), null) + checkEvaluation(Shuffle(aa, Some(0)), Array(Array("c", "d"), Array("a", "b"), Array("e"))) + + val r = new Random(1234) + val seed1 = Some(r.nextLong()) + assert( + evaluateWithoutCodegen(Shuffle(ai0, seed1)) === + evaluateWithoutCodegen(Shuffle(ai0, seed1))) + assert( + evaluateWithMutableProjection(Shuffle(ai0, seed1)) === + evaluateWithMutableProjection(Shuffle(ai0, seed1))) + assert( + evaluateWithUnsafeProjection(Shuffle(ai0, seed1)) === + evaluateWithUnsafeProjection(Shuffle(ai0, seed1))) + + val seed2 = Some(r.nextLong()) + assert( + evaluateWithoutCodegen(Shuffle(ai0, seed1)) !== + evaluateWithoutCodegen(Shuffle(ai0, seed2))) + assert( + evaluateWithMutableProjection(Shuffle(ai0, seed1)) !== + evaluateWithMutableProjection(Shuffle(ai0, seed2))) + assert( + evaluateWithUnsafeProjection(Shuffle(ai0, seed1)) !== + evaluateWithUnsafeProjection(Shuffle(ai0, seed2))) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenComplexTypeSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenComplexTypeSuite.scala new file mode 100644 index 000000000000..f5f278361e1f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenComplexTypeSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenComplexTypeSuite extends ComplexTypeSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenConditionalExpressionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenConditionalExpressionSuite.scala new file mode 100644 index 000000000000..923f5f87bcc2 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenConditionalExpressionSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenConditionalExpressionSuite extends ConditionalExpressionSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDateExpressionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDateExpressionsSuite.scala new file mode 100644 index 000000000000..30198ad3b17d --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDateExpressionsSuite.scala @@ -0,0 +1,653 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.catalyst.util.DateTimeConstants._ +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, TimeZoneUTC} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +import java.sql.{Date, Timestamp} +import java.text.SimpleDateFormat +import java.time.{LocalDateTime, ZoneId} +import java.util.{Calendar, Locale, TimeZone} +import java.util.concurrent.TimeUnit._ + +class GlutenDateExpressionsSuite extends DateExpressionsSuite with GlutenTestsTrait { + override def testIntegralInput(testFunc: Number => Unit): Unit = { + def checkResult(input: Long): Unit = { + if (input.toByte == input) { + testFunc(input.toByte) + } else if (input.toShort == input) { + testFunc(input.toShort) + } else if (input.toInt == input) { + testFunc(input.toInt) + } else { + testFunc(input) + } + } + + checkResult(0) + checkResult(Byte.MaxValue) + checkResult(Byte.MinValue) + checkResult(Short.MaxValue) + checkResult(Short.MinValue) + // Spark collect causes integer overflow. + // checkResult(Int.MaxValue) + // checkResult(Int.MinValue) + // checkResult(Int.MaxValue.toLong + 100) + // checkResult(Int.MinValue.toLong - 100) + } + + testGluten("TIMESTAMP_MICROS") { + def testIntegralFunc(value: Number): Unit = { + checkEvaluation(MicrosToTimestamp(Literal(value)), value.longValue()) + } + + // test null input + checkEvaluation(MicrosToTimestamp(Literal(null, IntegerType)), null) + + // test integral input + testIntegralInput(testIntegralFunc) + // test max/min input + // Spark collect causes long overflow. + // testIntegralFunc(Long.MaxValue) + // testIntegralFunc(Long.MinValue) + } + + val outstandingTimezonesIds: Seq[String] = Seq( + // Velox doesn't support timezones like UTC. + // "UTC", + PST.getId, + CET.getId, + "Africa/Dakar", + LA.getId, + "Asia/Urumqi", + "Asia/Hong_Kong", + "Europe/Brussels") + val outstandingZoneIds: Seq[ZoneId] = outstandingTimezonesIds.map(getZoneId) + + testGluten("unix_timestamp") { + Seq("legacy", "corrected").foreach { + legacyParserPolicy => + withDefaultTimeZone(UTC) { + for (zid <- outstandingZoneIds) { + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> zid.getId + ) { + val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) + val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" + val sdf2 = new SimpleDateFormat(fmt2, Locale.US) + val fmt3 = "yy-MM-dd" + val sdf3 = new SimpleDateFormat(fmt3, Locale.US) + sdf3.setTimeZone(TimeZoneUTC) + + val timeZoneId = Option(zid.getId) + val tz = TimeZone.getTimeZone(zid) + sdf1.setTimeZone(tz) + sdf2.setTimeZone(tz) + + val date1 = Date.valueOf("2015-07-24") + checkEvaluation( + UnixTimestamp( + Literal(sdf1.format(new Timestamp(0))), + Literal("yyyy-MM-dd HH:mm:ss"), + timeZoneId), + 0L) + checkEvaluation( + UnixTimestamp( + Literal(sdf1.format(new Timestamp(1000000))), + Literal("yyyy-MM-dd HH:mm:ss"), + timeZoneId), + 1000L) + checkEvaluation( + UnixTimestamp( + Literal(new Timestamp(1000000)), + Literal("yyyy-MM-dd HH:mm:ss"), + timeZoneId), + 1000L) + checkEvaluation( + UnixTimestamp( + Literal( + DateTimeUtils.microsToLocalDateTime(DateTimeUtils.millisToMicros(1000000))), + Literal("yyyy-MM-dd HH:mm:ss"), + timeZoneId), + 1000L) + checkEvaluation( + UnixTimestamp(Literal(date1), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + MICROSECONDS.toSeconds( + DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), tz.toZoneId)) + ) + checkEvaluation( + UnixTimestamp( + Literal(sdf2.format(new Timestamp(-1000000))), + Literal(fmt2), + timeZoneId), + -1000L) + checkEvaluation( + UnixTimestamp( + Literal(sdf3.format(Date.valueOf("2015-07-24"))), + Literal(fmt3), + timeZoneId), + MICROSECONDS.toSeconds( + DateTimeUtils.daysToMicros( + DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), + tz.toZoneId)) + ) + checkEvaluation( + UnixTimestamp( + Literal.create(null, DateType), + Literal.create(null, StringType), + timeZoneId), + null) + checkEvaluation( + UnixTimestamp( + Literal.create(null, DateType), + Literal("yyyy-MM-dd HH:mm:ss"), + timeZoneId), + null) + checkEvaluation( + UnixTimestamp(Literal(date1), Literal.create(null, StringType), timeZoneId), + MICROSECONDS.toSeconds( + DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), tz.toZoneId)) + ) + } + } + } + } + // Test escaping of format + GenerateUnsafeProjection.generate( + UnixTimestamp(Literal("2015-07-24"), Literal("\""), UTC_OPT) :: Nil) + } + + testGluten("to_unix_timestamp") { + withDefaultTimeZone(UTC) { + for (zid <- outstandingZoneIds) { + Seq("legacy", "corrected").foreach { + legacyParserPolicy => + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> zid.getId + ) { + val fmt1 = "yyyy-MM-dd HH:mm:ss" + val sdf1 = new SimpleDateFormat(fmt1, Locale.US) + val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" + val sdf2 = new SimpleDateFormat(fmt2, Locale.US) + val fmt3 = "yy-MM-dd" + val sdf3 = new SimpleDateFormat(fmt3, Locale.US) + sdf3.setTimeZone(TimeZoneUTC) + + val timeZoneId = Option(zid.getId) + val tz = TimeZone.getTimeZone(zid) + sdf1.setTimeZone(tz) + sdf2.setTimeZone(tz) + + val date1 = Date.valueOf("2015-07-24") + checkEvaluation( + ToUnixTimestamp(Literal(sdf1.format(new Timestamp(0))), Literal(fmt1), timeZoneId), + 0L) + checkEvaluation( + ToUnixTimestamp( + Literal(sdf1.format(new Timestamp(1000000))), + Literal(fmt1), + timeZoneId), + 1000L) + checkEvaluation( + ToUnixTimestamp(Literal(new Timestamp(1000000)), Literal(fmt1)), + 1000L) + checkEvaluation( + ToUnixTimestamp( + Literal( + DateTimeUtils.microsToLocalDateTime(DateTimeUtils.millisToMicros(1000000))), + Literal(fmt1)), + 1000L) + checkEvaluation( + ToUnixTimestamp(Literal(date1), Literal(fmt1), timeZoneId), + MICROSECONDS.toSeconds( + DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), zid))) + checkEvaluation( + ToUnixTimestamp( + Literal(sdf2.format(new Timestamp(-1000000))), + Literal(fmt2), + timeZoneId), + -1000L) + checkEvaluation( + ToUnixTimestamp( + Literal(sdf3.format(Date.valueOf("2015-07-24"))), + Literal(fmt3), + timeZoneId), + MICROSECONDS.toSeconds(DateTimeUtils + .daysToMicros(DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), zid)) + ) + checkEvaluation( + ToUnixTimestamp( + Literal.create(null, DateType), + Literal.create(null, StringType), + timeZoneId), + null) + checkEvaluation( + ToUnixTimestamp(Literal.create(null, DateType), Literal(fmt1), timeZoneId), + null) + checkEvaluation( + ToUnixTimestamp(Literal(date1), Literal.create(null, StringType), timeZoneId), + MICROSECONDS.toSeconds( + DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), zid)) + ) + + // SPARK-28072 The codegen path for non-literal input should also work + checkEvaluation( + expression = ToUnixTimestamp( + BoundReference(ordinal = 0, dataType = StringType, nullable = true), + BoundReference(ordinal = 1, dataType = StringType, nullable = true), + timeZoneId), + expected = 0L, + inputRow = InternalRow( + UTF8String.fromString(sdf1.format(new Timestamp(0))), + UTF8String.fromString(fmt1)) + ) + } + } + } + } + // Test escaping of format + GenerateUnsafeProjection.generate( + ToUnixTimestamp(Literal("2015-07-24"), Literal("\""), UTC_OPT) :: Nil) + } + + // Modified based on vanilla spark to explicitly set timezone in config. + testGluten("DateFormat") { + val PST_OPT = Option("America/Los_Angeles") + val JST_OPT = Option("Asia/Tokyo") + + Seq("legacy", "corrected").foreach { + legacyParserPolicy => + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> UTC_OPT.get) { + checkEvaluation( + DateFormatClass(Literal.create(null, TimestampType), Literal("y"), UTC_OPT), + null) + checkEvaluation( + DateFormatClass( + Cast(Literal(d), TimestampType, UTC_OPT), + Literal.create(null, StringType), + UTC_OPT), + null) + + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, UTC_OPT), Literal("y"), UTC_OPT), + "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), UTC_OPT), "2013") + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, UTC_OPT), Literal("H"), UTC_OPT), + "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), UTC_OPT), "13") + } + + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> PST_OPT.get) { + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, PST_OPT), Literal("y"), PST_OPT), + "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), PST_OPT), "2013") + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, PST_OPT), Literal("H"), PST_OPT), + "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), PST_OPT), "5") + } + + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> JST_OPT.get) { + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, JST_OPT), Literal("y"), JST_OPT), + "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), JST_OPT), "2013") + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, JST_OPT), Literal("H"), JST_OPT), + "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), JST_OPT), "22") + } + } + } + + testGluten("from_unixtime") { + val outstandingTimezonesIds: Seq[String] = Seq( + // Velox doesn't support timezones like "UTC". + // "UTC", + // Not supported in velox. + // PST.getId, + // CET.getId, + "Africa/Dakar", + LA.getId, + "Asia/Urumqi", + "Asia/Hong_Kong", + "Europe/Brussels" + ) + val outstandingZoneIds: Seq[ZoneId] = outstandingTimezonesIds.map(getZoneId) + Seq("legacy", "corrected").foreach { + legacyParserPolicy => + for (zid <- outstandingZoneIds) { + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> zid.getId) { + val fmt1 = "yyyy-MM-dd HH:mm:ss" + val sdf1 = new SimpleDateFormat(fmt1, Locale.US) + val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" + val sdf2 = new SimpleDateFormat(fmt2, Locale.US) + val timeZoneId = Option(zid.getId) + val tz = TimeZone.getTimeZone(zid) + sdf1.setTimeZone(tz) + sdf2.setTimeZone(tz) + + checkEvaluation( + FromUnixTime(Literal(0L), Literal(fmt1), timeZoneId), + sdf1.format(new Timestamp(0))) + checkEvaluation( + FromUnixTime(Literal(1000L), Literal(fmt1), timeZoneId), + sdf1.format(new Timestamp(1000000))) + checkEvaluation( + FromUnixTime(Literal(-1000L), Literal(fmt2), timeZoneId), + sdf2.format(new Timestamp(-1000000))) + checkEvaluation( + FromUnixTime(Literal(Long.MaxValue), Literal(fmt2), timeZoneId), + sdf2.format(new Timestamp(-1000))) + checkEvaluation( + FromUnixTime( + Literal.create(null, LongType), + Literal.create(null, StringType), + timeZoneId), + null) + checkEvaluation( + FromUnixTime(Literal.create(null, LongType), Literal(fmt1), timeZoneId), + null) + checkEvaluation( + FromUnixTime(Literal(1000L), Literal.create(null, StringType), timeZoneId), + null) + + // SPARK-28072 The codegen path for non-literal input should also work + checkEvaluation( + expression = FromUnixTime( + BoundReference(ordinal = 0, dataType = LongType, nullable = true), + BoundReference(ordinal = 1, dataType = StringType, nullable = true), + timeZoneId), + expected = UTF8String.fromString(sdf1.format(new Timestamp(0))), + inputRow = InternalRow(0L, UTF8String.fromString(fmt1)) + ) + } + } + } + // Test escaping of format + GenerateUnsafeProjection.generate(FromUnixTime(Literal(0L), Literal("\""), UTC_OPT) :: Nil) + } + + testGluten("Hour") { + val outstandingTimezonesIds: Seq[String] = Seq( + // Velox doesn't support timezones like "UTC". + // "UTC", + // Due to known issue: "-08:00/+01:00 not found in timezone database", + // skip check PST, CET timezone here. + // https://github.com/facebookincubator/velox/issues/7804 + // PST.getId, CET.getId, + "Africa/Dakar", + LA.getId, + "Asia/Urumqi", + "Asia/Hong_Kong", + "Europe/Brussels" + ) + withDefaultTimeZone(UTC) { + Seq("legacy", "corrected").foreach { + legacyParserPolicy => + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy + ) { + assert(Hour(Literal.create(null, DateType), UTC_OPT).resolved === false) + assert(Hour(Literal(ts), UTC_OPT).resolved) + Seq(TimestampType, TimestampNTZType).foreach { + dt => + checkEvaluation(Hour(Cast(Literal(d), dt, UTC_OPT), UTC_OPT), 0) + checkEvaluation(Hour(Cast(Literal(date), dt, UTC_OPT), UTC_OPT), 13) + } + checkEvaluation(Hour(Literal(ts), UTC_OPT), 13) + } + + val c = Calendar.getInstance() + outstandingTimezonesIds.foreach { + zid => + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> zid + ) { + val timeZoneId = Option(zid) + c.setTimeZone(TimeZone.getTimeZone(zid)) + (0 to 24 by 5).foreach { + h => + // validate timestamp with local time zone + c.set(2015, 18, 3, h, 29, 59) + checkEvaluation( + Hour(Literal(new Timestamp(c.getTimeInMillis)), timeZoneId), + c.get(Calendar.HOUR_OF_DAY)) + + // validate timestamp without time zone + val localDateTime = LocalDateTime.of(2015, 1, 3, h, 29, 59) + checkEvaluation(Hour(Literal(localDateTime), timeZoneId), h) + } + Seq(TimestampType, TimestampNTZType).foreach { + dt => + checkConsistencyBetweenInterpretedAndCodegen( + (child: Expression) => Hour(child, timeZoneId), + dt) + } + } + } + } + } + } + + testGluten("SPARK-42635: timestampadd near daylight saving transition") { + // In America/Los_Angeles timezone, timestamp value `skippedTime` is 2011-03-13 03:00:00. + // The next second of 2011-03-13 01:59:59 jumps to 2011-03-13 03:00:00. + val skippedTime = 1300010400000000L + // In America/Los_Angeles timezone, both timestamp range `[repeatedTime - MICROS_PER_HOUR, + // repeatedTime)` and `[repeatedTime, repeatedTime + MICROS_PER_HOUR)` map to + // [2011-11-06 01:00:00, 2011-11-06 02:00:00). + // The next second of 2011-11-06 01:59:59 (pre-transition) jumps back to 2011-11-06 01:00:00. + val repeatedTime = 1320570000000000L + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> LA.getId) { + // Adding one day is **not** equivalent to adding _PER_DAY time units, because not every + // day has 24 hours: 2011-03-13 has 23 hours, 2011-11-06 has 25 hours. + + // timestampadd(DAY, 1L, 2011-03-12 03:00:00) = 2011-03-13 03:00:00 + checkEvaluation( + TimestampAdd( + "DAY", + Literal(1L), + Literal(skippedTime - 23 * MICROS_PER_HOUR, TimestampType)), + skippedTime) + // timestampadd(HOUR, 24L, 2011-03-12 03:00:00) = 2011-03-13 04:00:00 + checkEvaluation( + TimestampAdd( + "HOUR", + Literal(24L), + Literal(skippedTime - 23 * MICROS_PER_HOUR, TimestampType)), + skippedTime + MICROS_PER_HOUR) + // timestampadd(HOUR, 23L, 2011-03-12 03:00:00) = 2011-03-13 03:00:00 + checkEvaluation( + TimestampAdd( + "HOUR", + Literal(23L), + Literal(skippedTime - 23 * MICROS_PER_HOUR, TimestampType)), + skippedTime) + // timestampadd(SECOND, SECONDS_PER_DAY, 2011-03-12 03:00:00) = 2011-03-13 04:00:00 + checkEvaluation( + TimestampAdd( + "SECOND", + Literal(SECONDS_PER_DAY), + Literal(skippedTime - 23 * MICROS_PER_HOUR, TimestampType)), + skippedTime + MICROS_PER_HOUR) + // timestampadd(SECOND, SECONDS_PER_DAY, 2011-03-12 03:00:00) = 2011-03-13 03:59:59 + checkEvaluation( + TimestampAdd( + "SECOND", + Literal(SECONDS_PER_DAY - 1), + Literal(skippedTime - 23 * MICROS_PER_HOUR, TimestampType)), + skippedTime + MICROS_PER_HOUR - MICROS_PER_SECOND + ) + + // timestampadd(DAY, 1L, 2011-11-05 02:00:00) = 2011-11-06 02:00:00 + checkEvaluation( + TimestampAdd( + "DAY", + Literal(1L), + Literal(repeatedTime - 24 * MICROS_PER_HOUR, TimestampType)), + repeatedTime + MICROS_PER_HOUR) + // timestampadd(DAY, 1L, 2011-11-05 01:00:00) = 2011-11-06 01:00:00 (pre-transition) + checkEvaluation( + TimestampAdd( + "DAY", + Literal(1L), + Literal(repeatedTime - 25 * MICROS_PER_HOUR, TimestampType)), + repeatedTime - MICROS_PER_HOUR) + // timestampadd(DAY, -1L, 2011-11-07 01:00:00) = 2011-11-06 01:00:00 (post-transition) + // Vanilla spark result is 1320570000000000L, velox result is 1320566400000000L, they + // are all 2011-11-06 01:00:00. + checkEvaluation( + TimestampAdd( + "DAY", + Literal(-1L), + Literal(repeatedTime + 24 * MICROS_PER_HOUR, TimestampType)), + repeatedTime - MICROS_PER_HOUR) + // timestampadd(MONTH, 1L, 2011-10-06 01:00:00) = 2011-11-06 01:00:00 (pre-transition) + checkEvaluation( + TimestampAdd( + "MONTH", + Literal(1L), + Literal(repeatedTime - MICROS_PER_HOUR - 31 * MICROS_PER_DAY, TimestampType)), + repeatedTime - MICROS_PER_HOUR) + // timestampadd(MONTH, -1L, 2011-12-06 01:00:00) = 2011-11-06 01:00:00 (post-transition) + // Vanilla spark result is 1320570000000000L, velox result is 1320566400000000L, they + // are all 2011-11-06 01:00:00. + checkEvaluation( + TimestampAdd( + "MONTH", + Literal(-1L), + Literal(repeatedTime + 30 * MICROS_PER_DAY, TimestampType)), + repeatedTime - MICROS_PER_HOUR) + // timestampadd(HOUR, 23L, 2011-11-05 02:00:00) = 2011-11-06 01:00:00 (pre-transition) + checkEvaluation( + TimestampAdd( + "HOUR", + Literal(23L), + Literal(repeatedTime - 24 * MICROS_PER_HOUR, TimestampType)), + repeatedTime - MICROS_PER_HOUR) + // timestampadd(HOUR, 24L, 2011-11-05 02:00:00) = 2011-11-06 01:00:00 (post-transition) + checkEvaluation( + TimestampAdd( + "HOUR", + Literal(24L), + Literal(repeatedTime - 24 * MICROS_PER_HOUR, TimestampType)), + repeatedTime) + } + } + + testGluten("months_between") { + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) + for (zid <- outstandingZoneIds) { + withSQLConf( + SQLConf.SESSION_LOCAL_TIMEZONE.key -> zid.getId + ) { + val timeZoneId = Option(zid.getId) + sdf.setTimeZone(TimeZone.getTimeZone(zid)) + + checkEvaluation( + MonthsBetween( + Literal(new Timestamp(sdf.parse("1997-02-28 10:30:00").getTime)), + Literal(new Timestamp(sdf.parse("1996-10-30 00:00:00").getTime)), + Literal.TrueLiteral, + timeZoneId = timeZoneId + ), + 3.94959677 + ) + checkEvaluation( + MonthsBetween( + Literal(new Timestamp(sdf.parse("1997-02-28 10:30:00").getTime)), + Literal(new Timestamp(sdf.parse("1996-10-30 00:00:00").getTime)), + Literal.FalseLiteral, + timeZoneId = timeZoneId + ), + 3.9495967741935485 + ) + + Seq(Literal.FalseLiteral, Literal.TrueLiteral).foreach { + roundOff => + checkEvaluation( + MonthsBetween( + Literal(new Timestamp(sdf.parse("2015-01-30 11:52:00").getTime)), + Literal(new Timestamp(sdf.parse("2015-01-30 11:50:00").getTime)), + roundOff, + timeZoneId = timeZoneId + ), + 0.0 + ) + checkEvaluation( + MonthsBetween( + Literal(new Timestamp(sdf.parse("2015-01-31 00:00:00").getTime)), + Literal(new Timestamp(sdf.parse("2015-03-31 22:00:00").getTime)), + roundOff, + timeZoneId = timeZoneId + ), + -2.0 + ) + checkEvaluation( + MonthsBetween( + Literal(new Timestamp(sdf.parse("2015-03-31 22:00:00").getTime)), + Literal(new Timestamp(sdf.parse("2015-02-28 00:00:00").getTime)), + roundOff, + timeZoneId = timeZoneId + ), + 1.0 + ) + } + val t = Literal(Timestamp.valueOf("2015-03-31 22:00:00")) + val tnull = Literal.create(null, TimestampType) + checkEvaluation(MonthsBetween(t, tnull, Literal.TrueLiteral, timeZoneId = timeZoneId), null) + checkEvaluation(MonthsBetween(tnull, t, Literal.TrueLiteral, timeZoneId = timeZoneId), null) + checkEvaluation( + MonthsBetween(tnull, tnull, Literal.TrueLiteral, timeZoneId = timeZoneId), + null) + checkEvaluation( + MonthsBetween(t, t, Literal.create(null, BooleanType), timeZoneId = timeZoneId), + null) + checkConsistencyBetweenInterpretedAndCodegen( + (time1: Expression, time2: Expression, roundOff: Expression) => + MonthsBetween(time1, time2, roundOff, timeZoneId = timeZoneId), + TimestampType, + TimestampType, + BooleanType + ) + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDecimalExpressionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDecimalExpressionSuite.scala new file mode 100644 index 000000000000..8f9054928e40 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDecimalExpressionSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenDecimalExpressionSuite extends DecimalExpressionSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDecimalPrecisionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDecimalPrecisionSuite.scala new file mode 100644 index 000000000000..97e752d7d046 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDecimalPrecisionSuite.scala @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.gluten.expression._ + +import org.apache.spark.sql.GlutenTestsTrait +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EmptyFunctionRegistry, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} +import org.apache.spark.sql.types._ + +class GlutenDecimalPrecisionSuite extends GlutenTestsTrait { + private val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry) + private val analyzer = new Analyzer(catalog) + + private val relation = LocalRelation( + AttributeReference("i", IntegerType)(), + AttributeReference("d1", DecimalType(2, 1))(), + AttributeReference("d2", DecimalType(5, 2))(), + AttributeReference("u", DecimalType.SYSTEM_DEFAULT)(), + AttributeReference("f", FloatType)(), + AttributeReference("b", DoubleType)() + ) + + private val i: Expression = UnresolvedAttribute("i") + private val d1: Expression = UnresolvedAttribute("d1") + private val d2: Expression = UnresolvedAttribute("d2") + private val u: Expression = UnresolvedAttribute("u") + private val f: Expression = UnresolvedAttribute("f") + private val b: Expression = UnresolvedAttribute("b") + + private def checkType(expression: Expression, expectedType: DataType): Unit = { + val plan = analyzer.execute(Project(Seq(Alias(expression, "c")()), relation)) + assert(plan.isInstanceOf[Project]) + val expr = plan.asInstanceOf[Project].projectList.head + assert(expr.dataType == expectedType) + val transformedExpr = + ExpressionConverter.replaceWithExpressionTransformer(expr, plan.inputSet.toSeq) + assert(transformedExpr.dataType == expectedType) + } + + private def stripAlias(expr: Expression): Expression = { + expr match { + case a: Alias => stripAlias(a.child) + case _ => expr + } + } + + private def checkComparison(expression: Expression, expectedType: DataType): Unit = { + val plan = analyzer.execute(Project(Alias(expression, "c")() :: Nil, relation)) + assert(plan.isInstanceOf[Project]) + val expr = stripAlias(plan.asInstanceOf[Project].projectList.head) + val transformedExpr = + ExpressionConverter.replaceWithExpressionTransformer(expr, plan.inputSet.toSeq) + assert(transformedExpr.isInstanceOf[GenericExpressionTransformer]) + val binaryComparison = transformedExpr.asInstanceOf[GenericExpressionTransformer] + assert(binaryComparison.original.isInstanceOf[BinaryComparison]) + assert(binaryComparison.children.size == 2) + assert(binaryComparison.children.forall(_.dataType == expectedType)) + } + + test("basic operations") { + checkType(Add(d1, d2), DecimalType(6, 2)) + checkType(Subtract(d1, d2), DecimalType(6, 2)) + checkType(Multiply(d1, d2), DecimalType(8, 3)) + checkType(Divide(d1, d2), DecimalType(10, 7)) + checkType(Divide(d2, d1), DecimalType(10, 6)) + + checkType(Add(Add(d1, d2), d1), DecimalType(7, 2)) + checkType(Add(Add(d1, d1), d1), DecimalType(4, 1)) + checkType(Add(d1, Add(d1, d1)), DecimalType(4, 1)) + checkType(Add(Add(Add(d1, d2), d1), d2), DecimalType(8, 2)) + checkType(Add(Add(d1, d2), Add(d1, d2)), DecimalType(7, 2)) + checkType(Subtract(Subtract(d2, d1), d1), DecimalType(7, 2)) + checkType(Multiply(Multiply(d1, d1), d2), DecimalType(11, 4)) + checkType(Divide(d2, Add(d1, d1)), DecimalType(10, 6)) + } + + test("Comparison operations") { + checkComparison(EqualTo(i, d1), DecimalType(11, 1)) + checkComparison(EqualNullSafe(d2, d1), DecimalType(5, 2)) + checkComparison(LessThan(i, d1), DecimalType(11, 1)) + checkComparison(LessThanOrEqual(d1, d2), DecimalType(5, 2)) + checkComparison(GreaterThan(d2, u), DecimalType.SYSTEM_DEFAULT) + checkComparison(GreaterThanOrEqual(d1, f), DoubleType) + checkComparison(GreaterThan(d2, d2), DecimalType(5, 2)) + } + + test("bringing in primitive types") { + checkType(Add(d1, i), DecimalType(12, 1)) + checkType(Add(d1, f), DoubleType) + checkType(Add(i, d1), DecimalType(12, 1)) + checkType(Add(f, d1), DoubleType) + checkType(Add(d1, Cast(i, LongType)), DecimalType(22, 1)) + checkType(Add(d1, Cast(i, ShortType)), DecimalType(7, 1)) + checkType(Add(d1, Cast(i, ByteType)), DecimalType(5, 1)) + checkType(Add(d1, Cast(i, DoubleType)), DoubleType) + } + + test("maximum decimals") { + for (expr <- Seq(d1, d2, i, u)) { + checkType(Add(expr, u), DecimalType(38, 17)) + checkType(Subtract(expr, u), DecimalType(38, 17)) + } + + checkType(Multiply(d1, u), DecimalType(38, 16)) + checkType(Multiply(d2, u), DecimalType(38, 14)) + checkType(Multiply(i, u), DecimalType(38, 7)) + checkType(Multiply(u, u), DecimalType(38, 6)) + + checkType(Divide(u, d1), DecimalType(38, 17)) + checkType(Divide(u, d2), DecimalType(38, 16)) + checkType(Divide(u, i), DecimalType(38, 18)) + checkType(Divide(u, u), DecimalType(38, 6)) + + for (expr <- Seq(f, b)) { + checkType(Add(expr, u), DoubleType) + checkType(Subtract(expr, u), DoubleType) + checkType(Multiply(expr, u), DoubleType) + checkType(Divide(expr, u), DoubleType) + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenGeneratorExpressionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenGeneratorExpressionSuite.scala new file mode 100644 index 000000000000..d1867936c149 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenGeneratorExpressionSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenGeneratorExpressionSuite extends GeneratorExpressionSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenHashExpressionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenHashExpressionsSuite.scala new file mode 100644 index 000000000000..4f9d1ffff271 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenHashExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenHashExpressionsSuite extends HashExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenHigherOrderFunctionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenHigherOrderFunctionsSuite.scala new file mode 100644 index 000000000000..6687e707924b --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenHigherOrderFunctionsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenHigherOrderFunctionsSuite extends HigherOrderFunctionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenIntervalExpressionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenIntervalExpressionsSuite.scala new file mode 100644 index 000000000000..2b8aec03d7bd --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenIntervalExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenIntervalExpressionsSuite extends IntervalExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenJsonExpressionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenJsonExpressionsSuite.scala new file mode 100644 index 000000000000..f9d314e508e9 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenJsonExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenJsonExpressionsSuite extends JsonExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenLiteralExpressionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenLiteralExpressionSuite.scala new file mode 100644 index 000000000000..556d185af078 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenLiteralExpressionSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenLiteralExpressionSuite extends LiteralExpressionSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenMathExpressionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenMathExpressionsSuite.scala new file mode 100644 index 000000000000..d49bbd3555eb --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenMathExpressionsSuite.scala @@ -0,0 +1,280 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.types._ + +class GlutenMathExpressionsSuite extends MathExpressionsSuite with GlutenTestsTrait { + testGluten("round/bround/floor/ceil") { + val scales = -6 to 6 + val doublePi: Double = math.Pi + val shortPi: Short = 31415 + val intPi: Int = 314159265 + val longPi: Long = 31415926535897932L + val bdPi: BigDecimal = BigDecimal(31415927L, 7) + val floatPi: Float = 3.1415f + + val doubleResults: Seq[Double] = + Seq(0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 3.0, 3.1, 3.14, 3.142, 3.1416, 3.14159, 3.141593) + + val floatResults: Seq[Float] = + Seq(0.0f, 0.0f, 0.0f, 0.0f, 0.0f, 0.0f, 3.0f, 3.1f, 3.14f, 3.142f, 3.1415f, 3.1415f, 3.1415f) + + val bRoundFloatResults: Seq[Float] = + Seq(0.0f, 0.0f, 0.0f, 0.0f, 0.0f, 0.0f, 3.0f, 3.1f, 3.14f, 3.141f, 3.1415f, 3.1415f, 3.1415f) + + val shortResults: Seq[Short] = Seq[Short](0, 0, 30000, 31000, 31400, 31420) ++ + Seq.fill[Short](7)(31415) + + val intResults: Seq[Int] = + Seq(314000000, 314200000, 314160000, 314159000, 314159300, 314159270) ++ Seq.fill(7)( + 314159265) + + val longResults: Seq[Long] = Seq(31415926536000000L, 31415926535900000L, 31415926535900000L, + 31415926535898000L, 31415926535897900L, 31415926535897930L) ++ + Seq.fill(7)(31415926535897932L) + + val intResultsB: Seq[Int] = + Seq(314000000, 314200000, 314160000, 314159000, 314159300, 314159260) ++ Seq.fill(7)( + 314159265) + + def doubleResultsFloor(i: Int): Decimal = { + val results = Seq(0, 0, 0, 0, 0, 0, 3, 3.1, 3.14, 3.141, 3.1415, 3.14159, 3.141592) + Decimal(results(i)) + } + + def doubleResultsCeil(i: Int): Any = { + val results = + Seq(1000000, 100000, 10000, 1000, 100, 10, 4, 3.2, 3.15, 3.142, 3.1416, 3.1416, 3.141593) + Decimal(results(i)) + } + + def floatResultsFloor(i: Int): Any = { + val results = Seq(0, 0, 0, 0, 0, 0, 3, 3.1, 3.14, 3.141, 3.1415, 3.1415, 3.1415) + Decimal(results(i)) + } + + def floatResultsCeil(i: Int): Any = { + val results = + Seq(1000000, 100000, 10000, 1000, 100, 10, 4, 3.2, 3.15, 3.142, 3.1415, 3.1415, 3.1415) + Decimal(results(i)) + } + + def shortResultsFloor(i: Int): Decimal = { + val results = Seq(0, 0, 30000, 31000, 31400, 31410) ++ Seq.fill(7)(31415) + Decimal(results(i)) + } + + def shortResultsCeil(i: Int): Decimal = { + val results = Seq(1000000, 100000, 40000, 32000, 31500, 31420) ++ Seq.fill(7)(31415) + Decimal(results(i)) + } + + def longResultsFloor(i: Int): Decimal = { + val results = Seq(31415926535000000L, 31415926535800000L, 31415926535890000L, + 31415926535897000L, 31415926535897900L, 31415926535897930L, 31415926535897932L) ++ + Seq.fill(6)(31415926535897932L) + Decimal(results(i)) + } + + def longResultsCeil(i: Int): Decimal = { + val results = Seq(31415926536000000L, 31415926535900000L, 31415926535900000L, + 31415926535898000L, 31415926535898000L, 31415926535897940L) ++ + Seq.fill(7)(31415926535897932L) + Decimal(results(i)) + } + + def intResultsFloor(i: Int): Decimal = { + val results = + Seq(314000000, 314100000, 314150000, 314159000, 314159200, 314159260) ++ Seq.fill(7)( + 314159265) + Decimal(results(i)) + } + + def intResultsCeil(i: Int): Decimal = { + val results = + Seq(315000000, 314200000, 314160000, 314160000, 314159300, 314159270) ++ Seq.fill(7)( + 314159265) + Decimal(results(i)) + } + + scales.zipWithIndex.foreach { + case (scale, i) => + checkEvaluation(Round(doublePi, scale), doubleResults(i), EmptyRow) + checkEvaluation(Round(shortPi, scale), shortResults(i), EmptyRow) + checkEvaluation(Round(intPi, scale), intResults(i), EmptyRow) + checkEvaluation(Round(longPi, scale), longResults(i), EmptyRow) + checkEvaluation(Round(floatPi, scale), floatResults(i), EmptyRow) + checkEvaluation(BRound(doublePi, scale), doubleResults(i), EmptyRow) + checkEvaluation(BRound(shortPi, scale), shortResults(i), EmptyRow) + checkEvaluation(BRound(intPi, scale), intResultsB(i), EmptyRow) + checkEvaluation(BRound(longPi, scale), longResults(i), EmptyRow) + checkEvaluation(BRound(floatPi, scale), bRoundFloatResults(i), EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundFloor(Literal(doublePi), Literal(scale))), + doubleResultsFloor(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundFloor(Literal(shortPi), Literal(scale))), + shortResultsFloor(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundFloor(Literal(intPi), Literal(scale))), + intResultsFloor(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundFloor(Literal(longPi), Literal(scale))), + longResultsFloor(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundFloor(Literal(floatPi), Literal(scale))), + floatResultsFloor(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundCeil(Literal(doublePi), Literal(scale))), + doubleResultsCeil(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundCeil(Literal(shortPi), Literal(scale))), + shortResultsCeil(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundCeil(Literal(intPi), Literal(scale))), + intResultsCeil(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundCeil(Literal(longPi), Literal(scale))), + longResultsCeil(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundCeil(Literal(floatPi), Literal(scale))), + floatResultsCeil(i), + EmptyRow) + } + + val bdResults: Seq[BigDecimal] = Seq( + BigDecimal(3), + BigDecimal("3.1"), + BigDecimal("3.14"), + BigDecimal("3.142"), + BigDecimal("3.1416"), + BigDecimal("3.14159"), + BigDecimal("3.141593"), + BigDecimal("3.1415927") + ) + + val bdResultsFloor: Seq[BigDecimal] = + Seq( + BigDecimal(3), + BigDecimal("3.1"), + BigDecimal("3.14"), + BigDecimal("3.141"), + BigDecimal("3.1415"), + BigDecimal("3.14159"), + BigDecimal("3.141592"), + BigDecimal("3.1415927") + ) + + val bdResultsCeil: Seq[BigDecimal] = Seq( + BigDecimal(4), + BigDecimal("3.2"), + BigDecimal("3.15"), + BigDecimal("3.142"), + BigDecimal("3.1416"), + BigDecimal("3.14160"), + BigDecimal("3.141593"), + BigDecimal("3.1415927") + ) + + (0 to 7).foreach { + i => + checkEvaluation(Round(bdPi, i), bdResults(i), EmptyRow) + checkEvaluation(BRound(bdPi, i), bdResults(i), EmptyRow) + checkEvaluation(RoundFloor(bdPi, i), bdResultsFloor(i), EmptyRow) + checkEvaluation(RoundCeil(bdPi, i), bdResultsCeil(i), EmptyRow) + } + (8 to 10).foreach { + scale => + checkEvaluation(Round(bdPi, scale), bdPi, EmptyRow) + checkEvaluation(BRound(bdPi, scale), bdPi, EmptyRow) + checkEvaluation(RoundFloor(bdPi, scale), bdPi, EmptyRow) + checkEvaluation(RoundCeil(bdPi, scale), bdPi, EmptyRow) + } + + DataTypeTestUtils.numericTypes.foreach { + dataType => + checkEvaluation(Round(Literal.create(null, dataType), Literal(2)), null) + checkEvaluation( + Round(Literal.create(null, dataType), Literal.create(null, IntegerType)), + null) + checkEvaluation(BRound(Literal.create(null, dataType), Literal(2)), null) + checkEvaluation( + BRound(Literal.create(null, dataType), Literal.create(null, IntegerType)), + null) + checkEvaluation( + checkDataTypeAndCast(RoundFloor(Literal.create(null, dataType), Literal(2))), + null) + checkEvaluation( + checkDataTypeAndCast(RoundCeil(Literal.create(null, dataType), Literal(2))), + null) + } + + checkEvaluation(Round(2.5, 0), 3.0) + checkEvaluation(Round(3.5, 0), 4.0) + checkEvaluation(Round(-2.5, 0), -3.0) + checkEvaluation(Round(-3.5, 0), -4.0) + checkEvaluation(Round(-0.35, 1), -0.4) + checkEvaluation(Round(-35, -1), -40) + checkEvaluation(Round(BigDecimal("45.00"), -1), BigDecimal(50)) + checkEvaluation(Round(44, -1), 40) + checkEvaluation(Round(78, 1), 78) + checkEvaluation(BRound(2.5, 0), 2.0) + checkEvaluation(BRound(3.5, 0), 4.0) + checkEvaluation(BRound(-2.5, 0), -2.0) + checkEvaluation(BRound(-3.5, 0), -4.0) + checkEvaluation(BRound(-0.35, 1), -0.4) + checkEvaluation(BRound(-35, -1), -40) + checkEvaluation(Round(1.12345678901234567, 8), 1.12345679) + checkEvaluation(Round(-0.98765432109876543, 5), -0.98765) + checkEvaluation(Round(12345.67890123456789, 6), 12345.678901) + // Enable the test after fixing https://github.com/apache/incubator-gluten/issues/6827 + // checkEvaluation(Round(0.5549999999999999, 2), 0.55) + checkEvaluation(BRound(BigDecimal("45.00"), -1), BigDecimal(40)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(2.5), Literal(0))), Decimal(2)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(3.5), Literal(0))), Decimal(3)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(-2.5), Literal(0))), Decimal(-3L)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(-3.5), Literal(0))), Decimal(-4L)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(-0.35), Literal(1))), Decimal(-0.4)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(-35), Literal(-1))), Decimal(-40)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(-0.1), Literal(0))), Decimal(-1)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(5), Literal(0))), Decimal(5)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(3.1411), Literal(-3))), Decimal(0)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(135.135), Literal(-2))), Decimal(100)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(2.5), Literal(0))), Decimal(3)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(3.5), Literal(0))), Decimal(4L)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(-2.5), Literal(0))), Decimal(-2L)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(-3.5), Literal(0))), Decimal(-3L)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(-0.35), Literal(1))), Decimal(-0.3)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(-35), Literal(-1))), Decimal(-30)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(-0.1), Literal(0))), Decimal(0)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(5), Literal(0))), Decimal(5)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(3.1411), Literal(-3))), Decimal(1000)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(135.135), Literal(-2))), Decimal(200)) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenMiscExpressionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenMiscExpressionsSuite.scala new file mode 100644 index 000000000000..c734a9cfbbdc --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenMiscExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenMiscExpressionsSuite extends MiscExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenNondeterministicSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenNondeterministicSuite.scala new file mode 100644 index 000000000000..34830b368cae --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenNondeterministicSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenNondeterministicSuite extends NondeterministicSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenNullExpressionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenNullExpressionsSuite.scala new file mode 100644 index 000000000000..900fd764d0d9 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenNullExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenNullExpressionsSuite extends NullExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenPredicateSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenPredicateSuite.scala new file mode 100644 index 000000000000..90e93f3593ee --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenPredicateSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenPredicateSuite extends PredicateSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenRandomSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenRandomSuite.scala new file mode 100644 index 000000000000..95d2e71ffe59 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenRandomSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenRandomSuite extends RandomSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenRegexpExpressionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenRegexpExpressionsSuite.scala new file mode 100644 index 000000000000..33cb9a783585 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenRegexpExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenRegexpExpressionsSuite extends RegexpExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenSortOrderExpressionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenSortOrderExpressionsSuite.scala new file mode 100644 index 000000000000..37c630f495f2 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenSortOrderExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenSortOrderExpressionsSuite extends SortOrderExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenStringExpressionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenStringExpressionsSuite.scala new file mode 100644 index 000000000000..cdb67efeccf3 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenStringExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenStringExpressionsSuite extends StringExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenTryCastSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenTryCastSuite.scala new file mode 100644 index 000000000000..fc15ebfeef8b --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenTryCastSuite.scala @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{withDefaultTimeZone, ALL_TIMEZONES, UTC, UTC_OPT} +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{fromJavaTimestamp, millisToMicros, TimeZoneUTC} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.util.DebuggableThreadUtils + +import java.sql.{Date, Timestamp} +import java.util.{Calendar, TimeZone} + +class GlutenTryCastSuite extends TryCastSuite with GlutenTestsTrait { + override def beforeAll(): Unit = { + super.beforeAll() + // Need to explicitly set spark.sql.preserveCharVarcharTypeInfo=true for gluten's test + // framework. In Gluten, it overrides the checkEvaluation that invokes Spark's RowEncoder, + // which requires this configuration to be set. + // In Vanilla spark, the checkEvaluation method doesn't invoke RowEncoder. + conf.setConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO, true) + } + + testGluten("data type casting") { + val sd = "1970-01-01" + val d = Date.valueOf(sd) + val zts = sd + " 00:00:00" + val sts = sd + " 00:00:02" + val nts = sts + ".1" + val ts = withDefaultTimeZone(UTC)(Timestamp.valueOf(nts)) + + // SystemV timezones are a legacy way of specifying timezones in Unix-like OS. + // It is not supported by Velox. + for (tz <- ALL_TIMEZONES.filterNot(_.getId.contains("SystemV"))) { + withSQLConf( + SQLConf.SESSION_LOCAL_TIMEZONE.key -> tz.getId + ) { + val timeZoneId = Option(tz.getId) + var c = Calendar.getInstance(TimeZoneUTC) + c.set(2015, 2, 8, 2, 30, 0) + checkEvaluation( + cast( + cast(new Timestamp(c.getTimeInMillis), StringType, timeZoneId), + TimestampType, + timeZoneId), + millisToMicros(c.getTimeInMillis)) + c = Calendar.getInstance(TimeZoneUTC) + c.set(2015, 10, 1, 2, 30, 0) + checkEvaluation( + cast( + cast(new Timestamp(c.getTimeInMillis), StringType, timeZoneId), + TimestampType, + timeZoneId), + millisToMicros(c.getTimeInMillis)) + } + } + + checkEvaluation(cast("abdef", StringType), "abdef") + checkEvaluation(cast("12.65", DecimalType.SYSTEM_DEFAULT), Decimal(12.65)) + + checkEvaluation(cast(cast(sd, DateType), StringType), sd) + checkEvaluation(cast(cast(d, StringType), DateType), 0) + + withSQLConf( + SQLConf.SESSION_LOCAL_TIMEZONE.key -> UTC_OPT.get + ) { + checkEvaluation(cast(cast(nts, TimestampType, UTC_OPT), StringType, UTC_OPT), nts) + checkEvaluation( + cast(cast(ts, StringType, UTC_OPT), TimestampType, UTC_OPT), + fromJavaTimestamp(ts)) + + // all convert to string type to check + checkEvaluation( + cast(cast(cast(nts, TimestampType, UTC_OPT), DateType, UTC_OPT), StringType), + sd) + checkEvaluation( + cast(cast(cast(ts, DateType, UTC_OPT), TimestampType, UTC_OPT), StringType, UTC_OPT), + zts) + } + + checkEvaluation(cast(cast("abdef", BinaryType), StringType), "abdef") + + checkEvaluation( + cast( + cast(cast(cast(cast(cast("5", ByteType), ShortType), IntegerType), FloatType), DoubleType), + LongType), + 5.toLong) + + checkEvaluation(cast("23", DoubleType), 23d) + checkEvaluation(cast("23", IntegerType), 23) + checkEvaluation(cast("23", FloatType), 23f) + checkEvaluation(cast("23", DecimalType.USER_DEFAULT), Decimal(23)) + checkEvaluation(cast("23", ByteType), 23.toByte) + checkEvaluation(cast("23", ShortType), 23.toShort) + checkEvaluation(cast(123, IntegerType), 123) + + checkEvaluation(cast(Literal.create(null, IntegerType), ShortType), null) + } + + testGluten("cast string to timestamp") { + DebuggableThreadUtils.parmap( + ALL_TIMEZONES + .filterNot(_.getId.contains("SystemV")) + .filterNot(_.getId.contains("Europe/Kyiv")) + .filterNot(_.getId.contains("America/Ciudad_Juarez")) + .filterNot(_.getId.contains("Antarctica/Vostok")) + .filterNot(_.getId.contains("Pacific/Kanton")) + .filterNot(_.getId.contains("Asia/Tehran")) + .filterNot(_.getId.contains("Iran")), + prefix = "CastSuiteBase-cast-string-to-timestamp", + maxThreads = 1 + ) { + zid => + withSQLConf( + SQLConf.SESSION_LOCAL_TIMEZONE.key -> zid.getId + ) { + def checkCastStringToTimestamp(str: String, expected: Timestamp): Unit = { + checkEvaluation(cast(Literal(str), TimestampType, Option(zid.getId)), expected) + } + + val tz = TimeZone.getTimeZone(zid) + var c = Calendar.getInstance(tz) + c.set(2015, 0, 1, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015", new Timestamp(c.getTimeInMillis)) + c = Calendar.getInstance(tz) + c.set(2015, 2, 1, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03", new Timestamp(c.getTimeInMillis)) + c = Calendar.getInstance(tz) + c.set(2015, 2, 18, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18 ", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(tz) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18 12:03:17", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18T12:03:17", new Timestamp(c.getTimeInMillis)) + + // If the string value includes timezone string, it represents the timestamp string + // in the timezone regardless of the timeZoneId parameter. + c = Calendar.getInstance(TimeZone.getTimeZone(UTC)) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18T12:03:17Z", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18 12:03:17Z", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + // Unsupported timezone format for Velox backend. + // checkCastStringToTimestamp("2015-03-18T12:03:17-1:0", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18T12:03:17-01:00", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + checkCastStringToTimestamp("2015-03-18T12:03:17+07:30", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:03")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 0) + // Unsupported timezone format for Velox backend. + // checkCastStringToTimestamp("2015-03-18T12:03:17+7:3", + // new Timestamp(c.getTimeInMillis)) + + // tests for the string including milliseconds. + c = Calendar.getInstance(tz) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + checkCastStringToTimestamp("2015-03-18 12:03:17.123", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18T12:03:17.123", new Timestamp(c.getTimeInMillis)) + + // If the string value includes timezone string, it represents the timestamp string + // in the timezone regardless of the timeZoneId parameter. + c = Calendar.getInstance(TimeZone.getTimeZone(UTC)) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 456) + checkCastStringToTimestamp("2015-03-18T12:03:17.456Z", new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp("2015-03-18 12:03:17.456Z", new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + // Unsupported timezone format for Velox backend. + // checkCastStringToTimestamp("2015-03-18T12:03:17.123-1:0", + // new Timestamp(c.getTimeInMillis)) + checkCastStringToTimestamp( + "2015-03-18T12:03:17.123-01:00", + new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + checkCastStringToTimestamp( + "2015-03-18T12:03:17.123+07:30", + new Timestamp(c.getTimeInMillis)) + + c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:03")) + c.set(2015, 2, 18, 12, 3, 17) + c.set(Calendar.MILLISECOND, 123) + // Unsupported timezone format for Velox backend. + // checkCastStringToTimestamp("2015-03-18T12:03:17.123+7:3", + // new Timestamp(c.getTimeInMillis)) + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenTryEvalSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenTryEvalSuite.scala new file mode 100644 index 000000000000..6af97677e5d8 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenTryEvalSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenTryEvalSuite extends TryEvalSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/GlutenPercentileSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/GlutenPercentileSuite.scala new file mode 100644 index 000000000000..5f89c2810e6e --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/GlutenPercentileSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions.aggregate + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenPercentileSuite extends PercentileSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2DataFrameSessionCatalogSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2DataFrameSessionCatalogSuite.scala new file mode 100644 index 000000000000..4099ea138227 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2DataFrameSessionCatalogSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceV2DataFrameSessionCatalogSuite + extends DataSourceV2DataFrameSessionCatalogSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2DataFrameSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2DataFrameSuite.scala new file mode 100644 index 000000000000..327c930bfb3f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2DataFrameSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceV2DataFrameSuite + extends DataSourceV2DataFrameSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2FunctionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2FunctionSuite.scala new file mode 100644 index 000000000000..10f4d90f54f5 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2FunctionSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceV2FunctionSuite + extends DataSourceV2FunctionSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSessionCatalogSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSessionCatalogSuite.scala new file mode 100644 index 000000000000..7e1a1cdaca9a --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSessionCatalogSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceV2SQLSessionCatalogSuite + extends DataSourceV2SQLSessionCatalogSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSuite.scala new file mode 100644 index 000000000000..28c9f9ff7d30 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSuite.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.spark.sql._ + +class GlutenDataSourceV2SQLSuiteV1Filter + extends DataSourceV2SQLSuiteV1Filter + with GlutenSQLTestsBaseTrait {} + +class GlutenDataSourceV2SQLSuiteV2Filter + extends DataSourceV2SQLSuiteV2Filter + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2Suite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2Suite.scala new file mode 100644 index 000000000000..080487820a16 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2Suite.scala @@ -0,0 +1,168 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.gluten.execution.SortExecTransformer + +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} +import org.apache.spark.sql.execution.ColumnarShuffleExchangeExec +import org.apache.spark.sql.expressions.Window +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.functions.row_number +import org.apache.spark.sql.internal.SQLConf + +import test.org.apache.spark.sql.connector.{JavaOrderAndPartitionAwareDataSource, JavaPartitionAwareDataSource} + +class GlutenDataSourceV2Suite extends DataSourceV2Suite with GlutenSQLTestsBaseTrait { + import testImplicits._ + + testGluten("partitioning reporting") { + import org.apache.spark.sql.functions.{count, sum} + withSQLConf(SQLConf.V2_BUCKETING_ENABLED.key -> "true") { + Seq(classOf[PartitionAwareDataSource], classOf[JavaPartitionAwareDataSource]).foreach { + cls => + withClue(cls.getName) { + val df = spark.read.format(cls.getName).load() + checkAnswer(df, Seq(Row(1, 4), Row(1, 4), Row(3, 6), Row(2, 6), Row(4, 2), Row(4, 2))) + + val groupByColA = df.groupBy(col("i")).agg(sum(col("j"))) + checkAnswer(groupByColA, Seq(Row(1, 8), Row(2, 6), Row(3, 6), Row(4, 4))) + assert(collectFirst(groupByColA.queryExecution.executedPlan) { + case e: ColumnarShuffleExchangeExec => e + }.isEmpty) + + val groupByColAB = df.groupBy(col("i"), col("j")).agg(count("*")) + checkAnswer(groupByColAB, Seq(Row(1, 4, 2), Row(2, 6, 1), Row(3, 6, 1), Row(4, 2, 2))) + assert(collectFirst(groupByColAB.queryExecution.executedPlan) { + case e: ColumnarShuffleExchangeExec => e + }.isEmpty) + + val groupByColB = df.groupBy(col("j")).agg(sum(col("i"))) + checkAnswer(groupByColB, Seq(Row(2, 8), Row(4, 2), Row(6, 5))) + assert(collectFirst(groupByColB.queryExecution.executedPlan) { + case e: ColumnarShuffleExchangeExec => e + }.isDefined) + + val groupByAPlusB = df.groupBy(col("i") + col("j")).agg(count("*")) + checkAnswer(groupByAPlusB, Seq(Row(5, 2), Row(6, 2), Row(8, 1), Row(9, 1))) + assert(collectFirst(groupByAPlusB.queryExecution.executedPlan) { + case e: ColumnarShuffleExchangeExec => e + }.isDefined) + } + } + } + } + + testGluten("ordering and partitioning reporting") { + withSQLConf(SQLConf.V2_BUCKETING_ENABLED.key -> "true") { + Seq( + classOf[OrderAndPartitionAwareDataSource], + classOf[JavaOrderAndPartitionAwareDataSource] + ).foreach { + cls => + withClue(cls.getName) { + // we test report ordering (together with report partitioning) with these + // transformations: + // - groupBy("i").flatMapGroups: + // hash-partitions by "i" and sorts each partition by "i" + // requires partitioning and sort by "i" + // - aggregation function over window partitioned by "i" and ordered by "j": + // hash-partitions by "i" and sorts each partition by "j" + // requires partitioning by "i" and sort by "i" and "j" + Seq( + // with no partitioning and no order, we expect shuffling AND sorting + (None, None, (true, true), (true, true)), + // partitioned by i and no order, we expect NO shuffling BUT sorting + (Some("i"), None, (false, true), (false, true)), + // partitioned by i and in-partition sorted by i, + // we expect NO shuffling AND sorting for groupBy but sorting for window function + (Some("i"), Some("i"), (false, false), (false, true)), + // partitioned by i and in-partition sorted by j, we expect NO shuffling BUT sorting + (Some("i"), Some("j"), (false, true), (false, true)), + // partitioned by i and in-partition sorted by i,j, we expect NO shuffling NOR sorting + (Some("i"), Some("i,j"), (false, false), (false, false)), + // partitioned by j and in-partition sorted by i, we expect shuffling AND sorting + (Some("j"), Some("i"), (true, true), (true, true)), + // partitioned by j and in-partition sorted by i,j, we expect shuffling and sorting + (Some("j"), Some("i,j"), (true, true), (true, true)) + ).foreach { + testParams => + val (partitionKeys, orderKeys, groupByExpects, windowFuncExpects) = testParams + + withClue(f"${partitionKeys.orNull} ${orderKeys.orNull}") { + val df = spark.read + .option("partitionKeys", partitionKeys.orNull) + .option("orderKeys", orderKeys.orNull) + .format(cls.getName) + .load() + checkAnswer( + df, + Seq(Row(1, 4), Row(1, 5), Row(3, 5), Row(2, 6), Row(4, 1), Row(4, 2))) + + // groupBy(i).flatMapGroups + { + val groupBy = df + .groupBy($"i") + .as[Int, (Int, Int)] + .flatMapGroups { + (i: Int, it: Iterator[(Int, Int)]) => Iterator.single((i, it.length)) + } + checkAnswer( + groupBy.toDF(), + Seq(Row(1, 2), Row(2, 1), Row(3, 1), Row(4, 2)) + ) + + val (shuffleExpected, sortExpected) = groupByExpects + assert(collectFirst(groupBy.queryExecution.executedPlan) { + case e: ColumnarShuffleExchangeExec => e + }.isDefined === shuffleExpected) + assert(collectFirst(groupBy.queryExecution.executedPlan) { + case e: SortExecTransformer => e + }.isDefined === sortExpected) + } + + // aggregation function over window partitioned by i and ordered by j + { + val windowPartByColIOrderByColJ = df.withColumn( + "no", + row_number().over(Window.partitionBy(col("i")).orderBy(col("j")))) + checkAnswer( + windowPartByColIOrderByColJ, + Seq( + Row(1, 4, 1), + Row(1, 5, 2), + Row(2, 6, 1), + Row(3, 5, 1), + Row(4, 1, 1), + Row(4, 2, 2) + )) + + val (shuffleExpected, sortExpected) = windowFuncExpects + assert(collectFirst(windowPartByColIOrderByColJ.queryExecution.executedPlan) { + case e: ColumnarShuffleExchangeExec => e + }.isDefined === shuffleExpected) + assert(collectFirst(windowPartByColIOrderByColJ.queryExecution.executedPlan) { + case e: SortExecTransformer => e + }.isDefined === sortExpected) + } + } + } + } + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedDeleteFromTableSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedDeleteFromTableSuite.scala new file mode 100644 index 000000000000..74893c5e51a2 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedDeleteFromTableSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDeltaBasedDeleteFromTableSuite + extends DeltaBasedDeleteFromTableSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedMergeIntoTableSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedMergeIntoTableSuite.scala new file mode 100644 index 000000000000..ca2531186878 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedMergeIntoTableSuite.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDeltaBasedMergeIntoTableSuite + extends DeltaBasedMergeIntoTableSuite + with GlutenSQLTestsBaseTrait + with GlutenMergeIntoTableSuiteBase {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedMergeIntoTableUpdateAsDeleteAndInsertSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedMergeIntoTableUpdateAsDeleteAndInsertSuite.scala new file mode 100644 index 000000000000..458ee89ffdf7 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedMergeIntoTableUpdateAsDeleteAndInsertSuite.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDeltaBasedMergeIntoTableUpdateAsDeleteAndInsertSuite + extends DeltaBasedMergeIntoTableUpdateAsDeleteAndInsertSuite + with GlutenSQLTestsBaseTrait + with GlutenMergeIntoTableSuiteBase {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedUpdateAsDeleteAndInsertTableSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedUpdateAsDeleteAndInsertTableSuite.scala new file mode 100644 index 000000000000..dd4f93140c48 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedUpdateAsDeleteAndInsertTableSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDeltaBasedUpdateAsDeleteAndInsertTableSuite + extends DeltaBasedUpdateAsDeleteAndInsertTableSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedUpdateTableSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedUpdateTableSuite.scala new file mode 100644 index 000000000000..b173c743a99f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenDeltaBasedUpdateTableSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDeltaBasedUpdateTableSuite + extends DeltaBasedUpdateTableSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenFileDataSourceV2FallBackSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenFileDataSourceV2FallBackSuite.scala new file mode 100644 index 000000000000..02198099aa4b --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenFileDataSourceV2FallBackSuite.scala @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.gluten.execution.FileSourceScanExecTransformer + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetDataSourceV2 +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.util.QueryExecutionListener + +import scala.collection.Seq +import scala.collection.mutable.ArrayBuffer + +class GlutenFileDataSourceV2FallBackSuite + extends FileDataSourceV2FallBackSuite + with GlutenSQLTestsBaseTrait { + + testGluten("Fallback Parquet V2 to V1") { + Seq("parquet", classOf[ParquetDataSourceV2].getCanonicalName).foreach { + format => + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> format) { + val commands = ArrayBuffer.empty[(String, LogicalPlan)] + val exceptions = ArrayBuffer.empty[(String, Exception)] + val listener = new QueryExecutionListener { + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = { + exceptions += funcName -> exception + } + + override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { + commands += funcName -> qe.logical + } + } + spark.listenerManager.register(listener) + + try { + withTempPath { + path => + val inputData = spark.range(10) + inputData.write.format(format).save(path.getCanonicalPath) + sparkContext.listenerBus.waitUntilEmpty() + assert(commands.length == 1) + assert(commands.head._1 == "command") + assert(commands.head._2.isInstanceOf[InsertIntoHadoopFsRelationCommand]) + assert( + commands.head._2 + .asInstanceOf[InsertIntoHadoopFsRelationCommand] + .fileFormat + .isInstanceOf[ParquetFileFormat]) + val df = spark.read.format(format).load(path.getCanonicalPath) + checkAnswer(df, inputData.toDF()) + assert( + df.queryExecution.executedPlan.exists( + _.isInstanceOf[FileSourceScanExecTransformer])) + } + } finally { + spark.listenerManager.unregister(listener) + } + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenGroupBasedDeleteFromTableSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenGroupBasedDeleteFromTableSuite.scala new file mode 100644 index 000000000000..5e9abeb07716 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenGroupBasedDeleteFromTableSuite.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenGroupBasedDeleteFromTableSuite + extends GroupBasedDeleteFromTableSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenDeleteFromTableSuite + extends GroupBasedDeleteFromTableSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenGroupBasedMergeIntoTableSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenGroupBasedMergeIntoTableSuite.scala new file mode 100644 index 000000000000..791a0d7a86c8 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenGroupBasedMergeIntoTableSuite.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenGroupBasedMergeIntoTableSuite + extends GroupBasedMergeIntoTableSuite + with GlutenSQLTestsBaseTrait + with GlutenMergeIntoTableSuiteBase {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenKeyGroupedPartitioningSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenKeyGroupedPartitioningSuite.scala new file mode 100644 index 000000000000..00d370b39f32 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenKeyGroupedPartitioningSuite.scala @@ -0,0 +1,1825 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.SortMergeJoinExecTransformer + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait, Row} +import org.apache.spark.sql.connector.catalog.{Column, Identifier, InMemoryTableCatalog} +import org.apache.spark.sql.connector.distributions.Distributions +import org.apache.spark.sql.connector.expressions.Expressions.{bucket, days, identity, years} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.execution.{ColumnarShuffleExchangeExec, SparkPlan} +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.exchange.{ShuffleExchangeExec, ShuffleExchangeLike} +import org.apache.spark.sql.execution.joins.SortMergeJoinExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +import java.util.Collections + +class GlutenKeyGroupedPartitioningSuite + extends KeyGroupedPartitioningSuite + with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = { + // Native SQL configs + super.sparkConf + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") + .set("spark.sql.adaptive.enabled", "false") + .set("spark.sql.shuffle.partitions", "5") + } + + private val emptyProps: java.util.Map[String, String] = { + Collections.emptyMap[String, String] + } + + private val columns: Array[Column] = Array( + Column.create("id", IntegerType), + Column.create("data", StringType), + Column.create("ts", TimestampType)) + + private val columns2: Array[Column] = Array( + Column.create("store_id", IntegerType), + Column.create("dept_id", IntegerType), + Column.create("data", StringType)) + + private def createTable( + table: String, + columns: Array[Column], + partitions: Array[Transform], + catalog: InMemoryTableCatalog = catalog): Unit = { + catalog.createTable( + Identifier.of(Array("ns"), table), + columns, + partitions, + emptyProps, + Distributions.unspecified(), + Array.empty, + None, + None, + numRowsPerSplit = 1) + } + + private def collectColumnarShuffleExchangeExec( + plan: SparkPlan): Seq[ColumnarShuffleExchangeExec] = { + // here we skip collecting shuffle operators that are not associated with SMJ + collect(plan) { + case s: SortMergeJoinExecTransformer => s + case s: SortMergeJoinExec => s + }.flatMap(smj => collect(smj) { case s: ColumnarShuffleExchangeExec => s }) + } + + private def collectShuffles(plan: SparkPlan): Seq[ShuffleExchangeLike] = { + // here we skip collecting shuffle operators that are not associated with SMJ + collect(plan) { + case s: SortMergeJoinExec => s + case s: SortMergeJoinExecTransformer => s + }.flatMap( + smj => + collect(smj) { + case s: ShuffleExchangeExec => s + case s: ColumnarShuffleExchangeExec => s + }) + } + + private def collectAllShuffles(plan: SparkPlan): Seq[ColumnarShuffleExchangeExec] = { + collect(plan) { case s: ColumnarShuffleExchangeExec => s } + } + + private def collectScans(plan: SparkPlan): Seq[BatchScanExec] = { + collect(plan) { case s: BatchScanExec => s } + } + + private def selectWithMergeJoinHint(t1: String, t2: String): String = { + s"SELECT /*+ MERGE($t1, $t2) */ " + } + + private def createJoinTestDF( + keys: Seq[(String, String)], + extraColumns: Seq[String] = Nil, + joinType: String = ""): DataFrame = { + val extraColList = if (extraColumns.isEmpty) "" else extraColumns.mkString(", ", ", ", "") + sql(s""" + |${selectWithMergeJoinHint("i", "p")} + |id, name, i.price as purchase_price, p.price as sale_price $extraColList + |FROM testcat.ns.$items i $joinType JOIN testcat.ns.$purchases p + |ON ${keys.map(k => s"i.${k._1} = p.${k._2}").mkString(" AND ")} + |ORDER BY id, purchase_price, sale_price $extraColList + |""".stripMargin) + } + + private val customers: String = "customers" + private val customersColumns: Array[Column] = Array( + Column.create("customer_name", StringType), + Column.create("customer_age", IntegerType), + Column.create("customer_id", LongType)) + + private val orders: String = "orders" + private val ordersColumns: Array[Column] = + Array(Column.create("order_amount", DoubleType), Column.create("customer_id", LongType)) + + private def testWithCustomersAndOrders( + customers_partitions: Array[Transform], + orders_partitions: Array[Transform], + expectedNumOfShuffleExecs: Int): Unit = { + createTable(customers, customersColumns, customers_partitions) + sql( + s"INSERT INTO testcat.ns.$customers VALUES " + + s"('aaa', 10, 1), ('bbb', 20, 2), ('ccc', 30, 3)") + + createTable(orders, ordersColumns, orders_partitions) + sql( + s"INSERT INTO testcat.ns.$orders VALUES " + + s"(100.0, 1), (200.0, 1), (150.0, 2), (250.0, 2), (350.0, 2), (400.50, 3)") + + val df = sql( + "SELECT customer_name, customer_age, order_amount " + + s"FROM testcat.ns.$customers c JOIN testcat.ns.$orders o " + + "ON c.customer_id = o.customer_id ORDER BY c.customer_id, order_amount") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + assert(shuffles.length == expectedNumOfShuffleExecs) + + checkAnswer( + df, + Seq( + Row("aaa", 10, 100.0), + Row("aaa", 10, 200.0), + Row("bbb", 20, 150.0), + Row("bbb", 20, 250.0), + Row("bbb", 20, 350.0), + Row("ccc", 30, 400.50))) + } + + testGluten("partitioned join: only one side reports partitioning") { + val customers_partitions = Array(bucket(4, "customer_id")) + val orders_partitions = Array(bucket(2, "customer_id")) + + testWithCustomersAndOrders(customers_partitions, orders_partitions, 2) + } + testGluten("partitioned join: exact distribution (same number of buckets) from both sides") { + val customers_partitions = Array(bucket(4, "customer_id")) + val orders_partitions = Array(bucket(4, "customer_id")) + + testWithCustomersAndOrders(customers_partitions, orders_partitions, 0) + } + + private val items: String = "items" + private val itemsColumns: Array[Column] = Array( + Column.create("id", LongType), + Column.create("name", StringType), + Column.create("price", FloatType), + Column.create("arrive_time", TimestampType)) + private val purchases: String = "purchases" + private val purchasesColumns: Array[Column] = Array( + Column.create("item_id", LongType), + Column.create("price", FloatType), + Column.create("time", TimestampType)) + + testGluten( + "SPARK-41413: partitioned join: partition values" + + " from one side are subset of those from the other side") { + val items_partitions = Array(bucket(4, "id")) + createTable(items, itemsColumns, items_partitions) + + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(3, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + "(4, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(bucket(4, "item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + "(1, 42.0, cast('2020-01-01' as timestamp)), " + + "(3, 19.5, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + withSQLConf(SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not add shuffle when partition values mismatch") + } else { + assert( + shuffles.nonEmpty, + "should add shuffle when partition values mismatch, and " + + "pushing down partition values is not enabled") + } + + checkAnswer(df, Seq(Row(1, "aa", 40.0, 42.0), Row(3, "bb", 10.0, 19.5))) + } + } + } + + testGluten("SPARK-41413: partitioned join: partition values from both sides overlaps") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + "(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + "(1, 42.0, cast('2020-01-01' as timestamp)), " + + "(2, 19.5, cast('2020-02-01' as timestamp)), " + + "(4, 30.0, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + withSQLConf(SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not add shuffle when partition values mismatch") + } else { + assert( + shuffles.nonEmpty, + "should add shuffle when partition values mismatch, and " + + "pushing down partition values is not enabled") + } + + checkAnswer(df, Seq(Row(1, "aa", 40.0, 42.0), Row(2, "bb", 10.0, 19.5))) + } + } + } + + testGluten("SPARK-41413: partitioned join: non-overlapping partition values from both sides") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + "(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + "(4, 42.0, cast('2020-01-01' as timestamp)), " + + "(5, 19.5, cast('2020-02-01' as timestamp)), " + + "(6, 30.0, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + withSQLConf(SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not add shuffle when partition values mismatch") + } else { + assert( + shuffles.nonEmpty, + "should add shuffle when partition values mismatch, and " + + "pushing down partition values is not enabled") + } + + checkAnswer(df, Seq.empty) + } + } + } + + testGluten( + "SPARK-42038: partially clustered:" + + " with same partition keys and one side fully clustered") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 45.0, cast('2020-01-01' as timestamp)), " + + s"(1, 50.0, cast('2020-01-02' as timestamp)), " + + s"(2, 15.0, cast('2020-01-02' as timestamp)), " + + s"(2, 20.0, cast('2020-01-03' as timestamp)), " + + s"(3, 20.0, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + Seq(("true", 5), ("false", 3)).foreach { + case (enable, expected) => + withSQLConf( + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> enable) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + assert(shuffles.isEmpty, "should not contain any shuffle") + if (pushDownValues) { + val scans = collectScans(df.queryExecution.executedPlan) + assert(scans.forall(_.inputRDD.partitions.length == expected)) + } + checkAnswer( + df, + Seq( + Row(1, "aa", 40.0, 45.0), + Row(1, "aa", 40.0, 50.0), + Row(2, "bb", 10.0, 15.0), + Row(2, "bb", 10.0, 20.0), + Row(3, "cc", 15.5, 20.0))) + } + } + } + } + + testGluten( + "SPARK-42038: partially clustered:" + + " with same partition keys and both sides partially clustered") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 45.0, cast('2020-01-01' as timestamp)), " + + s"(1, 50.0, cast('2020-01-02' as timestamp)), " + + s"(1, 55.0, cast('2020-01-02' as timestamp)), " + + s"(2, 15.0, cast('2020-01-02' as timestamp)), " + + s"(2, 20.0, cast('2020-01-03' as timestamp)), " + + s"(2, 22.0, cast('2020-01-03' as timestamp)), " + + s"(3, 20.0, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + Seq(("true", 7), ("false", 3)).foreach { + case (enable, expected) => + withSQLConf( + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> enable) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + assert(shuffles.isEmpty, "should not contain any shuffle") + if (pushDownValues) { + val scans = collectScans(df.queryExecution.executedPlan) + assert(scans.forall(_.inputRDD.partitions.length == expected)) + } + checkAnswer( + df, + Seq( + Row(1, "aa", 40.0, 45.0), + Row(1, "aa", 40.0, 50.0), + Row(1, "aa", 40.0, 55.0), + Row(1, "aa", 41.0, 45.0), + Row(1, "aa", 41.0, 50.0), + Row(1, "aa", 41.0, 55.0), + Row(2, "bb", 10.0, 15.0), + Row(2, "bb", 10.0, 20.0), + Row(2, "bb", 10.0, 22.0), + Row(3, "cc", 15.5, 20.0) + ) + ) + } + } + } + } + + testGluten( + "SPARK-42038: partially clustered: with different" + + " partition keys and both sides partially clustered") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp)), " + + s"(4, 'dd', 18.0, cast('2023-01-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 45.0, cast('2020-01-01' as timestamp)), " + + s"(1, 50.0, cast('2020-01-02' as timestamp)), " + + s"(1, 55.0, cast('2020-01-02' as timestamp)), " + + s"(2, 15.0, cast('2020-01-02' as timestamp)), " + + s"(2, 20.0, cast('2020-01-03' as timestamp)), " + + s"(2, 25.0, cast('2020-01-03' as timestamp)), " + + s"(2, 30.0, cast('2020-01-03' as timestamp)), " + + s"(3, 20.0, cast('2020-02-01' as timestamp)), " + + s"(5, 30.0, cast('2023-01-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + Seq(("true", 10), ("false", 5)).foreach { + case (enable, expected) => + withSQLConf( + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> enable) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not contain any shuffle") + val scans = collectScans(df.queryExecution.executedPlan) + assert(scans.forall(_.inputRDD.partitions.length == expected)) + } else { + assert( + shuffles.nonEmpty, + "should contain shuffle when not pushing down partition values") + } + checkAnswer( + df, + Seq( + Row(1, "aa", 40.0, 45.0), + Row(1, "aa", 40.0, 50.0), + Row(1, "aa", 40.0, 55.0), + Row(1, "aa", 41.0, 45.0), + Row(1, "aa", 41.0, 50.0), + Row(1, "aa", 41.0, 55.0), + Row(2, "bb", 10.0, 15.0), + Row(2, "bb", 10.0, 20.0), + Row(2, "bb", 10.0, 25.0), + Row(2, "bb", 10.0, 30.0), + Row(3, "cc", 15.5, 20.0) + ) + ) + } + } + } + } + + testGluten( + "SPARK-42038: partially clustered: with different" + + " partition keys and missing keys on left-hand side") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp)), " + + s"(4, 'dd', 18.0, cast('2023-01-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 45.0, cast('2020-01-01' as timestamp)), " + + s"(1, 50.0, cast('2020-01-02' as timestamp)), " + + s"(2, 15.0, cast('2020-01-02' as timestamp)), " + + s"(2, 20.0, cast('2020-01-03' as timestamp)), " + + s"(2, 25.0, cast('2020-01-03' as timestamp)), " + + s"(2, 30.0, cast('2020-01-03' as timestamp)), " + + s"(3, 20.0, cast('2020-02-01' as timestamp)), " + + s"(5, 30.0, cast('2023-01-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + Seq(("true", 9), ("false", 5)).foreach { + case (enable, expected) => + withSQLConf( + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> enable) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not contain any shuffle") + val scans = collectScans(df.queryExecution.executedPlan) + assert(scans.forall(_.inputRDD.partitions.length == expected)) + } else { + assert( + shuffles.nonEmpty, + "should contain shuffle when not pushing down partition values") + } + checkAnswer( + df, + Seq( + Row(1, "aa", 40.0, 45.0), + Row(1, "aa", 40.0, 50.0), + Row(1, "aa", 41.0, 45.0), + Row(1, "aa", 41.0, 50.0), + Row(3, "cc", 15.5, 20.0))) + } + } + } + } + + testGluten( + "SPARK-42038: partially clustered:" + + " with different partition keys and missing keys on right-hand side") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(2, 15.0, cast('2020-01-02' as timestamp)), " + + s"(2, 20.0, cast('2020-01-03' as timestamp)), " + + s"(3, 20.0, cast('2020-02-01' as timestamp)), " + + s"(4, 25.0, cast('2020-02-01' as timestamp)), " + + s"(5, 30.0, cast('2023-01-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + Seq(("true", 6), ("false", 5)).foreach { + case (enable, expected) => + withSQLConf( + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> enable) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not contain any shuffle") + val scans = collectScans(df.queryExecution.executedPlan) + assert(scans.forall(_.inputRDD.partitions.length == expected)) + } else { + assert( + shuffles.nonEmpty, + "should contain shuffle when not pushing down partition values") + } + checkAnswer( + df, + Seq(Row(2, "bb", 10.0, 15.0), Row(2, "bb", 10.0, 20.0), Row(3, "cc", 15.5, 20.0))) + } + } + } + } + + testGluten("SPARK-42038: partially clustered: left outer join") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(2, 'bb', 15.0, cast('2020-01-02' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(2, 20.0, cast('2020-01-01' as timestamp)), " + + s"(3, 20.0, cast('2020-02-01' as timestamp)), " + + s"(4, 25.0, cast('2020-02-01' as timestamp)), " + + s"(5, 30.0, cast('2023-01-01' as timestamp))") + + // In a left-outer join, and when the left side has larger stats, partially clustered + // distribution should kick in and pick the right hand side to replicate partitions. + Seq(true, false).foreach { + pushDownValues => + Seq(("true", 7), ("false", 5)).foreach { + case (enable, expected) => + withSQLConf( + SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION.key -> false.toString, + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> enable + ) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i LEFT JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id AND i.arrive_time = p.time " + + "ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not contain any shuffle") + val scans = collectScans(df.queryExecution.executedPlan) + assert( + scans.forall(_.inputRDD.partitions.length == expected), + s"Expected $expected but got ${scans.head.inputRDD.partitions.length}") + } else { + assert( + shuffles.nonEmpty, + "should contain shuffle when not pushing down partition values") + } + checkAnswer( + df, + Seq( + Row(1, "aa", 40.0, null), + Row(1, "aa", 41.0, null), + Row(2, "bb", 10.0, 20.0), + Row(2, "bb", 15.0, null), + Row(3, "cc", 15.5, 20.0))) + } + } + } + } + + testGluten("SPARK-42038: partially clustered: right outer join") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 45.0, cast('2020-01-01' as timestamp)), " + + s"(2, 15.0, cast('2020-01-01' as timestamp)), " + + s"(2, 20.0, cast('2020-01-01' as timestamp)), " + + s"(3, 20.0, cast('2020-02-01' as timestamp)), " + + s"(4, 25.0, cast('2020-02-01' as timestamp)), " + + s"(5, 30.0, cast('2023-01-01' as timestamp))") + + // The left-hand side is picked as the side to replicate partitions based on stats, but since + // this is right outer join, partially clustered distribution won't kick in, and Spark should + // only push down partition values on both side. + Seq(true, false).foreach { + pushDownValues => + Seq(("true", 5), ("false", 5)).foreach { + case (enable, expected) => + withSQLConf( + SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION.key -> false.toString, + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> enable + ) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i RIGHT JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id AND i.arrive_time = p.time " + + "ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not contain any shuffle") + val scans = collectScans(df.queryExecution.executedPlan) + assert(scans.map(_.inputRDD.partitions.length).toSet.size == 1) + assert( + scans.forall(_.inputRDD.partitions.length == expected), + s"Expected $expected but got ${scans.head.inputRDD.partitions.length}") + } else { + assert( + shuffles.nonEmpty, + "should contain shuffle when not pushing down partition values") + } + checkAnswer( + df, + Seq( + Row(null, null, null, 25.0), + Row(null, null, null, 30.0), + Row(1, "aa", 40.0, 45.0), + Row(2, "bb", 10.0, 15.0), + Row(2, "bb", 10.0, 20.0), + Row(3, "cc", 15.5, 20.0))) + } + } + } + } + + testGluten("SPARK-42038: partially clustered: full outer join is not applicable") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-01-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 45.0, cast('2020-01-01' as timestamp)), " + + s"(2, 15.0, cast('2020-01-01' as timestamp)), " + + s"(2, 20.0, cast('2020-01-02' as timestamp)), " + + s"(3, 20.0, cast('2020-01-01' as timestamp)), " + + s"(4, 25.0, cast('2020-01-01' as timestamp)), " + + s"(5, 30.0, cast('2023-01-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + Seq(("true", 5), ("false", 5)).foreach { + case (enable, expected) => + withSQLConf( + SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION.key -> false.toString, + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> enable + ) { + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i FULL OUTER JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id AND i.arrive_time = p.time " + + "ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not contain any shuffle") + val scans = collectScans(df.queryExecution.executedPlan) + assert(scans.map(_.inputRDD.partitions.length).toSet.size == 1) + assert( + scans.forall(_.inputRDD.partitions.length == expected), + s"Expected $expected but got ${scans.head.inputRDD.partitions.length}") + } else { + assert( + shuffles.nonEmpty, + "should contain shuffle when not pushing down partition values") + } + checkAnswer( + df, + Seq( + Row(null, null, null, 20.0), + Row(null, null, null, 25.0), + Row(null, null, null, 30.0), + Row(1, "aa", 40.0, 45.0), + Row(1, "aa", 41.0, null), + Row(2, "bb", 10.0, 15.0), + Row(3, "cc", 15.5, 20.0) + ) + ) + } + } + } + } + + testGluten("SPARK-44641: duplicated records when SPJ is not triggered") { + val items_partitions = Array(bucket(8, "id")) + createTable(items, itemsColumns, items_partitions) + sql(s""" + INSERT INTO testcat.ns.$items VALUES + (1, 'aa', 40.0, cast('2020-01-01' as timestamp)), + (1, 'aa', 41.0, cast('2020-01-15' as timestamp)), + (2, 'bb', 10.0, cast('2020-01-01' as timestamp)), + (2, 'bb', 10.5, cast('2020-01-01' as timestamp)), + (3, 'cc', 15.5, cast('2020-02-01' as timestamp))""") + + val purchases_partitions = Array(bucket(8, "item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql(s"""INSERT INTO testcat.ns.$purchases VALUES + (1, 42.0, cast('2020-01-01' as timestamp)), + (1, 44.0, cast('2020-01-15' as timestamp)), + (1, 45.0, cast('2020-01-15' as timestamp)), + (2, 11.0, cast('2020-01-01' as timestamp)), + (3, 19.5, cast('2020-02-01' as timestamp))""") + + Seq(true, false).foreach { + pushDownValues => + Seq(true, false).foreach { + partiallyClusteredEnabled => + withSQLConf( + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> + partiallyClusteredEnabled.toString + ) { + + // join keys are not the same as the partition keys, therefore SPJ is not triggered. + val df = sql(s""" + SELECT id, name, i.price as purchase_price, p.item_id, p.price as sale_price + FROM testcat.ns.$items i JOIN testcat.ns.$purchases p + ON i.arrive_time = p.time ORDER BY id, purchase_price, p.item_id, sale_price + """) + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + assert(shuffles.nonEmpty, "shuffle should exist when SPJ is not used") + + checkAnswer( + df, + Seq( + Row(1, "aa", 40.0, 1, 42.0), + Row(1, "aa", 40.0, 2, 11.0), + Row(1, "aa", 41.0, 1, 44.0), + Row(1, "aa", 41.0, 1, 45.0), + Row(2, "bb", 10.0, 1, 42.0), + Row(2, "bb", 10.0, 2, 11.0), + Row(2, "bb", 10.5, 1, 42.0), + Row(2, "bb", 10.5, 2, 11.0), + Row(3, "cc", 15.5, 3, 19.5) + ) + ) + } + } + } + } + + testGluten("partitioned join: join with two partition keys and matching & sorted partitions") { + val items_partitions = Array(bucket(8, "id"), days("arrive_time")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(2, 'bb', 10.5, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(bucket(8, "item_id"), days("time")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 42.0, cast('2020-01-01' as timestamp)), " + + s"(1, 44.0, cast('2020-01-15' as timestamp)), " + + s"(1, 45.0, cast('2020-01-15' as timestamp)), " + + s"(2, 11.0, cast('2020-01-01' as timestamp)), " + + s"(3, 19.5, cast('2020-02-01' as timestamp))") + + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id AND i.arrive_time = p.time ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + assert(shuffles.isEmpty, "should not add shuffle for both sides of the join") + checkAnswer( + df, + Seq( + Row(1, "aa", 40.0, 42.0), + Row(1, "aa", 41.0, 44.0), + Row(1, "aa", 41.0, 45.0), + Row(2, "bb", 10.0, 11.0), + Row(2, "bb", 10.5, 11.0), + Row(3, "cc", 15.5, 19.5))) + } + + testGluten("partitioned join: join with two partition keys and unsorted partitions") { + val items_partitions = Array(bucket(8, "id"), days("arrive_time")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp)), " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(2, 'bb', 10.5, cast('2020-01-01' as timestamp))") + + val purchases_partitions = Array(bucket(8, "item_id"), days("time")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(2, 11.0, cast('2020-01-01' as timestamp)), " + + s"(1, 42.0, cast('2020-01-01' as timestamp)), " + + s"(1, 44.0, cast('2020-01-15' as timestamp)), " + + s"(1, 45.0, cast('2020-01-15' as timestamp)), " + + s"(3, 19.5, cast('2020-02-01' as timestamp))") + + val df = sql( + "SELECT id, name, i.price as purchase_price, p.price as sale_price " + + s"FROM testcat.ns.$items i JOIN testcat.ns.$purchases p " + + "ON i.id = p.item_id AND i.arrive_time = p.time ORDER BY id, purchase_price, sale_price") + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + assert(shuffles.isEmpty, "should not add shuffle for both sides of the join") + checkAnswer( + df, + Seq( + Row(1, "aa", 40.0, 42.0), + Row(1, "aa", 41.0, 44.0), + Row(1, "aa", 41.0, 45.0), + Row(2, "bb", 10.0, 11.0), + Row(2, "bb", 10.5, 11.0), + Row(3, "cc", 15.5, 19.5))) + } + + testGluten("partitioned join: join with two partition keys and different # of partition keys") { + val items_partitions = Array(bucket(8, "id"), days("arrive_time")) + createTable(items, itemsColumns, items_partitions) + + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(bucket(8, "item_id"), days("time")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 42.0, cast('2020-01-01' as timestamp)), " + + s"(2, 11.0, cast('2020-01-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + withSQLConf(SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString) { + val df = createJoinTestDF(Seq("id" -> "item_id", "arrive_time" -> "time")) + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (pushDownValues) { + assert(shuffles.isEmpty, "should not add shuffle when partition values mismatch") + } else { + assert( + shuffles.nonEmpty, + "should add shuffle when partition values mismatch, and " + + "pushing down partition values is not enabled") + } + + checkAnswer(df, Seq(Row(1, "aa", 40.0, 42.0), Row(2, "bb", 10.0, 11.0))) + } + } + } + + testGluten("data source partitioning + dynamic partition filtering") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "10" + ) { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(2, 'bb', 10.5, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(identity("item_id")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 42.0, cast('2020-01-01' as timestamp)), " + + s"(1, 44.0, cast('2020-01-15' as timestamp)), " + + s"(1, 45.0, cast('2020-01-15' as timestamp)), " + + s"(2, 11.0, cast('2020-01-01' as timestamp)), " + + s"(3, 19.5, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + withSQLConf( + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString) { + // number of unique partitions changed after dynamic filtering - the gap + // should be filled with empty partitions and the job should still succeed + var df = sql( + s"SELECT sum(p.price) from testcat.ns.$items i, testcat.ns.$purchases p " + + "WHERE i.id = p.item_id AND i.price > 40.0") + checkAnswer(df, Seq(Row(131))) + + // dynamic filtering doesn't change partitioning so storage-partitioned join should kick + // in + df = sql( + s"SELECT sum(p.price) from testcat.ns.$items i, testcat.ns.$purchases p " + + "WHERE i.id = p.item_id AND i.price >= 10.0") + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + assert(shuffles.isEmpty, "should not add shuffle for both sides of the join") + checkAnswer(df, Seq(Row(303.5))) + } + } + } + } + + testGluten( + "SPARK-41471: shuffle one side: only one side reports partitioning with two identity") { + val items_partitions = Array(identity("id"), identity("arrive_time")) + createTable(items, itemsColumns, items_partitions) + + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(3, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + "(4, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + createTable(purchases, purchasesColumns, Array.empty) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + "(1, 42.0, cast('2020-01-01' as timestamp)), " + + "(3, 19.5, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { + shuffle => + withSQLConf(SQLConf.V2_BUCKETING_SHUFFLE_ENABLED.key -> shuffle.toString) { + val df = createJoinTestDF(Seq("id" -> "item_id", "arrive_time" -> "time")) + val shuffles = collectShuffles(df.queryExecution.executedPlan) + if (shuffle) { + assert(shuffles.size == 1, "only shuffle one side not report partitioning") + } else { + assert( + shuffles.size == 2, + "should add two side shuffle when bucketing shuffle one side" + + " is not enabled") + } + + checkAnswer(df, Seq(Row(1, "aa", 40.0, 42.0))) + } + } + } + + testGluten("SPARK-41471: shuffle one side: only one side reports partitioning") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(3, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + "(4, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + createTable(purchases, purchasesColumns, Array.empty) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + "(1, 42.0, cast('2020-01-01' as timestamp)), " + + "(3, 19.5, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { + shuffle => + withSQLConf(SQLConf.V2_BUCKETING_SHUFFLE_ENABLED.key -> shuffle.toString) { + val df = createJoinTestDF(Seq("id" -> "item_id")) + val shuffles = collectShuffles(df.queryExecution.executedPlan) + if (shuffle) { + assert(shuffles.size == 1, "only shuffle one side not report partitioning") + } else { + assert( + shuffles.size == 2, + "should add two side shuffle when bucketing shuffle one side" + + " is not enabled") + } + + checkAnswer(df, Seq(Row(1, "aa", 40.0, 42.0), Row(3, "bb", 10.0, 19.5))) + } + } + } + + testGluten("SPARK-41471: shuffle one side: shuffle side has more partition value") { + val items_partitions = Array(identity("id")) + createTable(items, itemsColumns, items_partitions) + + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(3, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + "(4, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + createTable(purchases, purchasesColumns, Array.empty) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + "(1, 42.0, cast('2020-01-01' as timestamp)), " + + "(3, 19.5, cast('2020-02-01' as timestamp)), " + + "(5, 26.0, cast('2023-01-01' as timestamp)), " + + "(6, 50.0, cast('2023-02-01' as timestamp))") + + Seq(true, false).foreach { + shuffle => + withSQLConf(SQLConf.V2_BUCKETING_SHUFFLE_ENABLED.key -> shuffle.toString) { + Seq("", "LEFT OUTER", "RIGHT OUTER", "FULL OUTER").foreach { + joinType => + val df = createJoinTestDF(Seq("id" -> "item_id"), joinType = joinType) + val shuffles = collectShuffles(df.queryExecution.executedPlan) + if (shuffle) { + assert(shuffles.size == 1, "only shuffle one side not report partitioning") + } else { + assert( + shuffles.size == 2, + "should add two side shuffle when bucketing shuffle one " + + "side is not enabled") + } + joinType match { + case "" => + checkAnswer(df, Seq(Row(1, "aa", 40.0, 42.0), Row(3, "bb", 10.0, 19.5))) + case "LEFT OUTER" => + checkAnswer( + df, + Seq( + Row(1, "aa", 40.0, 42.0), + Row(3, "bb", 10.0, 19.5), + Row(4, "cc", 15.5, null))) + case "RIGHT OUTER" => + checkAnswer( + df, + Seq( + Row(null, null, null, 26.0), + Row(null, null, null, 50.0), + Row(1, "aa", 40.0, 42.0), + Row(3, "bb", 10.0, 19.5))) + case "FULL OUTER" => + checkAnswer( + df, + Seq( + Row(null, null, null, 26.0), + Row(null, null, null, 50.0), + Row(1, "aa", 40.0, 42.0), + Row(3, "bb", 10.0, 19.5), + Row(4, "cc", 15.5, null))) + } + } + } + } + } + + testGluten("SPARK-41471: shuffle one side: partitioning with transform") { + val items_partitions = Array(years("arrive_time")) + createTable(items, itemsColumns, items_partitions) + + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(3, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + "(4, 'cc', 15.5, cast('2021-02-01' as timestamp))") + + createTable(purchases, purchasesColumns, Array.empty) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + "(1, 42.0, cast('2020-01-01' as timestamp)), " + + "(3, 19.5, cast('2021-02-01' as timestamp))") + + Seq(true, false).foreach { + shuffle => + withSQLConf(SQLConf.V2_BUCKETING_SHUFFLE_ENABLED.key -> shuffle.toString) { + val df = createJoinTestDF(Seq("arrive_time" -> "time")) + val shuffles = collectShuffles(df.queryExecution.executedPlan) + if (shuffle) { + assert(shuffles.size == 1, "partitioning with transform should trigger SPJ") + } else { + assert( + shuffles.size == 2, + "should add two side shuffle when bucketing shuffle one side" + + " is not enabled") + } + + checkAnswer( + df, + Seq(Row(1, "aa", 40.0, 42.0), Row(3, "bb", 10.0, 42.0), Row(4, "cc", 15.5, 19.5))) + } + } + } + + testGluten( + "SPARK-44647: SPJ: test join key is subset of cluster key " + + "with push values and partially-clustered") { + val table1 = "tab1e1" + val table2 = "table2" + val partition = Array(identity("id"), identity("data")) + createTable(table1, columns, partition) + sql( + s"INSERT INTO testcat.ns.$table1 VALUES " + + "(1, 'aa', cast('2020-01-01' as timestamp)), " + + "(2, 'bb', cast('2020-01-01' as timestamp)), " + + "(2, 'cc', cast('2020-01-01' as timestamp)), " + + "(3, 'dd', cast('2020-01-01' as timestamp)), " + + "(3, 'dd', cast('2020-01-01' as timestamp)), " + + "(3, 'ee', cast('2020-01-01' as timestamp)), " + + "(3, 'ee', cast('2020-01-01' as timestamp))") + + createTable(table2, columns, partition) + sql( + s"INSERT INTO testcat.ns.$table2 VALUES " + + "(4, 'zz', cast('2020-01-01' as timestamp)), " + + "(4, 'zz', cast('2020-01-01' as timestamp)), " + + "(3, 'yy', cast('2020-01-01' as timestamp)), " + + "(3, 'yy', cast('2020-01-01' as timestamp)), " + + "(3, 'xx', cast('2020-01-01' as timestamp)), " + + "(3, 'xx', cast('2020-01-01' as timestamp)), " + + "(2, 'ww', cast('2020-01-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + Seq(true, false).foreach { + filter => + Seq(true, false).foreach { + partiallyClustered => + Seq(true, false).foreach { + allowJoinKeysSubsetOfPartitionKeys => + withSQLConf( + SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION.key -> "false", + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> + partiallyClustered.toString, + SQLConf.V2_BUCKETING_PARTITION_FILTER_ENABLED.key -> filter.toString, + SQLConf.V2_BUCKETING_ALLOW_JOIN_KEYS_SUBSET_OF_PARTITION_KEYS.key -> + allowJoinKeysSubsetOfPartitionKeys.toString + ) { + val df = sql(s""" + |${selectWithMergeJoinHint("t1", "t2")} + |t1.id AS id, t1.data AS t1data, t2.data AS t2data + |FROM testcat.ns.$table1 t1 JOIN testcat.ns.$table2 t2 + |ON t1.id = t2.id ORDER BY t1.id, t1data, t2data + |""".stripMargin) + val shuffles = + collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (allowJoinKeysSubsetOfPartitionKeys) { + assert(shuffles.isEmpty, "SPJ should be triggered") + } else { + assert(shuffles.nonEmpty, "SPJ should not be triggered") + } + + val scannedPartitions = collectScans(df.queryExecution.executedPlan) + .map(_.inputRDD.partitions.length) + (allowJoinKeysSubsetOfPartitionKeys, partiallyClustered, filter) match { + // SPJ, partially-clustered, with filter + case (true, true, true) => assert(scannedPartitions == Seq(6, 6)) + + // SPJ, partially-clustered, no filter + case (true, true, false) => assert(scannedPartitions == Seq(8, 8)) + + // SPJ and not partially-clustered, with filter + case (true, false, true) => assert(scannedPartitions == Seq(2, 2)) + + // SPJ and not partially-clustered, no filter + case (true, false, false) => assert(scannedPartitions == Seq(4, 4)) + + // No SPJ + case _ => assert(scannedPartitions == Seq(5, 4)) + } + + checkAnswer( + df, + Seq( + Row(2, "bb", "ww"), + Row(2, "cc", "ww"), + Row(3, "dd", "xx"), + Row(3, "dd", "xx"), + Row(3, "dd", "xx"), + Row(3, "dd", "xx"), + Row(3, "dd", "yy"), + Row(3, "dd", "yy"), + Row(3, "dd", "yy"), + Row(3, "dd", "yy"), + Row(3, "ee", "xx"), + Row(3, "ee", "xx"), + Row(3, "ee", "xx"), + Row(3, "ee", "xx"), + Row(3, "ee", "yy"), + Row(3, "ee", "yy"), + Row(3, "ee", "yy"), + Row(3, "ee", "yy") + ) + ) + } + } + } + } + } + } + + testGluten("SPARK-44647: test join key is the second cluster key") { + val table1 = "tab1e1" + val table2 = "table2" + val partition = Array(identity("id"), identity("data")) + createTable(table1, columns, partition) + sql( + s"INSERT INTO testcat.ns.$table1 VALUES " + + "(1, 'aa', cast('2020-01-01' as timestamp)), " + + "(2, 'bb', cast('2020-01-02' as timestamp)), " + + "(3, 'cc', cast('2020-01-03' as timestamp))") + + createTable(table2, columns, partition) + sql( + s"INSERT INTO testcat.ns.$table2 VALUES " + + "(4, 'aa', cast('2020-01-01' as timestamp)), " + + "(5, 'bb', cast('2020-01-02' as timestamp)), " + + "(6, 'cc', cast('2020-01-03' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + Seq(true, false).foreach { + partiallyClustered => + Seq(true, false).foreach { + allowJoinKeysSubsetOfPartitionKeys => + withSQLConf( + SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION.key -> "false", + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> + pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> + partiallyClustered.toString, + SQLConf.V2_BUCKETING_ALLOW_JOIN_KEYS_SUBSET_OF_PARTITION_KEYS.key -> + allowJoinKeysSubsetOfPartitionKeys.toString + ) { + + val df = sql(s""" + |${selectWithMergeJoinHint("t1", "t2")} + |t1.id AS t1id, t2.id as t2id, t1.data AS data + |FROM testcat.ns.$table1 t1 JOIN testcat.ns.$table2 t2 + |ON t1.data = t2.data + |ORDER BY t1id, t1id, data + |""".stripMargin) + checkAnswer(df, Seq(Row(1, 4, "aa"), Row(2, 5, "bb"), Row(3, 6, "cc"))) + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (allowJoinKeysSubsetOfPartitionKeys) { + assert(shuffles.isEmpty, "SPJ should be triggered") + } else { + assert(shuffles.nonEmpty, "SPJ should not be triggered") + } + + val scans = collectScans(df.queryExecution.executedPlan) + .map(_.inputRDD.partitions.length) + (pushDownValues, allowJoinKeysSubsetOfPartitionKeys, partiallyClustered) match { + // SPJ and partially-clustered + case (true, true, true) => assert(scans == Seq(3, 3)) + // non-SPJ or SPJ/partially-clustered + case _ => assert(scans == Seq(3, 3)) + } + } + } + } + } + } + + testGluten("SPARK-44647: test join key is the second partition key and a transform") { + val items_partitions = Array(bucket(8, "id"), days("arrive_time")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-15' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(2, 'bb', 10.5, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + val purchases_partitions = Array(bucket(8, "item_id"), days("time")) + createTable(purchases, purchasesColumns, purchases_partitions) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + s"(1, 42.0, cast('2020-01-01' as timestamp)), " + + s"(1, 44.0, cast('2020-01-15' as timestamp)), " + + s"(1, 45.0, cast('2020-01-15' as timestamp)), " + + s"(2, 11.0, cast('2020-01-01' as timestamp)), " + + s"(3, 19.5, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { + pushDownValues => + Seq(true, false).foreach { + partiallyClustered => + Seq(true, false).foreach { + allowJoinKeysSubsetOfPartitionKeys => + withSQLConf( + SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION.key -> "false", + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushDownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> + partiallyClustered.toString, + SQLConf.V2_BUCKETING_ALLOW_JOIN_KEYS_SUBSET_OF_PARTITION_KEYS.key -> + allowJoinKeysSubsetOfPartitionKeys.toString + ) { + val df = + createJoinTestDF(Seq("arrive_time" -> "time"), extraColumns = Seq("p.item_id")) + // Currently SPJ for case where join key not same as partition key + // only supported when push-part-values enabled + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + if (allowJoinKeysSubsetOfPartitionKeys) { + assert(shuffles.isEmpty, "SPJ should be triggered") + } else { + assert(shuffles.nonEmpty, "SPJ should not be triggered") + } + + val scans = collectScans(df.queryExecution.executedPlan) + .map(_.inputRDD.partitions.length) + (allowJoinKeysSubsetOfPartitionKeys, partiallyClustered) match { + // SPJ and partially-clustered + case (true, true) => assert(scans == Seq(5, 5)) + // SPJ and not partially-clustered + case (true, false) => assert(scans == Seq(3, 3)) + // No SPJ + case _ => assert(scans == Seq(4, 4)) + } + + checkAnswer( + df, + Seq( + Row(1, "aa", 40.0, 11.0, 2), + Row(1, "aa", 40.0, 42.0, 1), + Row(1, "aa", 41.0, 44.0, 1), + Row(1, "aa", 41.0, 45.0, 1), + Row(2, "bb", 10.0, 11.0, 2), + Row(2, "bb", 10.0, 42.0, 1), + Row(2, "bb", 10.5, 11.0, 2), + Row(2, "bb", 10.5, 42.0, 1), + Row(3, "cc", 15.5, 19.5, 3) + ) + ) + } + } + } + } + } + + testGluten("SPARK-44647: shuffle one side and join keys are less than partition keys") { + val items_partitions = Array(identity("id"), identity("name")) + createTable(items, itemsColumns, items_partitions) + + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(1, 'aa', 30.0, cast('2020-01-02' as timestamp)), " + + "(3, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + "(4, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + createTable(purchases, purchasesColumns, Array.empty) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + "(1, 42.0, cast('2020-01-01' as timestamp)), " + + "(1, 89.0, cast('2020-01-03' as timestamp)), " + + "(3, 19.5, cast('2020-02-01' as timestamp)), " + + "(5, 26.0, cast('2023-01-01' as timestamp)), " + + "(6, 50.0, cast('2023-02-01' as timestamp))") + + Seq(true, false).foreach { + pushdownValues => + withSQLConf( + SQLConf.V2_BUCKETING_SHUFFLE_ENABLED.key -> "true", + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> pushdownValues.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> "false", + SQLConf.V2_BUCKETING_ALLOW_JOIN_KEYS_SUBSET_OF_PARTITION_KEYS.key -> "true" + ) { + val df = createJoinTestDF(Seq("id" -> "item_id")) + val shuffles = collectShuffles(df.queryExecution.executedPlan) + assert(shuffles.size == 1, "SPJ should be triggered") + checkAnswer( + df, + Seq( + Row(1, "aa", 30.0, 42.0), + Row(1, "aa", 30.0, 89.0), + Row(1, "aa", 40.0, 42.0), + Row(1, "aa", 40.0, 89.0), + Row(3, "bb", 10.0, 19.5))) + } + } + } + + testGluten( + "SPARK-47094: Compatible buckets does not support SPJ with " + + "push-down values or partially-clustered") { + val table1 = "tab1e1" + val table2 = "table2" + + val partition1 = Array(bucket(4, "store_id"), bucket(2, "dept_id")) + val partition2 = Array(bucket(2, "store_id"), bucket(2, "dept_id")) + + createTable(table1, columns2, partition1) + sql( + s"INSERT INTO testcat.ns.$table1 VALUES " + + "(0, 0, 'aa'), " + + "(1, 1, 'bb'), " + + "(2, 2, 'cc')" + ) + + createTable(table2, columns2, partition2) + sql( + s"INSERT INTO testcat.ns.$table2 VALUES " + + "(0, 0, 'aa'), " + + "(1, 1, 'bb'), " + + "(2, 2, 'cc')" + ) + + Seq(true, false).foreach { + allowPushDown => + Seq(true, false).foreach { + partiallyClustered => + withSQLConf( + SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION.key -> "false", + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> allowPushDown.toString, + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> + partiallyClustered.toString, + SQLConf.V2_BUCKETING_ALLOW_JOIN_KEYS_SUBSET_OF_PARTITION_KEYS.key -> "true", + SQLConf.V2_BUCKETING_ALLOW_COMPATIBLE_TRANSFORMS.key -> "true" + ) { + val df = sql(s""" + |${selectWithMergeJoinHint("t1", "t2")} + |t1.store_id, t1.store_id, t1.dept_id, t2.dept_id, t1.data, t2.data + |FROM testcat.ns.$table1 t1 JOIN testcat.ns.$table2 t2 + |ON t1.store_id = t2.store_id AND t1.dept_id = t2.dept_id + |ORDER BY t1.store_id, t1.dept_id, t1.data, t2.data + |""".stripMargin) + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + val scans = + collectScans(df.queryExecution.executedPlan).map(_.inputRDD.partitions.length) + + (allowPushDown, partiallyClustered) match { + case (true, false) => + assert(shuffles.isEmpty, "SPJ should be triggered") + assert(scans == Seq(2, 2)) + case (_, _) => + assert(shuffles.nonEmpty, "SPJ should not be triggered") + assert(scans == Seq(3, 2)) + } + + checkAnswer( + df, + Seq( + Row(0, 0, 0, 0, "aa", "aa"), + Row(1, 1, 1, 1, "bb", "bb"), + Row(2, 2, 2, 2, "cc", "cc") + )) + } + } + } + } + + testGluten( + "SPARK-47094: SPJ: Does not trigger when incompatible number of buckets on both side") { + val table1 = "tab1e1" + val table2 = "table2" + + Seq( + (2, 3), + (3, 4) + ).foreach { + case (table1buckets1, table2buckets1) => + catalog.clearTables() + + val partition1 = Array(bucket(table1buckets1, "store_id")) + val partition2 = Array(bucket(table2buckets1, "store_id")) + + Seq((table1, partition1), (table2, partition2)).foreach { + case (tab, part) => + createTable(tab, columns2, part) + val insertStr = s"INSERT INTO testcat.ns.$tab VALUES " + + "(0, 0, 'aa'), " + + "(1, 0, 'ab'), " + // duplicate partition key + "(2, 2, 'ac'), " + + "(3, 3, 'ad'), " + + "(4, 2, 'bc') " + + sql(insertStr) + } + + Seq(true, false).foreach { + allowJoinKeysSubsetOfPartitionKeys => + withSQLConf( + SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION.key -> "false", + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> "true", + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> "false", + SQLConf.V2_BUCKETING_ALLOW_JOIN_KEYS_SUBSET_OF_PARTITION_KEYS.key -> + allowJoinKeysSubsetOfPartitionKeys.toString, + SQLConf.V2_BUCKETING_ALLOW_COMPATIBLE_TRANSFORMS.key -> "true" + ) { + val df = sql(s""" + |${selectWithMergeJoinHint("t1", "t2")} + |t1.store_id, t1.dept_id, t1.data, t2.data + |FROM testcat.ns.$table1 t1 JOIN testcat.ns.$table2 t2 + |ON t1.store_id = t2.store_id AND t1.dept_id = t2.dept_id + |""".stripMargin) + + val shuffles = collectColumnarShuffleExchangeExec(df.queryExecution.executedPlan) + assert(shuffles.nonEmpty, "SPJ should not be triggered") + } + } + } + } + + testGluten("SPARK-48655: order by on partition keys should not introduce additional shuffle") { + val items_partitions = Array(identity("price"), identity("id")) + createTable(items, itemsColumns, items_partitions) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + s"(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + s"(1, 'aa', 41.0, cast('2020-01-02' as timestamp)), " + + s"(2, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + s"(3, 'cc', 15.5, cast('2020-02-01' as timestamp)), " + + s"(null, 'cc', 15.5, cast('2020-02-01' as timestamp)), " + + s"(3, 'cc', null, cast('2020-02-01' as timestamp))") + + Seq(true, false).foreach { + sortingEnabled => + withSQLConf(SQLConf.V2_BUCKETING_SORTING_ENABLED.key -> sortingEnabled.toString) { + + def verifyShuffle(cmd: String, answer: Seq[Row]): Unit = { + val df = sql(cmd) + if (sortingEnabled) { + assert( + collectAllShuffles(df.queryExecution.executedPlan).isEmpty, + "should contain no shuffle when sorting by partition values") + } else { + assert( + collectAllShuffles(df.queryExecution.executedPlan).size == 1, + "should contain one shuffle when optimization is disabled") + } + checkAnswer(df, answer) + }: Unit + + verifyShuffle( + s"SELECT price, id FROM testcat.ns.$items ORDER BY price ASC, id ASC", + Seq( + Row(null, 3), + Row(10.0, 2), + Row(15.5, null), + Row(15.5, 3), + Row(40.0, 1), + Row(41.0, 1))) + + verifyShuffle( + s"SELECT price, id FROM testcat.ns.$items " + + s"ORDER BY price ASC NULLS LAST, id ASC NULLS LAST", + Seq( + Row(10.0, 2), + Row(15.5, 3), + Row(15.5, null), + Row(40.0, 1), + Row(41.0, 1), + Row(null, 3)) + ) + + verifyShuffle( + s"SELECT price, id FROM testcat.ns.$items ORDER BY price DESC, id ASC", + Seq( + Row(41.0, 1), + Row(40.0, 1), + Row(15.5, null), + Row(15.5, 3), + Row(10.0, 2), + Row(null, 3)) + ) + + verifyShuffle( + s"SELECT price, id FROM testcat.ns.$items ORDER BY price DESC, id DESC", + Seq( + Row(41.0, 1), + Row(40.0, 1), + Row(15.5, 3), + Row(15.5, null), + Row(10.0, 2), + Row(null, 3)) + ) + + verifyShuffle( + s"SELECT price, id FROM testcat.ns.$items " + + s"ORDER BY price DESC NULLS FIRST, id DESC NULLS FIRST", + Seq( + Row(null, 3), + Row(41.0, 1), + Row(40.0, 1), + Row(15.5, null), + Row(15.5, 3), + Row(10.0, 2)) + ); + } + } + } + + testGluten("SPARK-48012: one-side shuffle with partition transforms") { + val items_partitions = Array(bucket(2, "id"), identity("arrive_time")) + val items_partitions2 = Array(identity("arrive_time"), bucket(2, "id")) + + Seq(items_partitions, items_partitions2).foreach { + partition => + catalog.clearTables() + + createTable(items, itemsColumns, partition) + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(1, 'bb', 30.0, cast('2020-01-01' as timestamp)), " + + "(1, 'cc', 30.0, cast('2020-01-02' as timestamp)), " + + "(3, 'dd', 10.0, cast('2020-01-01' as timestamp)), " + + "(4, 'ee', 15.5, cast('2020-02-01' as timestamp)), " + + "(5, 'ff', 32.1, cast('2020-03-01' as timestamp))") + + createTable(purchases, purchasesColumns, Array.empty) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + "(1, 42.0, cast('2020-01-01' as timestamp)), " + + "(2, 10.7, cast('2020-01-01' as timestamp))," + + "(3, 19.5, cast('2020-02-01' as timestamp))," + + "(4, 56.5, cast('2020-02-01' as timestamp))") + + withSQLConf(SQLConf.V2_BUCKETING_SHUFFLE_ENABLED.key -> "true") { + val df = createJoinTestDF(Seq("id" -> "item_id", "arrive_time" -> "time")) + val shuffles = collectShuffles(df.queryExecution.executedPlan) + assert(shuffles.size == 1, "only shuffle side that does not report partitioning") + + checkAnswer( + df, + Seq(Row(1, "bb", 30.0, 42.0), Row(1, "aa", 40.0, 42.0), Row(4, "ee", 15.5, 56.5))) + } + } + } + + testGluten("SPARK-48012: one-side shuffle with partition transforms and pushdown values") { + val items_partitions = Array(bucket(2, "id"), identity("arrive_time")) + createTable(items, itemsColumns, items_partitions) + + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(1, 'bb', 30.0, cast('2020-01-01' as timestamp)), " + + "(1, 'cc', 30.0, cast('2020-01-02' as timestamp))") + + createTable(purchases, purchasesColumns, Array.empty) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + "(1, 42.0, cast('2020-01-01' as timestamp)), " + + "(2, 10.7, cast('2020-01-01' as timestamp))") + + Seq(true, false).foreach { + pushDown => + { + withSQLConf( + SQLConf.V2_BUCKETING_SHUFFLE_ENABLED.key -> "true", + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> + pushDown.toString) { + val df = createJoinTestDF(Seq("id" -> "item_id", "arrive_time" -> "time")) + val shuffles = collectShuffles(df.queryExecution.executedPlan) + assert(shuffles.size == 1, "only shuffle side that does not report partitioning") + + checkAnswer(df, Seq(Row(1, "bb", 30.0, 42.0), Row(1, "aa", 40.0, 42.0))) + } + } + } + } + + testGluten( + "SPARK-48012: one-side shuffle with partition transforms " + + "with fewer join keys than partition kes") { + val items_partitions = Array(bucket(2, "id"), identity("name")) + createTable(items, itemsColumns, items_partitions) + + sql( + s"INSERT INTO testcat.ns.$items VALUES " + + "(1, 'aa', 40.0, cast('2020-01-01' as timestamp)), " + + "(1, 'aa', 30.0, cast('2020-01-02' as timestamp)), " + + "(3, 'bb', 10.0, cast('2020-01-01' as timestamp)), " + + "(4, 'cc', 15.5, cast('2020-02-01' as timestamp))") + + createTable(purchases, purchasesColumns, Array.empty) + sql( + s"INSERT INTO testcat.ns.$purchases VALUES " + + "(1, 42.0, cast('2020-01-01' as timestamp)), " + + "(1, 89.0, cast('2020-01-03' as timestamp)), " + + "(3, 19.5, cast('2020-02-01' as timestamp)), " + + "(5, 26.0, cast('2023-01-01' as timestamp)), " + + "(6, 50.0, cast('2023-02-01' as timestamp))") + + withSQLConf( + SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION.key -> "false", + SQLConf.V2_BUCKETING_SHUFFLE_ENABLED.key -> "true", + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED.key -> "true", + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key -> "false", + SQLConf.V2_BUCKETING_ALLOW_JOIN_KEYS_SUBSET_OF_PARTITION_KEYS.key -> "true" + ) { + val df = createJoinTestDF(Seq("id" -> "item_id")) + val shuffles = collectShuffles(df.queryExecution.executedPlan) + assert(shuffles.size == 1, "SPJ should be triggered") + checkAnswer( + df, + Seq( + Row(1, "aa", 30.0, 42.0), + Row(1, "aa", 30.0, 89.0), + Row(1, "aa", 40.0, 42.0), + Row(1, "aa", 40.0, 89.0), + Row(3, "bb", 10.0, 19.5))) + } + } + +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenLocalScanSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenLocalScanSuite.scala new file mode 100644 index 000000000000..735b5d1a0e1b --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenLocalScanSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenLocalScanSuite extends LocalScanSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenMergeIntoTableSuiteBase.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenMergeIntoTableSuiteBase.scala new file mode 100644 index 000000000000..11d80e20e69f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenMergeIntoTableSuiteBase.scala @@ -0,0 +1,218 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsTrait +import org.apache.spark.sql.internal.SQLConf + +/** + * A trait that provides Gluten-compatible cardinality error assertion for merge operations. + * + * In Gluten, SparkRuntimeException is wrapped inside GlutenException, so we need to check the + * exception chain for the expected error message instead of matching the exact exception type. + */ +trait GlutenMergeIntoTableSuiteBase extends MergeIntoTableSuiteBase with GlutenSQLTestsTrait { + + import testImplicits._ + + /** Helper method to find if any exception in the chain contains the expected message. */ + private def findInExceptionChain(e: Throwable, expectedMessage: String): Boolean = { + var current: Throwable = e + while (current != null) { + if (current.getMessage != null && current.getMessage.contains(expectedMessage)) { + return true + } + current = current.getCause + } + false + } + + /** + * Gluten-compatible version of assertCardinalityError. The original method expects + * SparkRuntimeException directly, but Gluten wraps it in GlutenException. + */ + protected def assertGlutenCardinalityError(query: String): Unit = { + val e = intercept[Exception] { + sql(query) + } + assert( + findInExceptionChain(e, "ON search condition of the MERGE statement"), + s"Expected cardinality violation error but got: ${e.getMessage}") + } + + testGluten("merge cardinality check with conditional MATCHED clause (delete)") { + withTempView("source") { + createAndInitTable( + "pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 6, "salary": 600, "dep": "software" } + |""".stripMargin + ) + + val sourceRows = Seq((1, 101, "support"), (1, 102, "support"), (2, 201, "support")) + sourceRows.toDF("pk", "salary", "dep").createOrReplaceTempView("source") + + assertGlutenCardinalityError( + s"""MERGE INTO $tableNameAsString AS t + |USING source AS s + |ON t.pk = s.pk + |WHEN MATCHED AND s.salary = 101 THEN + | DELETE + |""".stripMargin + ) + } + } + + testGluten("merge cardinality check with small target and large source (broadcast enabled)") { + withTempView("source") { + createAndInitTable( + "pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |""".stripMargin + ) + + val sourceRows = (1 to 1000).map(pk => (pk, pk * 100, "support")) + sourceRows.toDF("pk", "salary", "dep").createOrReplaceTempView("source") + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> Long.MaxValue.toString) { + assertGlutenCardinalityError( + s"""MERGE INTO $tableNameAsString AS t + |USING (SELECT * FROM source UNION ALL SELECT * FROM source) AS s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET * + |""".stripMargin + ) + + assert(sql(s"SELECT * FROM $tableNameAsString").count() == 2) + } + } + } + + testGluten("merge cardinality check with small target and large source (broadcast disabled)") { + withTempView("source") { + createAndInitTable( + "pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |""".stripMargin + ) + + val sourceRows = (1 to 1000).map(pk => (pk, pk * 100, "support")) + sourceRows.toDF("pk", "salary", "dep").createOrReplaceTempView("source") + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + assertGlutenCardinalityError( + s"""MERGE INTO $tableNameAsString AS t + |USING (SELECT * FROM source UNION ALL SELECT * FROM source) AS s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET * + |""".stripMargin + ) + + assert(sql(s"SELECT * FROM $tableNameAsString").count() == 2) + } + } + } + + testGluten("merge cardinality check with small target and large source (shuffle hash enabled)") { + withTempView("source") { + createAndInitTable( + "pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |""".stripMargin + ) + + val sourceRows = (1 to 1000).map(pk => (pk, pk * 100, "support")) + sourceRows.toDF("pk", "salary", "dep").createOrReplaceTempView("source") + + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.PREFER_SORTMERGEJOIN.key -> "false") { + assertGlutenCardinalityError( + s"""MERGE INTO $tableNameAsString AS t + |USING (SELECT * FROM source UNION ALL SELECT * FROM source) AS s + |ON t.pk = s.pk + |WHEN MATCHED THEN + | UPDATE SET * + |""".stripMargin + ) + + assert(sql(s"SELECT * FROM $tableNameAsString").count() == 2) + } + } + } + + testGluten("merge cardinality check without equality condition and only MATCHED clauses") { + withTempView("source") { + createAndInitTable( + "pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |""".stripMargin + ) + + val sourceRows = (1 to 1000).map(pk => (pk, pk * 100, "support")) + sourceRows.toDF("pk", "salary", "dep").createOrReplaceTempView("source") + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + assertGlutenCardinalityError( + s"""MERGE INTO $tableNameAsString AS t + |USING (SELECT * FROM source UNION ALL SELECT * FROM source) AS s + |ON t.pk > s.pk + |WHEN MATCHED THEN + | UPDATE SET * + |""".stripMargin + ) + + assert(sql(s"SELECT * FROM $tableNameAsString").count() == 2) + } + } + } + + testGluten("merge cardinality check without equality condition") { + withTempView("source") { + createAndInitTable( + "pk INT NOT NULL, salary INT, dep STRING", + """{ "pk": 1, "salary": 100, "dep": "hr" } + |{ "pk": 2, "salary": 200, "dep": "software" } + |""".stripMargin + ) + + val sourceRows = (1 to 1000).map(pk => (pk, pk * 100, "support")) + sourceRows.toDF("pk", "salary", "dep").createOrReplaceTempView("source") + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + assertGlutenCardinalityError( + s"""MERGE INTO $tableNameAsString AS t + |USING (SELECT * FROM source UNION ALL SELECT * FROM source) AS s + |ON t.pk > s.pk + |WHEN MATCHED THEN + | UPDATE SET * + |WHEN NOT MATCHED THEN + | INSERT * + |""".stripMargin + ) + + assert(sql(s"SELECT * FROM $tableNameAsString").count() == 2) + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenMetadataColumnSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenMetadataColumnSuite.scala new file mode 100644 index 000000000000..59a14fb11c00 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenMetadataColumnSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenMetadataColumnSuite extends MetadataColumnSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenSupportsCatalogOptionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenSupportsCatalogOptionsSuite.scala new file mode 100644 index 000000000000..92f2a04cebe1 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenSupportsCatalogOptionsSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenSupportsCatalogOptionsSuite + extends SupportsCatalogOptionsSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenTableCapabilityCheckSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenTableCapabilityCheckSuite.scala new file mode 100644 index 000000000000..93502b7adb05 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenTableCapabilityCheckSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenTableCapabilityCheckSuite + extends TableCapabilityCheckSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenWriteDistributionAndOrderingSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenWriteDistributionAndOrderingSuite.scala new file mode 100644 index 000000000000..f96ec9a6d1df --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/connector/GlutenWriteDistributionAndOrderingSuite.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connector + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenWriteDistributionAndOrderingSuite + extends WriteDistributionAndOrderingSuite + with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = { + // Native SQL configs + super.sparkConf + .set("spark.sql.shuffle.partitions", "5") + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/errors/GlutenQueryCompilationErrorsDSv2Suite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/errors/GlutenQueryCompilationErrorsDSv2Suite.scala new file mode 100644 index 000000000000..6c14c16664a7 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/errors/GlutenQueryCompilationErrorsDSv2Suite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.errors + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenQueryCompilationErrorsDSv2Suite + extends QueryCompilationErrorsDSv2Suite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/errors/GlutenQueryCompilationErrorsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/errors/GlutenQueryCompilationErrorsSuite.scala new file mode 100644 index 000000000000..7ccb3b059ac6 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/errors/GlutenQueryCompilationErrorsSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.errors + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenQueryCompilationErrorsSuite + extends QueryCompilationErrorsSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/errors/GlutenQueryExecutionErrorsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/errors/GlutenQueryExecutionErrorsSuite.scala new file mode 100644 index 000000000000..8c661f86023a --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/errors/GlutenQueryExecutionErrorsSuite.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.errors + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenQueryExecutionErrorsSuite + extends QueryExecutionErrorsSuite + with GlutenSQLTestsBaseTrait { + + override def sparkConf: SparkConf = { + // Disables VeloxAppendBatches in which GeneralOutIterator wraps vanilla Spark's exceptions + // with GlutenException. + super.sparkConf + .set("spark.gluten.sql.columnar.backend.velox.resizeBatches.shuffleInput", "false") + } + + override protected def getResourceParquetFilePath(name: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + name + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/errors/GlutenQueryParsingErrorsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/errors/GlutenQueryParsingErrorsSuite.scala new file mode 100644 index 000000000000..307a740396ea --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/errors/GlutenQueryParsingErrorsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.errors + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenQueryParsingErrorsSuite extends QueryParsingErrorsSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenBroadcastExchangeSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenBroadcastExchangeSuite.scala new file mode 100644 index 000000000000..e689c2a3c1e3 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenBroadcastExchangeSuite.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.broadcast.TorrentBroadcast +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, GlutenTestsBaseTrait} +import org.apache.spark.sql.classic.SparkSession +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.functions.broadcast + +class GlutenBroadcastExchangeSuite extends BroadcastExchangeSuite with GlutenSQLTestsBaseTrait {} + +// Additional tests run in 'local-cluster' mode. +class GlutenLocalBroadcastExchangeSuite + extends SparkFunSuite + with LocalSparkContext + with GlutenTestsBaseTrait + with AdaptiveSparkPlanHelper { + + def newSparkConf(): SparkConf = { + val conf = new SparkConf().setMaster("local-cluster[2,1,1024]") + GlutenSQLTestsBaseTrait.nativeSparkConf(conf, warehouse) + } + + test("SPARK-39983 - Broadcasted relation is not cached on the driver") { + // Use distributed cluster as in local mode the broabcast value is actually cached. + val conf = newSparkConf() + sc = new SparkContext(conf) + val spark = new SparkSession(sc) + + val df = spark.range(1).toDF() + val joinDF = df.join(broadcast(df), "id") + joinDF.collect() + val broadcastExchangeExec = collect(joinDF.queryExecution.executedPlan) { + case p: ColumnarBroadcastExchangeExec => p + } + assert(broadcastExchangeExec.size == 1, "one and only ColumnarBroadcastExchangeExec") + + // The broadcasted relation should not be cached on the driver. + val broadcasted = + broadcastExchangeExec(0).relationFuture.get().asInstanceOf[TorrentBroadcast[Any]] + assert(!broadcasted.hasCachedValue) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenCoalesceShufflePartitionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenCoalesceShufflePartitionsSuite.scala new file mode 100644 index 000000000000..fec67b9a279f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenCoalesceShufflePartitionsSuite.scala @@ -0,0 +1,284 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config.IO_ENCRYPTION_ENABLED +import org.apache.spark.internal.config.UI.UI_ENABLED +import org.apache.spark.sql.{GlutenTestsCommonTrait, QueryTest, SparkSession} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.internal.SQLConf + +class GlutenCoalesceShufflePartitionsSuite + extends CoalesceShufflePartitionsSuite + with GlutenTestsCommonTrait { + + override protected def afterAll(): Unit = {} + + override def withSparkSession( + f: SparkSession => Unit, + targetPostShuffleInputSize: Int, + minNumPostShufflePartitions: Option[Int], + enableIOEncryption: Boolean = false): Unit = { + val sparkConf = + new SparkConf(false) + .setMaster("local[*]") + .setAppName("test") + .set(UI_ENABLED, false) + .set(IO_ENCRYPTION_ENABLED, enableIOEncryption) + .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") + .set(SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key, "5") + .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") + .set(SQLConf.FETCH_SHUFFLE_BLOCKS_IN_BATCH.key, "true") + .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") + .set(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key, targetPostShuffleInputSize.toString) + .set(SQLConf.COALESCE_PARTITIONS_ENABLED.key, "true") + // Gluten config + .set("spark.plugins", "org.apache.gluten.GlutenPlugin") + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "5g") + minNumPostShufflePartitions match { + case Some(numPartitions) => + sparkConf.set(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key, numPartitions.toString) + case None => + sparkConf.set(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key, "1") + } + + val spark = SparkSession + .builder() + .config(sparkConf) + .getOrCreate() + try f(spark) + finally { + spark.stop() + } + } + + Seq(Some(5), None).foreach { + minNumPostShufflePartitions => + val testNameNote = minNumPostShufflePartitions match { + case Some(numPartitions) => "(minNumPostShufflePartitions: " + numPartitions + ")" + case None => "" + } + + // Ported from vanilla spark with targetPostShuffleInputSize changed. + testGluten(s"determining the number of reducers: aggregate operator$testNameNote") { + val test: SparkSession => Unit = { + spark: SparkSession => + val df = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 20 as key", "id as value") + val agg = df.groupBy("key").count() + + // Check the answer first. + QueryTest.checkAnswer(agg, spark.range(0, 20).selectExpr("id", "50 as cnt").collect()) + + // Then, let's look at the number of post-shuffle partitions estimated + // by the ExchangeCoordinator. + val finalPlan = stripAQEPlan(agg.queryExecution.executedPlan) + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } + + minNumPostShufflePartitions match { + case Some(numPartitions) => + assert(shuffleReads.isEmpty) + case None => + assert(shuffleReads.length === 1) + shuffleReads.foreach(read => assert(read.outputPartitioning.numPartitions === 3)) + } + } + // Change the original value 2000 to 2500 for gluten. The test depends on the calculation + // for bytesByPartitionId in MapOutputStatistics. Gluten has a different statistic result. + // See ShufflePartitionsUtil.coalescePartitions & GlutenColumnarShuffleWriter's mapStatus. + withSparkSession(test, 2500, minNumPostShufflePartitions) + } + + testGluten(s"determining the number of reducers: join operator$testNameNote") { + val test: SparkSession => Unit = { + spark: SparkSession => + val df1 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key1", "id as value1") + val df2 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key2", "id as value2") + + val join = df1.join(df2, col("key1") === col("key2")).select(col("key1"), col("value2")) + + // Check the answer first. + val expectedAnswer = + spark + .range(0, 1000) + .selectExpr("id % 500 as key", "id as value") + .union(spark.range(0, 1000).selectExpr("id % 500 as key", "id as value")) + QueryTest.checkAnswer(join, expectedAnswer.collect()) + + // Then, let's look at the number of post-shuffle partitions estimated + // by the ExchangeCoordinator. + val finalPlan = stripAQEPlan(join.queryExecution.executedPlan) + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } + + minNumPostShufflePartitions match { + case Some(numPartitions) => + assert(shuffleReads.isEmpty) + + case None => + assert(shuffleReads.length === 2) + shuffleReads.foreach(read => assert(read.outputPartitioning.numPartitions === 2)) + } + } + // Change the original value 16384 to 20000 for gluten. The test depends on the calculation + // for bytesByPartitionId in MapOutputStatistics. Gluten has a different statistic result. + // See ShufflePartitionsUtil.coalescePartitions & GlutenColumnarShuffleWriter's mapStatus. + withSparkSession(test, 20000, minNumPostShufflePartitions) + } + + testGluten(s"determining the number of reducers: complex query 1$testNameNote") { + val test: (SparkSession) => Unit = { + spark: SparkSession => + val df1 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key1", "id as value1") + .groupBy("key1") + .count() + .toDF("key1", "cnt1") + val df2 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key2", "id as value2") + .groupBy("key2") + .count() + .toDF("key2", "cnt2") + + val join = df1.join(df2, col("key1") === col("key2")).select(col("key1"), col("cnt2")) + + // Check the answer first. + val expectedAnswer = + spark + .range(0, 500) + .selectExpr("id", "2 as cnt") + QueryTest.checkAnswer(join, expectedAnswer.collect()) + + // Then, let's look at the number of post-shuffle partitions estimated + // by the ExchangeCoordinator. + val finalPlan = stripAQEPlan(join.queryExecution.executedPlan) + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } + + minNumPostShufflePartitions match { + case Some(numPartitions) => + assert(shuffleReads.isEmpty) + + case None => + assert(shuffleReads.length === 2) + shuffleReads.foreach(read => assert(read.outputPartitioning.numPartitions === 2)) + } + } + + // Change the original value 16384 to 20000 for gluten. The test depends on the calculation + // for bytesByPartitionId in MapOutputStatistics. Gluten has a different statistic result. + // See ShufflePartitionsUtil.coalescePartitions & GlutenColumnarShuffleWriter's mapStatus. + withSparkSession(test, 20000, minNumPostShufflePartitions) + } + + testGluten(s"determining the number of reducers: complex query 2$testNameNote") { + val test: (SparkSession) => Unit = { + spark: SparkSession => + val df1 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key1", "id as value1") + .groupBy("key1") + .count() + .toDF("key1", "cnt1") + val df2 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key2", "id as value2") + + val join = + df1 + .join(df2, col("key1") === col("key2")) + .select(col("key1"), col("cnt1"), col("value2")) + + // Check the answer first. + val expectedAnswer = + spark + .range(0, 1000) + .selectExpr("id % 500 as key", "2 as cnt", "id as value") + QueryTest.checkAnswer(join, expectedAnswer.collect()) + + // Then, let's look at the number of post-shuffle partitions estimated + // by the ExchangeCoordinator. + val finalPlan = stripAQEPlan(join.queryExecution.executedPlan) + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } + + minNumPostShufflePartitions match { + case Some(numPartitions) => + assert(shuffleReads.isEmpty) + + case None => + assert(shuffleReads.length === 2) + shuffleReads.foreach(read => assert(read.outputPartitioning.numPartitions === 3)) + } + } + + // Change the original value 12000 to 16000 for gluten. The test depends on the calculation + // for bytesByPartitionId in MapOutputStatistics. Gluten has a different statistic result. + // See ShufflePartitionsUtil.coalescePartitions & GlutenColumnarShuffleWriter's mapStatus. + withSparkSession(test, 16000, minNumPostShufflePartitions) + } + + testGluten(s"determining the number of reducers: plan already partitioned$testNameNote") { + val test: SparkSession => Unit = { + spark: SparkSession => + try { + spark.range(1000).write.bucketBy(30, "id").saveAsTable("t") + // `df1` is hash partitioned by `id`. + val df1 = spark.read.table("t") + val df2 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key2", "id as value2") + + val join = df1.join(df2, col("id") === col("key2")).select(col("id"), col("value2")) + + // Check the answer first. + val expectedAnswer = spark + .range(0, 500) + .selectExpr("id % 500", "id as value") + .union(spark.range(500, 1000).selectExpr("id % 500", "id as value")) + QueryTest.checkAnswer(join, expectedAnswer.collect()) + + // Then, let's make sure we do not reduce number of post shuffle partitions. + val finalPlan = join.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec] + .executedPlan + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } + assert(shuffleReads.length === 0) + } finally { + spark.sql("drop table t") + } + } + withSparkSession(test, 12000, minNumPostShufflePartitions) + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenExchangeSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenExchangeSuite.scala new file mode 100644 index 000000000000..9ff51f84f604 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenExchangeSuite.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenExchangeSuite extends ExchangeSuite with GlutenSQLTestsBaseTrait { + + override def sparkConf: SparkConf = { + super.sparkConf.set("spark.sql.shuffle.partitions", "2") + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenQueryExecutionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenQueryExecutionSuite.scala new file mode 100644 index 000000000000..281958455855 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenQueryExecutionSuite.scala @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.util.Utils + +import org.apache.logging.log4j.{Level, LogManager} +import org.apache.logging.log4j.core.LoggerContext + +import scala.io.Source + +class GlutenQueryExecutionSuite extends QueryExecutionSuite with GlutenSQLTestsBaseTrait { + + override def checkDumpedPlans(path: String, expected: Int): Unit = + Utils.tryWithResource(Source.fromFile(path)) { + source => + assert( + source.getLines.toList + .takeWhile(_ != "== Whole Stage Codegen ==") + .map(_.replaceAll("#\\d+", "#x")) == List( + "== Parsed Logical Plan ==", + s"Range (0, $expected, step=1, splits=Some(2))", + "", + "== Analyzed Logical Plan ==", + "id: bigint", + s"Range (0, $expected, step=1, splits=Some(2))", + "", + "== Optimized Logical Plan ==", + s"Range (0, $expected, step=1, splits=Some(2))", + "", + "== Physical Plan ==", + "*(1) ColumnarToRow", + s"+- ColumnarRange 0, $expected, 1, 2, $expected, [id#xL]", + "" + )) + } + + testGluten("dumping query execution info to a file - explainMode=formatted") { + withTempDir { + dir => + val path = dir.getCanonicalPath + "/plans.txt" + val df = spark.range(0, 10) + df.queryExecution.debug.toFile(path, explainMode = Option("formatted")) + val lines = Utils.tryWithResource(Source.fromFile(path))(_.getLines().toList) + assert( + lines + .takeWhile(_ != "== Whole Stage Codegen ==") + .map(_.replaceAll("#\\d+", "#x")) == List( + "== Physical Plan ==", + "* ColumnarToRow (2)", + "+- ColumnarRange (1)", + "", + "", + "(1) ColumnarRange", + "Output [1]: [id#xL]", + "Arguments: 0, 10, 1, 2, 10, [id#xL]", + "", + "(2) ColumnarToRow [codegen id : 1]", + "Input [1]: [id#xL]", + "", + "" + )) + } + } + + testGluten("Logging plan changes for execution") { + val ctx = LogManager.getContext(false).asInstanceOf[LoggerContext] + val config = ctx.getConfiguration + val loggerConfig = config.getLoggerConfig(LogManager.ROOT_LOGGER_NAME) + loggerConfig.setLevel(Level.INFO) + ctx.updateLoggers() + + val testAppender = new LogAppender("plan changes") + withLogAppender(testAppender) { + withSQLConf( + SQLConf.PLAN_CHANGE_LOG_LEVEL.key -> "INFO" + ) { + spark.range(1).groupBy("id").count().queryExecution.executedPlan + } + } + Seq("=== Applying Rule org.apache.spark.sql.execution", "=== Result of Batch Preparations ===") + .foreach { + expectedMsg => + assert( + testAppender.loggingEvents.exists( + _.getMessage.getFormattedMessage.contains(expectedMsg) + ) + ) + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenRemoveRedundantWindowGroupLimitsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenRemoveRedundantWindowGroupLimitsSuite.scala new file mode 100644 index 000000000000..9d819d2bd90f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenRemoveRedundantWindowGroupLimitsSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenRemoveRedundantWindowGroupLimitsSuite + extends RemoveRedundantWindowGroupLimitsSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala new file mode 100644 index 000000000000..33bf1a1ec97e --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.gluten.execution.HashAggregateExecBaseTransformer + +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait} +import org.apache.spark.sql.execution.aggregate.{ObjectHashAggregateExec, SortAggregateExec} +import org.apache.spark.sql.internal.SQLConf + +class GlutenReplaceHashWithSortAggSuite + extends ReplaceHashWithSortAggSuite + with GlutenSQLTestsBaseTrait { + + private def checkNumAggs(df: DataFrame, hashAggCount: Int, sortAggCount: Int): Unit = { + val plan = df.queryExecution.executedPlan + assert(collectWithSubqueries(plan) { + case s @ (_: HashAggregateExecBaseTransformer | _: ObjectHashAggregateExec) => s + }.length == hashAggCount) + assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) + } + + private def checkAggs( + query: String, + enabledHashAggCount: Int, + enabledSortAggCount: Int, + disabledHashAggCount: Int, + disabledSortAggCount: Int): Unit = { + withSQLConf(SQLConf.REPLACE_HASH_WITH_SORT_AGG_ENABLED.key -> "true") { + val df = sql(query) + checkNumAggs(df, enabledHashAggCount, enabledSortAggCount) + val result = df.collect() + withSQLConf(SQLConf.REPLACE_HASH_WITH_SORT_AGG_ENABLED.key -> "false") { + val df = sql(query) + checkNumAggs(df, disabledHashAggCount, disabledSortAggCount) + checkAnswer(df, result) + } + } + } + + // === Following cases override super class's cases === + + testGluten("replace partial hash aggregate with sort aggregate") { + withTempView("t") { + spark.range(100).selectExpr("id as key").repartition(10).createOrReplaceTempView("t") + + Seq("FIRST", "COLLECT_LIST").foreach { + aggExpr => + // Because repartition modification causing the result sort order not same and the + // result not same, so we add order by key before comparing the result. + val query = + s""" + |SELECT key, $aggExpr(key) + |FROM + |( + | SELECT key + | FROM t + | WHERE key > 10 + | SORT BY key + |) + |GROUP BY key + |ORDER BY key + """.stripMargin + checkAggs(query, 2, 0, 2, 0) + } + } + } + + testGluten("replace partial and final hash aggregate together with sort aggregate") { + withTempView("t1", "t2") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") + spark.range(50).selectExpr("id as key").createOrReplaceTempView("t2") + Seq(("COUNT", 1, 0, 1, 0), ("COLLECT_LIST", 1, 0, 1, 0)).foreach { + aggExprInfo => + val query = + s""" + |SELECT key, ${aggExprInfo._1}(key) + |FROM + |( + | SELECT /*+ SHUFFLE_MERGE(t1) */ t1.key AS key + | FROM t1 + | JOIN t2 + | ON t1.key = t2.key + |) + |GROUP BY key + """.stripMargin + checkAggs(query, aggExprInfo._2, aggExprInfo._3, aggExprInfo._4, aggExprInfo._5) + } + } + } + + testGluten("do not replace hash aggregate if child does not have sort order") { + withTempView("t1", "t2") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") + spark.range(50).selectExpr("id as key").createOrReplaceTempView("t2") + Seq("COUNT", "COLLECT_LIST").foreach { + aggExpr => + val query = + s""" + |SELECT key, $aggExpr(key) + |FROM + |( + | SELECT /*+ BROADCAST(t1) */ t1.key AS key + | FROM t1 + | JOIN t2 + | ON t1.key = t2.key + |) + |GROUP BY key + """.stripMargin + checkAggs(query, 2, 0, 2, 0) + } + } + } + + testGluten("do not replace hash aggregate if there is no group-by column") { + withTempView("t1") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") + Seq("COUNT", "COLLECT_LIST").foreach { + aggExpr => + val query = + s""" + |SELECT $aggExpr(key) + |FROM t1 + """.stripMargin + checkAggs(query, 2, 0, 2, 0) + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenReuseExchangeAndSubquerySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenReuseExchangeAndSubquerySuite.scala new file mode 100644 index 000000000000..d7232f6a06c8 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenReuseExchangeAndSubquerySuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenReuseExchangeAndSubquerySuite + extends ReuseExchangeAndSubquerySuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenSQLAggregateFunctionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenSQLAggregateFunctionSuite.scala new file mode 100644 index 000000000000..b939c65f34bb --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenSQLAggregateFunctionSuite.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.gluten.execution.HashAggregateExecBaseTransformer + +import org.apache.spark.sql.{GlutenSQLTestsTrait, Row} +import org.apache.spark.sql.internal.SQLConf + +class GlutenSQLAggregateFunctionSuite extends GlutenSQLTestsTrait { + + // TODO: fix in Spark-4.0 + ignoreGluten("GLUTEN-4853: The result order is reversed for count and count distinct") { + val query = + """ + |select count(distinct if(sex = 'x', id, null)) as uv, count(if(sex = 'x', id, null)) as pv + |from values (1, 'x'), (1, 'x'), (2, 'y'), (3, 'x'), (3, 'x'), (4, 'y'), (5, 'x') + |AS tab(id, sex) + |""".stripMargin + val df = sql(query) + checkAnswer(df, Seq(Row(3, 5))) + assert(getExecutedPlan(df).count(_.isInstanceOf[HashAggregateExecBaseTransformer]) == 4) + } + + // TODO: fix in Spark-4.0 + ignoreGluten("Return NaN or null when dividing by zero") { + val query = + """ + |select skewness(value), kurtosis(value) + |from values (1), (1) + |AS tab(value) + |""".stripMargin + val df = sql(query) + + withSQLConf( + SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "true" + ) { + checkAnswer(df, Seq(Row(Double.NaN, Double.NaN))) + assert(getExecutedPlan(df).count(_.isInstanceOf[HashAggregateExecBaseTransformer]) == 2) + } + + withSQLConf( + SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> + SQLConf.LEGACY_STATISTICAL_AGGREGATE.defaultValueString + ) { + checkAnswer(df, Seq(Row(null, null))) + assert(getExecutedPlan(df).count(_.isInstanceOf[HashAggregateExecBaseTransformer]) == 2) + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenSQLCollectLimitExecSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenSQLCollectLimitExecSuite.scala new file mode 100644 index 000000000000..a18d3024fab8 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenSQLCollectLimitExecSuite.scala @@ -0,0 +1,264 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.gluten.execution.ColumnarCollectLimitBaseExec + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsTrait, Row} + +class GlutenSQLCollectLimitExecSuite extends GlutenSQLTestsTrait { + + override def sparkConf: SparkConf = { + super.sparkConf + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + } + + private def assertGlutenOperatorMatch[T: reflect.ClassTag]( + df: DataFrame, + checkMatch: Boolean): Unit = { + val executedPlan = getExecutedPlan(df) + + val operatorFound = executedPlan.exists { + plan => + try { + implicitly[reflect.ClassTag[T]].runtimeClass.isInstance(plan) + } catch { + case _: Throwable => false + } + } + + val assertionCondition = operatorFound == checkMatch + val assertionMessage = + if (checkMatch) { + s"Operator ${implicitly[reflect.ClassTag[T]].runtimeClass.getSimpleName} not found " + + s"in executed plan:\n $executedPlan" + } else { + s"Operator ${implicitly[reflect.ClassTag[T]].runtimeClass.getSimpleName} was found " + + s"in executed plan:\n $executedPlan" + } + + assert(assertionCondition, assertionMessage) + } + + test("ColumnarCollectLimitExec - basic limit test") { + val df = spark.range(0, 1000, 1).toDF("id").limit(5) + val expectedData = Seq(Row(0L), Row(1L), Row(2L), Row(3L), Row(4L)) + + checkAnswer(df, expectedData) + + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df, checkMatch = true) + } + + test("ColumnarCollectLimitExec - with filter") { + val df = spark + .range(0, 20, 1) + .toDF("id") + .filter("id % 2 == 0") + .limit(5) + val expectedData = Seq(Row(0L), Row(2L), Row(4L), Row(6L), Row(8L)) + + checkAnswer(df, expectedData) + + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df, checkMatch = true) + } + + test("ColumnarCollectLimitExec - range with repartition") { + + val df = spark + .range(0, 10, 1) + .toDF("id") + .repartition(3) + .orderBy("id") + .limit(3) + val expectedData = Seq(Row(0L), Row(1L), Row(2L)) + + checkAnswer(df, expectedData) + } + + ignore("ColumnarCollectLimitExec - with distinct values") { + val df = spark + .range(0, 10, 1) + .toDF("id") + .select("id") + .distinct() + .limit(5) + val expectedData = Seq(Row(0L), Row(1L), Row(2L), Row(3L), Row(4L)) + + checkAnswer(df, expectedData) + + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df, checkMatch = true) + } + + test("ColumnarCollectLimitExec - chained limit") { + val df = spark + .range(0, 10, 1) + .toDF("id") + .limit(8) + .limit(3) + val expectedData = Seq(Row(0L), Row(1L), Row(2L)) + + checkAnswer(df, expectedData) + + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df, checkMatch = true) + } + + test("ColumnarCollectLimitExec - limit after union") { + val df1 = spark.range(0, 5).toDF("id") + val df2 = spark.range(5, 10).toDF("id") + val unionDf = df1.union(df2).limit(3) + + val expectedData = Seq(Row(0L), Row(1L), Row(2L)) + + checkAnswer(unionDf, expectedData) + + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](unionDf, checkMatch = true) + } + + test("ColumnarCollectLimitExec - offset test") { + val df1 = spark.range(0, 10, 1).toDF("id").limit(5).offset(2) + val expectedData1 = Seq(Row(2L), Row(3L), Row(4L)) + + checkAnswer(df1, expectedData1) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df1, checkMatch = true) + + val df2 = spark.range(0, 20, 1).toDF("id").limit(12).offset(5) + val expectedData2 = Seq(Row(5L), Row(6L), Row(7L), Row(8L), Row(9L), Row(10L), Row(11L)) + + checkAnswer(df2, expectedData2) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df2, checkMatch = true) + + val df3 = spark.range(0, 30, 1).toDF("id").limit(10).offset(3) + val expectedData3 = Seq(Row(3L), Row(4L), Row(5L), Row(6L), Row(7L), Row(8L), Row(9L)) + + checkAnswer(df3, expectedData3) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df3, checkMatch = true) + + val df4 = spark.range(0, 15, 1).toDF("id").limit(8).offset(4) + val expectedData4 = Seq(Row(4L), Row(5L), Row(6L), Row(7L)) + + checkAnswer(df4, expectedData4) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df4, checkMatch = true) + + val df5 = spark.range(0, 50, 1).toDF("id").limit(20).offset(10) + val expectedData5 = Seq( + Row(10L), + Row(11L), + Row(12L), + Row(13L), + Row(14L), + Row(15L), + Row(16L), + Row(17L), + Row(18L), + Row(19L)) + + checkAnswer(df5, expectedData5) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df5, checkMatch = true) + } + + test("ColumnarCollectLimitExec - pure offset test") { + val df1 = spark.range(0, 20, 1).toDF("id").offset(5) + val expectedData1 = Seq( + Row(5L), + Row(6L), + Row(7L), + Row(8L), + Row(9L), + Row(10L), + Row(11L), + Row(12L), + Row(13L), + Row(14L), + Row(15L), + Row(16L), + Row(17L), + Row(18L), + Row(19L)) + + checkAnswer(df1, expectedData1) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df1, checkMatch = true) + + val df2 = spark.range(0, 50, 1).toDF("id").offset(10) + val expectedData2 = (10L to 49L).map(Row(_)) + + checkAnswer(df2, expectedData2) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df2, checkMatch = true) + + val df3 = spark.range(0, 100, 2).toDF("id").offset(15) + val expectedData3 = (30L to 98L by 2).map(Row(_)) + + checkAnswer(df3, expectedData3) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df3, checkMatch = true) + + val df4 = spark.range(0, 30, 1).toDF("id").offset(20) + val expectedData4 = (20L to 29L).map(Row(_)) + + checkAnswer(df4, expectedData4) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df4, checkMatch = true) + + val df5 = spark.range(0, 200, 5).toDF("id").offset(10) + val expectedData5 = (50L to 195L by 5).map(Row(_)) + + checkAnswer(df5, expectedData5) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df5, checkMatch = true) + + val df6 = spark.range(0, 5, 1).toDF("id").limit(10) + val expectedData6 = (0L to 4L).map(Row(_)) + + checkAnswer(df6, expectedData6) + } + + test("ColumnarCollectLimitExec - offset with filter") { + val df = spark.range(0, 10, 1).toDF("id").filter("id % 2 == 0").limit(5).offset(2) + val expectedData = Seq(Row(4L), Row(6L), Row(8L)) + + checkAnswer(df, expectedData) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](df, checkMatch = true) + } + + test("ColumnarCollectLimitExec - offset after union") { + val df1 = spark.range(0, 5).toDF("id") + val df2 = spark.range(5, 10).toDF("id") + val unionDf = df1.union(df2).limit(6).offset(3) + + val expectedData = Seq(Row(3L), Row(4L), Row(5L)) + checkAnswer(unionDf, expectedData) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](unionDf, checkMatch = true) + } + + test("ColumnarCollectLimitExec - single partition with limit, offset, and limit + offset") { + + val singlePartitionDF = spark.range(0, 10, 1).toDF("id").coalesce(1) + + val limitDF = singlePartitionDF.limit(5) + val expectedLimitData = Seq(Row(0L), Row(1L), Row(2L), Row(3L), Row(4L)) + checkAnswer(limitDF, expectedLimitData) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](limitDF, checkMatch = true) + + val offsetDF = singlePartitionDF.offset(3) + val expectedOffsetData = Seq(Row(3L), Row(4L), Row(5L), Row(6L), Row(7L), Row(8L), Row(9L)) + checkAnswer(offsetDF, expectedOffsetData) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](offsetDF, checkMatch = true) + + val limitOffsetDF = singlePartitionDF.limit(5).offset(2) + val expectedLimitOffsetData = Seq(Row(2L), Row(3L), Row(4L)) + checkAnswer(limitOffsetDF, expectedLimitOffsetData) + assertGlutenOperatorMatch[ColumnarCollectLimitBaseExec](limitOffsetDF, checkMatch = true) + } + +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenSQLWindowFunctionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenSQLWindowFunctionSuite.scala new file mode 100644 index 000000000000..7c803dd78d20 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenSQLWindowFunctionSuite.scala @@ -0,0 +1,231 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.gluten.execution.{SortExecTransformer, WindowExecTransformer, WindowGroupLimitExecTransformer} + +import org.apache.spark.sql.GlutenSQLTestsTrait +import org.apache.spark.sql.Row +import org.apache.spark.sql.types._ + +class GlutenSQLWindowFunctionSuite extends SQLWindowFunctionSuite with GlutenSQLTestsTrait { + + private def decimal(v: BigDecimal): Decimal = Decimal(v, 7, 2) + + val customerSchema = StructType( + List( + StructField("c_custkey", IntegerType), + StructField("c_nationkey", IntegerType), + StructField("c_acctbal", DecimalType(7, 2)) + ) + ) + + val customerData = Seq( + Row(4553, 11, decimal(6388.41)), + Row(4953, 10, decimal(6037.28)), + Row(35403, 5, decimal(6034.70)), + Row(35803, 12, decimal(5284.87)), + Row(60865, 5, decimal(-227.82)), + Row(61065, 13, decimal(7284.77)), + Row(127412, 13, decimal(4621.41)), + Row(148303, 10, decimal(4302.30)), + Row(9954, 5, decimal(7587.25)), + Row(95337, 12, decimal(915.61)) + ) + + // TODO: fix in Spark-4.0 + ignore("Literal in window partition by and sort") { + withTable("customer") { + val rdd = spark.sparkContext.parallelize(customerData) + val customerDF = spark.createDataFrame(rdd, customerSchema) + customerDF.createOrReplaceTempView("customer") + val query = + """ + |SELECT + | c_custkey, + | c_acctbal, + | row_number() OVER ( + | PARTITION BY c_nationkey, + | "a" + | ORDER BY + | c_custkey, + | "a" + | ) AS row_num + |FROM + | customer + |ORDER BY 1, 2; + |""".stripMargin + val df = sql(query) + checkAnswer( + df, + Seq( + Row(4553, BigDecimal(638841L, 2), 1), + Row(4953, BigDecimal(603728L, 2), 1), + Row(9954, BigDecimal(758725L, 2), 1), + Row(35403, BigDecimal(603470L, 2), 2), + Row(35803, BigDecimal(528487L, 2), 1), + Row(60865, BigDecimal(-22782L, 2), 3), + Row(61065, BigDecimal(728477L, 2), 1), + Row(95337, BigDecimal(91561L, 2), 2), + Row(127412, BigDecimal(462141L, 2), 2), + Row(148303, BigDecimal(430230L, 2), 2) + ) + ) + assert( + getExecutedPlan(df).exists { + case _: WindowExecTransformer => true + case _ => false + } + ) + } + } + + // TODO: fix in Spark-4.0 + ignore("Filter on row number") { + withTable("customer") { + val rdd = spark.sparkContext.parallelize(customerData) + val customerDF = spark.createDataFrame(rdd, customerSchema) + customerDF.createOrReplaceTempView("customer") + val query = + """ + |SELECT * from (SELECT + | c_custkey, + | c_acctbal, + | row_number() OVER ( + | PARTITION BY c_nationkey, + | "a" + | ORDER BY + | c_custkey, + | "a" + | ) AS row_num + |FROM + | customer ORDER BY 1, 2) where row_num <=2 + |""".stripMargin + val df = sql(query) + checkAnswer( + df, + Seq( + Row(4553, BigDecimal(638841L, 2), 1), + Row(4953, BigDecimal(603728L, 2), 1), + Row(9954, BigDecimal(758725L, 2), 1), + Row(35403, BigDecimal(603470L, 2), 2), + Row(35803, BigDecimal(528487L, 2), 1), + Row(61065, BigDecimal(728477L, 2), 1), + Row(95337, BigDecimal(91561L, 2), 2), + Row(127412, BigDecimal(462141L, 2), 2), + Row(148303, BigDecimal(430230L, 2), 2) + ) + ) + assert( + getExecutedPlan(df).exists { + case _: WindowGroupLimitExecTransformer => true + case _ => false + } + ) + assert( + getExecutedPlan(df).collect { case s: SortExecTransformer if !s.global => s }.size == 1 + ) + } + } + + testGluten("Filter on rank") { + withTable("customer") { + val rdd = spark.sparkContext.parallelize(customerData) + val customerDF = spark.createDataFrame(rdd, customerSchema) + customerDF.createOrReplaceTempView("customer") + val query = + """ + |SELECT * from (SELECT + | c_custkey, + | c_acctbal, + | rank() OVER ( + | PARTITION BY c_nationkey, + | "a" + | ORDER BY + | c_custkey, + | "a" + | ) AS rank + |FROM + | customer ORDER BY 1, 2) where rank <=2 + |""".stripMargin + val df = sql(query) + checkAnswer( + df, + Seq( + Row(4553, BigDecimal(638841L, 2), 1), + Row(4953, BigDecimal(603728L, 2), 1), + Row(9954, BigDecimal(758725L, 2), 1), + Row(35403, BigDecimal(603470L, 2), 2), + Row(35803, BigDecimal(528487L, 2), 1), + Row(61065, BigDecimal(728477L, 2), 1), + Row(95337, BigDecimal(91561L, 2), 2), + Row(127412, BigDecimal(462141L, 2), 2), + Row(148303, BigDecimal(430230L, 2), 2) + ) + ) + assert( + !getExecutedPlan(df).exists { + case _: WindowGroupLimitExecTransformer => true + case _ => false + } + ) + } + } + + // TODO: fix in Spark-4.0 + ignore("Expression in WindowExpression") { + withTable("customer") { + val rdd = spark.sparkContext.parallelize(customerData) + val customerDF = spark.createDataFrame(rdd, customerSchema) + customerDF.createOrReplaceTempView("customer") + val query = + """ + |SELECT + | c_custkey, + | avg(c_acctbal) OVER ( + | PARTITION BY c_nationkey + | ORDER BY c_custkey + | ) + |FROM + | customer + |ORDER BY 1, 2; + |""".stripMargin + val df = sql(query) + checkAnswer( + df, + Seq( + Row(4553, BigDecimal(6388410000L, 6)), + Row(4953, BigDecimal(6037280000L, 6)), + Row(9954, BigDecimal(7587250000L, 6)), + Row(35403, BigDecimal(6810975000L, 6)), + Row(35803, BigDecimal(5284870000L, 6)), + Row(60865, BigDecimal(4464710000L, 6)), + Row(61065, BigDecimal(7284770000L, 6)), + Row(95337, BigDecimal(3100240000L, 6)), + Row(127412, BigDecimal(5953090000L, 6)), + Row(148303, BigDecimal(5169790000L, 6)) + ) + ) + assert( + getExecutedPlan(df).exists { + case _: WindowExecTransformer => true + case _ => false + } + ) + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenSameResultSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenSameResultSuite.scala new file mode 100644 index 000000000000..de9a897ffb09 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenSameResultSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenSameResultSuite extends SameResultSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenSortSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenSortSuite.scala new file mode 100644 index 000000000000..ae92e440c902 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenSortSuite.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.gluten.execution.SortExecTransformer + +import org.apache.spark.sql.{catalyst, GlutenQueryTestUtil, GlutenSQLTestsBaseTrait, Row} +import org.apache.spark.sql.catalyst.analysis.{Resolver, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.expressions.{Length, SortOrder} +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.functions.length + +class GlutenSortSuite extends SortSuite with GlutenSQLTestsBaseTrait with AdaptiveSparkPlanHelper { + import testImplicits._ + + protected val resolver: Resolver = conf.resolver + + protected def attr(name: String): UnresolvedAttribute = { + UnresolvedAttribute(name) + } + + protected def resolveAttrs[T <: QueryPlan[T]]( + expr: catalyst.expressions.Expression, + plan: QueryPlan[T]): catalyst.expressions.Expression = { + + expr.transform { + case UnresolvedAttribute(Seq(attrName)) => + plan.output.find(attr => resolver(attr.name, attrName)).get + case UnresolvedAttribute(nameParts) => + val attrName = nameParts.mkString(".") + fail(s"cannot resolve a nested attr: $attrName") + } + } + + // TODO: fix in Spark-4.0 + ignore("post-project outputOrdering check") { + val input = Seq( + ("Hello", 4, 2.0), + ("Hello Bob", 10, 1.0), + ("Hello Bob", 1, 3.0) + ) + + val df = input.toDF("a", "b", "c").orderBy(length($"a").desc, $"b".desc) + GlutenQueryTestUtil.checkAnswer( + df, + Seq( + Row("Hello Bob", 10, 1.0), + Row("Hello Bob", 1, 3.0), + Row("Hello", 4, 2.0) + ) + ) + + val ordering = Seq( + catalyst.expressions.SortOrder( + Length(attr("a")), + catalyst.expressions.Descending, + catalyst.expressions.NullsLast, + Seq.empty + ), + catalyst.expressions.SortOrder( + attr("b"), + catalyst.expressions.Descending, + catalyst.expressions.NullsLast, + Seq.empty + ) + ) + + assert( + getExecutedPlan(df).exists { + case _: SortExecTransformer => true + case _ => false + } + ) + val plan = stripAQEPlan(df.queryExecution.executedPlan) + val actualOrdering = plan.outputOrdering + val expectedOrdering = ordering.map(resolveAttrs(_, plan).asInstanceOf[SortOrder]) + assert(actualOrdering.length == expectedOrdering.length) + actualOrdering.zip(expectedOrdering).foreach { + case (actual, expected) => + assert(actual.semanticEquals(expected), "ordering must match") + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenStreamingQuerySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenStreamingQuerySuite.scala new file mode 100644 index 000000000000..d59c04693767 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenStreamingQuerySuite.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.spark.sql.GlutenSQLTestsTrait +import org.apache.spark.sql.execution.exchange.REQUIRED_BY_STATEFUL_OPERATOR +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream +import org.apache.spark.sql.streaming._ + +class GlutenStreamingQuerySuite extends StreamingQuerySuite with GlutenSQLTestsTrait { + + import testImplicits._ + + testGluten("SPARK-49905") { + val inputData = MemoryStream[Int] + + // Use the streaming aggregation as an example - all stateful operators are using the same + // distribution, named `StatefulOpClusteredDistribution`. + val df = inputData.toDF().groupBy("value").count() + + testStream(df, OutputMode.Update())( + AddData(inputData, 1, 2, 3, 1, 2, 3), + CheckAnswer((1, 2), (2, 2), (3, 2)), + Execute { + qe => + val shuffleOpt = qe.lastExecution.executedPlan.collect { + case s: ColumnarShuffleExchangeExec => s + } + + assert(shuffleOpt.nonEmpty, "No shuffle exchange found in the query plan") + assert(shuffleOpt.head.shuffleOrigin === REQUIRED_BY_STATEFUL_OPERATOR) + } + ) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenTakeOrderedAndProjectSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenTakeOrderedAndProjectSuite.scala new file mode 100644 index 000000000000..2731e0547111 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/GlutenTakeOrderedAndProjectSuite.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} +import org.apache.spark.sql.catalyst.expressions.{Alias, Literal, Rand} +import org.apache.spark.sql.types.{IntegerType, StructType} + +class GlutenTakeOrderedAndProjectSuite + extends TakeOrderedAndProjectSuite + with GlutenSQLTestsBaseTrait { + + private def noOpFilter(plan: SparkPlan): SparkPlan = FilterExec(Literal(true), plan) + + testGluten("SPARK-47104: Non-deterministic expressions in projection") { + val expected = (input: SparkPlan) => { + GlobalLimitExec(limit, LocalLimitExec(limit, SortExec(sortOrder, true, input))) + } + val schema = StructType.fromDDL("a int, b int, c double") + val rdd = sparkContext.parallelize( + Seq( + Row(1, 2, 0.6027633705776989d), + Row(2, 3, 0.7151893651681639d), + Row(3, 4, 0.5488135024422883d)), + 1) + val df = spark.createDataFrame(rdd, schema) + val projection = df.queryExecution.sparkPlan.output.take(2) :+ + Alias(Rand(Literal(0, IntegerType)), "_uuid")() + + // test executeCollect + checkThatPlansAgree( + df, + input => + TakeOrderedAndProjectExec(limit, sortOrder, projection, SortExec(sortOrder, false, input)), + input => expected(input), + sortAnswers = false) + + // test doExecute + checkThatPlansAgree( + df, + input => + noOpFilter( + TakeOrderedAndProjectExec( + limit, + sortOrder, + projection, + SortExec(sortOrder, false, input))), + input => expected(input), + sortAnswers = false) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala new file mode 100644 index 000000000000..903c7ccd13ab --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/adaptive/clickhouse/ClickHouseAdaptiveQueryExecSuite.scala @@ -0,0 +1,1569 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.adaptive.clickhouse + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.{BroadcastHashJoinExecTransformerBase, ColumnarToCarrierRowExecBase, ShuffledHashJoinExecTransformerBase, SortExecTransformer, SortMergeJoinExecTransformer} + +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} +import org.apache.spark.sql.{Dataset, GlutenSQLTestsTrait, Row} +import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.datasources.noop.NoopDataSource +import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec +import org.apache.spark.sql.execution.exchange._ +import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter +import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.functions.when +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestData.TestData +import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.util.QueryExecutionListener + +import org.apache.logging.log4j.Level + +class ClickHouseAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQLTestsTrait { + import testImplicits._ + + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") + .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") + } + + private def runAdaptiveAndVerifyResult(query: String): (SparkPlan, SparkPlan) = { + var finalPlanCnt = 0 + val listener = new SparkListener { + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case SparkListenerSQLAdaptiveExecutionUpdate(_, _, sparkPlanInfo) => + if (sparkPlanInfo.simpleString.startsWith("AdaptiveSparkPlan isFinalPlan=true")) { + finalPlanCnt += 1 + } + case _ => // ignore other events + } + } + } + spark.sparkContext.addSparkListener(listener) + + val dfAdaptive = sql(query) + val planBefore = dfAdaptive.queryExecution.executedPlan + assert(planBefore.toString.startsWith("AdaptiveSparkPlan isFinalPlan=false")) + val result = dfAdaptive.collect() + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + val df = sql(query) + checkAnswer(df, result) + } + val planAfter = dfAdaptive.queryExecution.executedPlan + assert(planAfter.toString.startsWith("AdaptiveSparkPlan isFinalPlan=true")) + val adaptivePlan = planAfter.asInstanceOf[AdaptiveSparkPlanExec].executedPlan + + spark.sparkContext.listenerBus.waitUntilEmpty() + // AQE will post `SparkListenerSQLAdaptiveExecutionUpdate` twice in case of subqueries that + // exist out of query stages. + val expectedFinalPlanCnt = adaptivePlan.find(_.subqueries.nonEmpty).map(_ => 2).getOrElse(1) + assert(finalPlanCnt == expectedFinalPlanCnt) + spark.sparkContext.removeSparkListener(listener) + + val exchanges = adaptivePlan.collect { case e: Exchange => e } + assert(exchanges.isEmpty, "The final plan should not contain any Exchange node.") + (dfAdaptive.queryExecution.sparkPlan, adaptivePlan) + } + + private def broadcastHashJoinSize(plan: SparkPlan): Int = { + findTopLevelBroadcastHashJoinTransform(plan).size + findTopLevelBroadcastHashJoin(plan).size + } + + private def findTopLevelBroadcastHashJoinTransform( + plan: SparkPlan): Seq[BroadcastHashJoinExecTransformerBase] = { + collect(plan) { case j: BroadcastHashJoinExecTransformerBase => j } + } + + private def findTopLevelBroadcastHashJoin(plan: SparkPlan): Seq[BroadcastHashJoinExec] = { + collect(plan) { case j: BroadcastHashJoinExec => j } + } + + private def findTopLevelSortMergeJoin(plan: SparkPlan): Seq[SortMergeJoinExec] = { + collect(plan) { case j: SortMergeJoinExec => j } + } + + private def findTopLevelSortMergeJoinTransform( + plan: SparkPlan): Seq[SortMergeJoinExecTransformer] = { + collect(plan) { case j: SortMergeJoinExecTransformer => j } + } + + private def sortMergeJoinSize(plan: SparkPlan): Int = { + findTopLevelSortMergeJoinTransform(plan).size + findTopLevelSortMergeJoin(plan).size + } + + private def findTopLevelShuffledHashJoin(plan: SparkPlan): Seq[ShuffledHashJoinExec] = { + collect(plan) { case j: ShuffledHashJoinExec => j } + } + + private def findTopLevelShuffledHashJoinTransform( + plan: SparkPlan): Seq[ShuffledHashJoinExecTransformerBase] = { + collect(plan) { case j: ShuffledHashJoinExecTransformerBase => j } + } + + private def findTopLevelBaseJoin(plan: SparkPlan): Seq[BaseJoinExec] = { + collect(plan) { case j: BaseJoinExec => j } + } + + private def findTopLevelSort(plan: SparkPlan): Seq[SortExec] = { + collect(plan) { case s: SortExec => s } + } + + private def findTopLevelSortTransform(plan: SparkPlan): Seq[SortExecTransformer] = { + collect(plan) { case s: SortExecTransformer => s } + } + + private def findReusedExchange(plan: SparkPlan): Seq[ReusedExchangeExec] = { + collectWithSubqueries(plan) { + case ShuffleQueryStageExec(_, e: ReusedExchangeExec, _) => e + case BroadcastQueryStageExec(_, e: ReusedExchangeExec, _) => e + } + } + + private def findReusedSubquery(plan: SparkPlan): Seq[ReusedSubqueryExec] = { + collectWithSubqueries(plan) { case e: ReusedSubqueryExec => e } + } + + private def checkNumLocalShuffleReads( + plan: SparkPlan, + numShufflesWithoutLocalRead: Int = 0): Unit = { + val numShuffles = collect(plan) { case s: ShuffleQueryStageExec => s }.length + + val numLocalReads = collect(plan) { + case r: AQEShuffleReadExec if r.isLocalRead => r + } + // because columnar local reads cannot execute + numLocalReads.foreach { + r => + val rdd = r.executeColumnar() + val parts = rdd.partitions + assert(parts.forall(rdd.preferredLocations(_).nonEmpty)) + } + assert(numShuffles === (numLocalReads.length + numShufflesWithoutLocalRead)) + } + + testGluten("Change merge join to broadcast join") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300" + ) { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + checkNumLocalShuffleReads(adaptivePlan) + } + } + + testGluten("Change broadcast join to merge join") { + withTable("t1", "t2") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10000", + SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + sql("CREATE TABLE t1 USING PARQUET AS SELECT 1 c1") + sql("CREATE TABLE t2 USING PARQUET AS SELECT 1 c1") + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult(""" + |SELECT * FROM ( + | SELECT distinct c1 from t1 + | ) tmp1 JOIN ( + | SELECT distinct c1 from t2 + | ) tmp2 ON tmp1.c1 = tmp2.c1 + |""".stripMargin) + assert(broadcastHashJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 0) + assert(findTopLevelSortMergeJoinTransform(adaptivePlan).size == 1) + } + } + } + + testGluten("Reuse the parallelism of coalesced shuffle in local shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "10") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + val localReads = collect(adaptivePlan) { + case read: AQEShuffleReadExec if read.isLocalRead => read + } + assert(localReads.length == 2) + } + } + + testGluten("Reuse the default parallelism in local shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + val localReads = collect(adaptivePlan) { + case read: AQEShuffleReadExec if read.isLocalRead => read + } + assert(localReads.length == 2) + val localShuffleRDD0 = localReads(0) + .executeColumnar() + .asInstanceOf[ShuffledColumnarBatchRDD] + val localShuffleRDD1 = localReads(1) + .executeColumnar() + .asInstanceOf[ShuffledColumnarBatchRDD] + // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 + // and the partitions length is 2 * numMappers = 4 + assert(localShuffleRDD0.getPartitions.length == 4) + // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 + // and the partitions length is 2 * numMappers = 4 + assert(localShuffleRDD1.getPartitions.length == 4) + } + } + + testGluten("Empty stage coalesced to 1-partition RDD") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + val df1 = spark.range(10).withColumn("a", col("id")) + val df2 = spark.range(10).withColumn("b", col("id")) + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val testDf = df1 + .where(col("a") > 10) + .join(df2.where(col("b") > 10), Seq("id"), "left_outer") + .groupBy(col("a")) + .count() + checkAnswer(testDf, Seq()) + val plan = testDf.queryExecution.executedPlan + assert(find(plan)(_.isInstanceOf[SortMergeJoinExecTransformer]).isDefined) + } + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1") { + val testDf = df1 + .where(col("a") > 10) + .join(df2.where(col("b") > 10), Seq("id"), "left_outer") + .groupBy(col("a")) + .count() + checkAnswer(testDf, Seq()) + val plan = testDf.queryExecution.executedPlan + assert(find(plan)(_.isInstanceOf[BroadcastHashJoinExecTransformerBase]).isDefined) + val coalescedReads = collect(plan) { case r: AQEShuffleReadExec => r } + assert(coalescedReads.length == 3, s"$plan") + coalescedReads.foreach(r => assert(r.isLocalRead || r.partitionSpecs.length == 1)) + } + } + } + + testGluten("Scalar subquery") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a " + + "where value = (SELECT max(a) from testData3)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + } + } + + testGluten("Scalar subquery in later stages") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a " + + "where (value + a) = (SELECT max(a) from testData3)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + + checkNumLocalShuffleReads(adaptivePlan) + } + } + + testGluten("multiple joins") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + """ + |WITH t4 AS ( + | SELECT * FROM lowercaseData t2 JOIN testData3 t3 ON t2.n = t3.a where t2.n = '1' + |) + |SELECT * FROM testData + |JOIN testData2 t2 ON key = t2.a + |JOIN t4 ON t2.b = t4.a + |WHERE value = 1 + """.stripMargin) + assert(sortMergeJoinSize(plan) == 3) + assert(broadcastHashJoinSize(adaptivePlan) == 3) + + // A possible resulting query plan: + // BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastExchange + // +-LocalShuffleReader* + // +- ShuffleExchange + + // After applied the 'OptimizeShuffleWithLocalRead' rule, we can convert all the four + // shuffle read to local shuffle read in the bottom two Symbol("b")roadcastHashJoin'. + // For the top level Symbol("b")roadcastHashJoin', the probe side is not shuffle query stage + // and the build side shuffle query stage is also converted to local shuffle read. + checkNumLocalShuffleReads(adaptivePlan, 0) + } + } + + testGluten("multiple joins with aggregate") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = + runAdaptiveAndVerifyResult(""" + |WITH t4 AS ( + | SELECT * FROM lowercaseData t2 JOIN ( + | select a, sum(b) from testData3 group by a + | ) t3 ON t2.n = t3.a where t2.n = '1' + |) + |SELECT * FROM testData + |JOIN testData2 t2 ON key = t2.a + |JOIN t4 ON t2.b = t4.a + |WHERE value = 1 + """.stripMargin) + assert(sortMergeJoinSize(plan) == 3) + assert(broadcastHashJoinSize(adaptivePlan) == 3) + + // A possible resulting query plan: + // BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastExchange + // +-HashAggregate + // +- CoalescedShuffleReader + // +- ShuffleExchange + + // The shuffle added by Aggregate can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 1) + } + } + + testGluten("multiple joins with aggregate 2") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") { + val (plan, adaptivePlan) = + runAdaptiveAndVerifyResult(""" + |WITH t4 AS ( + | SELECT * FROM lowercaseData t2 JOIN ( + | select a, max(b) b from testData2 group by a + | ) t3 ON t2.n = t3.b + |) + |SELECT * FROM testData + |JOIN testData2 t2 ON key = t2.a + |JOIN t4 ON value = t4.a + |WHERE value = 1 + """.stripMargin) + assert(sortMergeJoinSize(plan) == 3) + assert(broadcastHashJoinSize(adaptivePlan) == 3) + + // A possible resulting query plan: + // BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- Filter + // +- HashAggregate + // +- CoalescedShuffleReader + // +- ShuffleExchange + // +- BroadcastExchange + // +-LocalShuffleReader* + // +- ShuffleExchange + + // The shuffle added by Aggregate can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 1) + } + } + + testGluten("Exchange reuse") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "100", + SQLConf.SHUFFLE_PARTITIONS.key -> "5") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT value FROM testData join testData2 ON key = a " + + "join (SELECT value v from testData join testData3 ON key = a) on value = v") + assert(sortMergeJoinSize(plan) == 3) + // TODO: vanilla spark has 2 bhj, and 1 smj, but gluten has 3 bhj, + // make sure this will not cause performance regression and why it is bhj + assert(broadcastHashJoinSize(adaptivePlan) == 1) + // Vanilla spark still a SMJ, and its two shuffles can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 4) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.size == 1) + } + } + + testGluten("Exchange reuse with subqueries") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value = (SELECT max(a) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // // Even with local shuffle read, the query stage reuse can also work. + // gluten change the smj to bhj, stage is changed, so we cannot find the stage with old + // ReuseExchange from stageCache, then the reuse is removed + // https://github.com/apache/spark/pull/24706/ + // files#diff-ec42cd27662f3f528832c298a60fffa1d341feb04aa1d8c80044b70cbe0ebbfcR224 + // maybe vanilla spark should checkReuse rile again + // val ex = findReusedExchange(adaptivePlan) + // assert(ex.size == 1) + } + } + + testGluten("Exchange reuse across subqueries") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300", + SQLConf.SUBQUERY_REUSE_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value >= (SELECT max(a) from testData join testData2 ON key = a) " + + "and a <= (SELECT max(a) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.nonEmpty) + val sub = findReusedSubquery(adaptivePlan) + assert(sub.isEmpty) + } + } + + testGluten("Subquery reuse") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value >= (SELECT max(a) from testData join testData2 ON key = a) " + + "and a <= (SELECT max(a) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.isEmpty) + val sub = findReusedSubquery(adaptivePlan) + assert(sub.nonEmpty) + } + } + + testGluten("Broadcast exchange reuse across subqueries") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000", + SQLConf.SUBQUERY_REUSE_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value >= (" + + "SELECT /*+ broadcast(testData2) */ max(key) from testData join testData2 ON key = a) " + + "and a <= (" + + "SELECT /*+ broadcast(testData2) */ max(value) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.nonEmpty) + assert(ex.head.child.isInstanceOf[ColumnarBroadcastExchangeExec]) + val sub = findReusedSubquery(adaptivePlan) + assert(sub.isEmpty) + } + } + + // Cost is equal, not test cost is greater, need new test, but other test may contain cost change, + // so it maybe not essential + testGluten("Avoid plan change if cost is greater") {} + + testGluten("Change merge join to broadcast join without local shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.LOCAL_SHUFFLE_READER_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + """ + |SELECT * FROM testData t1 join testData2 t2 + |ON t1.key = t2.a join testData3 t3 on t2.a = t3.a + |where t1.value = 1 + """.stripMargin + ) + assert(sortMergeJoinSize(plan) == 2) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 2) + // There is still a SMJ, and its two shuffles can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 0) + } + } + + testGluten( + "Avoid changing merge join to broadcast join if too many empty partitions " + + "on build plan") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN.key -> "0.5", + // this config will make some empty partitions + SQLConf.SHUFFLE_PARTITIONS.key -> "5" + ) { + // `testData` is small enough to be broadcast but has empty partition ratio over the config. + // because testData2 in gluten sizeInBytes(from ColumnarShuffleExchangeExec plan stats) + // is 78B sometimes, so change the threshold from 80 to 60 + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "60") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + assert(sortMergeJoinSize(plan) == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.isEmpty) + } + // It is still possible to broadcast `testData2`. + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "2000") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + assert(sortMergeJoinSize(plan) == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + assert(bhj.head.joinBuildSide == BuildRight) + } + } + } + + testGluten("SPARK-30524: Do not optimize skew join if introduce additional shuffle") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "100", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "100" + ) { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 3 as key1", "id as value1") + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 1 as key2", "id as value2") + .createOrReplaceTempView("skewData2") + + def checkSkewJoin(query: String, optimizeSkewJoin: Boolean): Unit = { + val (_, innerAdaptivePlan) = runAdaptiveAndVerifyResult(query) + val innerSmj = findTopLevelSortMergeJoinTransform(innerAdaptivePlan) + assert(innerSmj.size == 1 && innerSmj.head.isSkewJoin == optimizeSkewJoin) + } + + // OptimizeSkewedJoin check the map status, because the + checkSkewJoin("SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2", true) + // Additional shuffle introduced, so disable the "OptimizeSkewedJoin" optimization + checkSkewJoin( + "SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2 GROUP BY key1", + false) + } + } + } + + testGluten("SPARK-29544: adaptive skew join with different join types") { + Seq("SHUFFLE_MERGE", "SHUFFLE_HASH").foreach { + joinHint => + def getJoinNode(plan: SparkPlan): Seq[BinaryExecNode] = if (joinHint == "SHUFFLE_MERGE") { + findTopLevelSortMergeJoinTransform(plan) + } else { + findTopLevelShuffledHashJoinTransform(plan) + } + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.SHUFFLE_PARTITIONS.key -> "100", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "800", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "800" + ) { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .select( + when(col("id") < 250, 249) + .when(col("id") >= 750, 1000) + .otherwise(col("id")) + .as("key1"), + col("id").as("value1")) + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .select( + when(col("id") < 250, 249) + .otherwise(col("id")) + .as("key2"), + col("id").as("value2")) + .createOrReplaceTempView("skewData2") + + def checkSkewJoin( + joins: Seq[BinaryExecNode], + leftSkewNum: Int, + rightSkewNum: Int): Unit = { + assert(joins.size == 1) + joins.head match { + case s: SortMergeJoinExecTransformer => assert(s.isSkewJoin) + case g: ShuffledHashJoinExecTransformerBase => assert(g.isSkewJoin) + case _ => assert(false) + } + assert( + joins.head.left + .collect { case r: AQEShuffleReadExec => r } + .head + .partitionSpecs + .collect { case p: PartialReducerPartitionSpec => p.reducerIndex } + .distinct + .length == leftSkewNum) + assert( + joins.head.right + .collect { case r: AQEShuffleReadExec => r } + .head + .partitionSpecs + .collect { case p: PartialReducerPartitionSpec => p.reducerIndex } + .distinct + .length == rightSkewNum) + } + + // skewed inner join optimization + val (_, innerAdaptivePlan) = runAdaptiveAndVerifyResult( + s"SELECT /*+ $joinHint(skewData1) */ * FROM skewData1 " + + "JOIN skewData2 ON key1 = key2") + val inner = getJoinNode(innerAdaptivePlan) + // checkSkewJoin(inner, 2, 1) + + // skewed left outer join optimization + val (_, leftAdaptivePlan) = runAdaptiveAndVerifyResult( + s"SELECT /*+ $joinHint(skewData2) */ * FROM skewData1 " + + "LEFT OUTER JOIN skewData2 ON key1 = key2") + val leftJoin = getJoinNode(leftAdaptivePlan) + // checkSkewJoin(leftJoin, 2, 0) + + // skewed right outer join optimization + val (_, rightAdaptivePlan) = runAdaptiveAndVerifyResult( + s"SELECT /*+ $joinHint(skewData1) */ * FROM skewData1 " + + "RIGHT OUTER JOIN skewData2 ON key1 = key2") + val rightJoin = getJoinNode(rightAdaptivePlan) + // checkSkewJoin(rightJoin, 0, 1) + } + } + } + } + + testGluten("SPARK-34682: AQEShuffleReadExec operating on canonicalized plan") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult("SELECT key FROM testData GROUP BY key") + val reads = collect(adaptivePlan) { case r: AQEShuffleReadExec => r } + assert(reads.length == 1) + val read = reads.head + val c = read.canonicalized.asInstanceOf[AQEShuffleReadExec] + // we can't just call execute() because that has separate checks for canonicalized plans + val ex = intercept[IllegalStateException] { + val doExecute = PrivateMethod[Unit](Symbol("doExecuteColumnar")) + c.invokePrivate(doExecute()) + } + assert(ex.getMessage === "operating on canonicalized plan") + } + } + + testGluten("metrics of the shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.SHUFFLE_PARTITIONS.key -> "5") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult("SELECT key FROM testData GROUP BY key") + val reads = collect(adaptivePlan) { case r: AQEShuffleReadExec => r } + assert(reads.length == 1) + val read = reads.head + assert(!read.isLocalRead) + assert(!read.hasSkewedPartition) + assert(read.hasCoalescedPartition) + assert( + read.metrics.keys.toSeq.sorted == Seq( + "numCoalescedPartitions", + "numPartitions", + "partitionDataSize")) + assert(read.metrics("numCoalescedPartitions").value == 1) + assert(read.metrics("numPartitions").value == read.partitionSpecs.length) + assert(read.metrics("partitionDataSize").value > 0) + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val join = collect(adaptivePlan) { case j: BroadcastHashJoinExecTransformerBase => j }.head + assert(join.joinBuildSide == BuildLeft) + + val reads = collect(join.right) { case r: AQEShuffleReadExec => r } + assert(reads.length == 1) + val read = reads.head + assert(read.isLocalRead) + assert(read.metrics.keys.toSeq == Seq("numPartitions")) + assert(read.metrics("numPartitions").value == read.partitionSpecs.length) + } + + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SHUFFLE_PARTITIONS.key -> "100", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "800", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "1000" + ) { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .select( + when(col("id") < 250, 249) + .when(col("id") >= 750, 1000) + .otherwise(col("id")) + .as("key1"), + col("id").as("value1")) + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .select( + when(col("id") < 250, 249) + .otherwise(col("id")) + .as("key2"), + col("id").as("value2")) + .createOrReplaceTempView("skewData2") + val (_, adaptivePlan) = + runAdaptiveAndVerifyResult("SELECT * FROM skewData1 join skewData2 ON key1 = key2") + } + } + } + } + + testGluten("SPARK-32717: AQEOptimizer should respect excludedRules configuration") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> Long.MaxValue.toString, + // This test is a copy of test(SPARK-32573), in order to test the configuration + // `spark.sql.adaptive.optimizer.excludedRules` works as expect. + "spark.gluten.sql.columnar.backend.ch.aqe.propagate.empty.relation" -> "false" + ) { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData2 t1 WHERE t1.b NOT IN (SELECT b FROM testData3)") + val bhj = findTopLevelBroadcastHashJoin(plan) + assert(bhj.size == 1) + val join = findTopLevelBaseJoin(adaptivePlan) + // this is different compares to test(SPARK-32573) due to the rule + // `EliminateUnnecessaryJoin` has been excluded. + assert(join.nonEmpty) + checkNumLocalShuffleReads(adaptivePlan) + } + } + + // EmptyRelation case + ignore( + GLUTEN_TEST + "SPARK-35455: Unify empty relation optimization " + + "between normal and AQE optimizer - single join") {} + + testGluten("SPARK-32753: Only copy tags to node with no tags") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + withTempView("v1") { + spark.range(10).union(spark.range(10)).createOrReplaceTempView("v1") + + val (_, adaptivePlan) = + runAdaptiveAndVerifyResult("SELECT id FROM v1 GROUP BY id DISTRIBUTE BY id") + assert(collect(adaptivePlan) { case s: ColumnarShuffleExchangeExec => s }.length == 1) + } + } + } + + testGluten("Logging plan changes for AQE") { + val testAppender = new LogAppender("plan changes") + withLogAppender(testAppender) { + withSQLConf( + // this test default level is WARN, so we should check warn level + SQLConf.PLAN_CHANGE_LOG_LEVEL.key -> "WARN", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80" + ) { + sql( + "SELECT * FROM testData JOIN testData2 ON key = a " + + "WHERE value = (SELECT max(a) FROM testData3)").collect() + } + Seq( + "=== Result of Batch AQE Preparations ===", + "=== Result of Batch AQE Post Stage Creation ===", + "=== Result of Batch AQE Replanning ===", + "=== Result of Batch AQE Query Stage Optimization ===" + ).foreach { + expectedMsg => + assert( + testAppender.loggingEvents.exists( + _.getMessage.getFormattedMessage.contains(expectedMsg))) + } + } + } + + testGluten("SPARK-33551: Do not use AQE shuffle read for repartition") { + def hasRepartitionShuffle(plan: SparkPlan): Boolean = { + find(plan) { + case s: ShuffleExchangeLike => + s.shuffleOrigin == REPARTITION_BY_COL || s.shuffleOrigin == REPARTITION_BY_NUM + case _ => false + }.isDefined + } + + def checkBHJ( + df: Dataset[Row], + optimizeOutRepartition: Boolean, + probeSideLocalRead: Boolean, + probeSideCoalescedRead: Boolean): Unit = { + df.collect() + val plan = df.queryExecution.executedPlan + // There should be only one shuffle that can't do local read, which is either the top shuffle + // from repartition, or BHJ probe side shuffle. + checkNumLocalShuffleReads(plan, 1) + assert(hasRepartitionShuffle(plan) == !optimizeOutRepartition) + val bhj = findTopLevelBroadcastHashJoinTransform(plan) + assert(bhj.length == 1) + + // Build side should do local read. + val buildSide = find(bhj.head.left)(_.isInstanceOf[AQEShuffleReadExec]) + assert(buildSide.isDefined) + assert(buildSide.get.asInstanceOf[AQEShuffleReadExec].isLocalRead) + + val probeSide = find(bhj.head.right)(_.isInstanceOf[AQEShuffleReadExec]) + if (probeSideLocalRead || probeSideCoalescedRead) { + assert(probeSide.isDefined) + if (probeSideLocalRead) { + assert(probeSide.get.asInstanceOf[AQEShuffleReadExec].isLocalRead) + } else { + assert(probeSide.get.asInstanceOf[AQEShuffleReadExec].hasCoalescedPartition) + } + } else { + assert(probeSide.isEmpty) + } + } + + def checkSMJ( + df: Dataset[Row], + optimizeOutRepartition: Boolean, + optimizeSkewJoin: Boolean, + coalescedRead: Boolean): Unit = { + df.collect() + val plan = df.queryExecution.executedPlan + assert(hasRepartitionShuffle(plan) == !optimizeOutRepartition) + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.length == 1) + assert(smj.head.isSkewJoin == optimizeSkewJoin) + val aqeReads = collect(smj.head) { case c: AQEShuffleReadExec => c } + if (coalescedRead || optimizeSkewJoin) { + assert(aqeReads.length == 2) + if (coalescedRead) assert(aqeReads.forall(_.hasCoalescedPartition)) + } else { + assert(aqeReads.isEmpty) + } + } + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.SHUFFLE_PARTITIONS.key -> "5") { + val df = sql(""" + |SELECT * FROM ( + | SELECT * FROM testData WHERE key = 1 + |) + |RIGHT OUTER JOIN testData2 + |ON value = b + """.stripMargin) + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + // Repartition with no partition num specified. + checkBHJ( + df.repartition(col("b")), + // The top shuffle from repartition is optimized out. + optimizeOutRepartition = true, + probeSideLocalRead = false, + probeSideCoalescedRead = true + ) + + // Repartition with default partition num (5 in test env) specified. + checkBHJ( + df.repartition(5, col("b")), + // The top shuffle from repartition is optimized out + // The final plan must have 5 partitions, no optimization can be made to the probe side. + optimizeOutRepartition = true, + probeSideLocalRead = false, + probeSideCoalescedRead = false + ) + + // Repartition with non-default partition num specified. + checkBHJ( + df.repartition(4, col("b")), + // The top shuffle from repartition is not optimized out + optimizeOutRepartition = false, + probeSideLocalRead = true, + probeSideCoalescedRead = true + ) + + // Repartition by col and project away the partition cols + checkBHJ( + df.repartition(col("b")).select(col("key")), + // The top shuffle from repartition is not optimized out + optimizeOutRepartition = false, + probeSideLocalRead = true, + probeSideCoalescedRead = true + ) + } + + // Force skew join + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SKEW_JOIN_ENABLED.key -> "true", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "1", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_FACTOR.key -> "0", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "10" + ) { + // Repartition with no partition num specified. + checkSMJ( + df.repartition(col("b")), + // The top shuffle from repartition is optimized out. + optimizeOutRepartition = true, + optimizeSkewJoin = false, + coalescedRead = true + ) + + // Repartition with default partition num (5 in test env) specified. + checkSMJ( + df.repartition(5, col("b")), + // The top shuffle from repartition is optimized out. + // The final plan must have 5 partitions, can't do coalesced read. + optimizeOutRepartition = true, + optimizeSkewJoin = false, + coalescedRead = false + ) + + // Repartition with non-default partition num specified. + checkSMJ( + df.repartition(4, col("b")), + // The top shuffle from repartition is not optimized out. + optimizeOutRepartition = false, + optimizeSkewJoin = true, + coalescedRead = false + ) + + // Repartition by col and project away the partition cols + checkSMJ( + df.repartition(col("b")).select(col("key")), + // The top shuffle from repartition is not optimized out. + optimizeOutRepartition = false, + optimizeSkewJoin = true, + coalescedRead = false + ) + } + } + } + + testGluten("SPARK-34091: Batch shuffle fetch in AQE partition coalescing") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.SHUFFLE_PARTITIONS.key -> "10", + SQLConf.FETCH_SHUFFLE_BLOCKS_IN_BATCH.key -> "true") { + withTable("t1") { + spark.range(100).selectExpr("id + 1 as a").write.format("parquet").saveAsTable("t1") + val query = "SELECT SUM(a) FROM t1 GROUP BY a" + val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) + val metricName = SQLShuffleReadMetricsReporter.LOCAL_BLOCKS_FETCHED + val blocksFetchedMetric = collectFirst(adaptivePlan) { + case p if p.metrics.contains(metricName) => p.metrics(metricName) + } + assert(blocksFetchedMetric.isDefined) + val blocksFetched = blocksFetchedMetric.get.value + withSQLConf(SQLConf.FETCH_SHUFFLE_BLOCKS_IN_BATCH.key -> "false") { + val (_, adaptivePlan2) = runAdaptiveAndVerifyResult(query) + val blocksFetchedMetric2 = collectFirst(adaptivePlan2) { + case p if p.metrics.contains(metricName) => p.metrics(metricName) + } + assert(blocksFetchedMetric2.isDefined) + val blocksFetched2 = blocksFetchedMetric2.get.value + assert(blocksFetched == blocksFetched2) + } + } + } + } + + testGluten("SPARK-34899: Use origin plan if we can not coalesce shuffle partition") { + def checkNoCoalescePartitions(ds: Dataset[Row], origin: ShuffleOrigin): Unit = { + assert(collect(ds.queryExecution.executedPlan) { + case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s + }.size == 1) + ds.collect() + val plan = ds.queryExecution.executedPlan + assert(collect(plan) { + case s: ColumnarShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => + s + }.size == 1) + checkAnswer(ds, testData) + } + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + // Pick a small value so that no coalesce can happen. + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "100", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.SHUFFLE_PARTITIONS.key -> "2" + ) { + val df = + spark.sparkContext.parallelize((1 to 100).map(i => TestData(i, i.toString)), 10).toDF() + + // partition size [1420, 1420] + checkNoCoalescePartitions(df.repartition($"key"), REPARTITION_BY_COL) + // partition size [1140, 1119] + checkNoCoalescePartitions(df.sort($"key"), ENSURE_REQUIREMENTS) + } + } + + testGluten("SPARK-35239: Coalesce shuffle partition should handle empty input RDD") { + withTable("t") { + withSQLConf( + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.SHUFFLE_PARTITIONS.key -> "2", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + spark.sql("CREATE TABLE t (c1 int) USING PARQUET") + val (_, adaptive) = runAdaptiveAndVerifyResult("SELECT c1, count(*) FROM t GROUP BY c1") + assert( + collect(adaptive) { + case c @ AQEShuffleReadExec(_, partitionSpecs) if partitionSpecs.length == 1 => + assert(c.hasCoalescedPartition) + c + }.length == 1 + ) + } + } + } + + testGluten("SPARK-35264: Support AQE side broadcastJoin threshold") { + withTempView("t1", "t2") { + def checkJoinStrategy(shouldBroadcast: Boolean): Unit = { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val (origin, adaptive) = + runAdaptiveAndVerifyResult("SELECT t1.c1, t2.c1 FROM t1 JOIN t2 ON t1.c1 = t2.c1") + assert(findTopLevelSortMergeJoin(origin).size == 1) + if (shouldBroadcast) { + assert(findTopLevelBroadcastHashJoinTransform(adaptive).size == 1) + } else { + assert(findTopLevelSortMergeJoinTransform(adaptive).size == 1) + } + } + } + + // t1: 1600 bytes + // t2: 160 bytes + spark.sparkContext + .parallelize((1 to 100).map(i => TestData(i, i.toString)), 10) + .toDF("c1", "c2") + .createOrReplaceTempView("t1") + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(i, i.toString)), 5) + .toDF("c1", "c2") + .createOrReplaceTempView("t2") + + checkJoinStrategy(false) + withSQLConf(SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + checkJoinStrategy(false) + } + + withSQLConf(SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "400") { + checkJoinStrategy(true) + } + } + } + + // table partition size is different with spark + testGluten("SPARK-35264: Support AQE side shuffled hash join formula") { + withTempView("t1", "t2") { + def checkJoinStrategy(shouldShuffleHashJoin: Boolean): Unit = { + Seq("100", "100000").foreach { + size => + withSQLConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> size) { + val (origin1, adaptive1) = + runAdaptiveAndVerifyResult("SELECT t1.c1, t2.c1 FROM t1 JOIN t2 ON t1.c1 = t2.c1") + assert(findTopLevelSortMergeJoin(origin1).size === 1) + if (shouldShuffleHashJoin && size.toInt < 100000) { + val shj = findTopLevelShuffledHashJoinTransform(adaptive1) + assert(shj.size === 1) + assert(shj.head.joinBuildSide == BuildRight) + } else { + assert(findTopLevelSortMergeJoinTransform(adaptive1).size === 1) + } + } + } + // respect user specified join hint + val (origin2, adaptive2) = runAdaptiveAndVerifyResult( + "SELECT /*+ MERGE(t1) */ t1.c1, t2.c1 FROM t1 JOIN t2 ON t1.c1 = t2.c1") + assert(findTopLevelSortMergeJoin(origin2).size === 1) + assert(findTopLevelSortMergeJoinTransform(adaptive2).size === 1) + } + + spark.sparkContext + .parallelize((1 to 100).map(i => TestData(i, i.toString)), 10) + .toDF("c1", "c2") + .createOrReplaceTempView("t1") + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(i, i.toString)), 5) + .toDF("c1", "c2") + .createOrReplaceTempView("t2") + + // t1 partition size: [395, 316, 313] + // t2 partition size: [140, 50, 0] + withSQLConf( + SQLConf.SHUFFLE_PARTITIONS.key -> "3", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.PREFER_SORTMERGEJOIN.key -> "true") { + // check default value + checkJoinStrategy(false) + // t1 no hint. + // t2 partition size are all smaller than 200, t2 has SHJ hint. The result is true. + withSQLConf(SQLConf.ADAPTIVE_MAX_SHUFFLE_HASH_JOIN_LOCAL_MAP_THRESHOLD.key -> "200") { + checkJoinStrategy(true) + } + // t1 no hint. + // Not all partition size of t2 are smaller than 100, t2 no hint. The result is false. + withSQLConf(SQLConf.ADAPTIVE_MAX_SHUFFLE_HASH_JOIN_LOCAL_MAP_THRESHOLD.key -> "100") { + checkJoinStrategy(false) + } + // t1, t2 partition size are all smaller than 1000, t1 and t2 can use SHJ. + // The result is true. + withSQLConf(SQLConf.ADAPTIVE_MAX_SHUFFLE_HASH_JOIN_LOCAL_MAP_THRESHOLD.key -> "1000") { + checkJoinStrategy(true) + } + } + } + } + + testGluten( + "SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of v2 write commands") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY.key -> "true") { + var plan: SparkPlan = null + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + plan = qe.executedPlan + } + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + withTable("t1") { + val format = classOf[NoopDataSource].getName + Seq((0, 1)).toDF("x", "y").write.format(format).mode("overwrite").save() + + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[V2TableWriteExec]) + val childPlan = plan.asInstanceOf[V2TableWriteExec].child + assert(childPlan.isInstanceOf[ColumnarToCarrierRowExecBase]) + assert( + childPlan + .asInstanceOf[ColumnarToCarrierRowExecBase] + .child + .isInstanceOf[AdaptiveSparkPlanExec]) + + spark.listenerManager.unregister(listener) + } + } + } + + testGluten("SPARK-35650: Coalesce number of partitions by AEQ") { + withSQLConf(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1") { + Seq("REPARTITION", "REBALANCE(key)") + .foreach { + repartition => + val query = s"SELECT /*+ $repartition */ * FROM testData" + val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) + collect(adaptivePlan) { case r: AQEShuffleReadExec => r } match { + case Seq(aqeShuffleRead) => + assert(aqeShuffleRead.partitionSpecs.size === 1) + assert(!aqeShuffleRead.isLocalRead) + case _ => + fail("There should be a AQEShuffleReadExec") + } + } + } + } + + testGluten("SPARK-35650: Use local shuffle read if can not coalesce number of partitions") { + withSQLConf(SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "false") { + val query = "SELECT /*+ REPARTITION */ * FROM testData" + val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) + collect(adaptivePlan) { case r: AQEShuffleReadExec => r } match { + case Seq(aqeShuffleRead) => + assert(aqeShuffleRead.partitionSpecs.size === 4) + assert(aqeShuffleRead.isLocalRead) + case _ => + fail("There should be a AQEShuffleReadExec") + } + } + } + + testGluten("SPARK-35725: Support optimize skewed partitions in RebalancePartitions") { + withTempView("v") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1" + ) { + + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(if (i > 4) 5 else i, i.toString)), 3) + .toDF("c1", "c2") + .createOrReplaceTempView("v") + + def checkPartitionNumber( + query: String, + skewedPartitionNumber: Int, + totalNumber: Int): Unit = { + val (_, adaptive) = runAdaptiveAndVerifyResult(query) + val read = collect(adaptive) { case read: AQEShuffleReadExec => read } + assert(read.size == 1) + assert( + read.head.partitionSpecs.count(_.isInstanceOf[PartialReducerPartitionSpec]) == + skewedPartitionNumber) + assert(read.head.partitionSpecs.size == totalNumber) + } + + // Changed ADVISORY_PARTITION_SIZE_IN_BYTES from 150 to 120 because Gluten has smaller + // partition size. + withSQLConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "120") { + // partition size [0,208,54,54,54] + checkPartitionNumber("SELECT /*+ REBALANCE(c1) */ * FROM v", 2, 4) + // partition size [108, 54, 60, 108, 54, 108, 54] + checkPartitionNumber("SELECT /*+ REBALANCE */ * FROM v", 6, 7) + } + + // no skewed partition should be optimized + withSQLConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "10000") { + checkPartitionNumber("SELECT /*+ REBALANCE(c1) */ * FROM v", 0, 1) + } + } + } + } + + testGluten("SPARK-35888: join with a 0-partition table") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + withTempView("t2") { + // create a temp view with 0 partition + spark + .createDataFrame(sparkContext.emptyRDD[Row], new StructType().add("b", IntegerType)) + .createOrReplaceTempView("t2") + val (_, adaptive) = + runAdaptiveAndVerifyResult("SELECT * FROM testData2 t1 left semi join t2 ON t1.a=t2.b") + val aqeReads = collect(adaptive) { case c: AQEShuffleReadExec => c } + assert(aqeReads.length == 2) + aqeReads.foreach { + c => + val stats = c.child.asInstanceOf[QueryStageExec].getRuntimeStatistics + assert(stats.sizeInBytes >= 0) + assert(stats.rowCount.get >= 0) + } + } + } + } + + testGluten("SPARK-35968: AQE coalescing should not produce too small partitions by default") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val (_, adaptive) = + runAdaptiveAndVerifyResult("SELECT sum(id) FROM RANGE(10) GROUP BY id % 3") + val coalesceRead = collect(adaptive) { + case r: AQEShuffleReadExec if r.hasCoalescedPartition => r + } + assert(coalesceRead.length == 1) + // RANGE(10) is a very small dataset and AQE coalescing should produce one partition. + assert(coalesceRead.head.partitionSpecs.length == 1) + } + } + + testGluten("SPARK-35794: Allow custom plugin for cost evaluator") { + CostEvaluator.instantiate( + classOf[SimpleShuffleSortCostEvaluator].getCanonicalName, + spark.sparkContext.getConf) + intercept[IllegalArgumentException] { + CostEvaluator.instantiate( + classOf[InvalidCostEvaluator].getCanonicalName, + spark.sparkContext.getConf) + } + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val query = "SELECT * FROM testData join testData2 ON key = a where value = '1'" + + withSQLConf( + SQLConf.ADAPTIVE_CUSTOM_COST_EVALUATOR_CLASS.key -> + "org.apache.spark.sql.execution.adaptive.SimpleShuffleSortCostEvaluator") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult(query) + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + checkNumLocalShuffleReads(adaptivePlan) + } + + withSQLConf( + SQLConf.ADAPTIVE_CUSTOM_COST_EVALUATOR_CLASS.key -> + "org.apache.spark.sql.execution.adaptive.InvalidCostEvaluator") { + intercept[IllegalArgumentException] { + runAdaptiveAndVerifyResult(query) + } + } + } + } + + testGluten("SPARK-36020: Check logical link in remove redundant projects") { + withTempView("t") { + spark + .range(10) + .selectExpr( + "id % 10 as key", + "cast(id * 2 as int) as a", + "cast(id * 3 as int) as b", + "array(id, id + 1, id + 3) as c") + .createOrReplaceTempView("t") + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "800") { + val query = + """ + |WITH tt AS ( + | SELECT key, a, b, explode(c) AS c FROM t + |) + |SELECT t1.key, t1.c, t2.key, t2.c + |FROM (SELECT a, b, c, key FROM tt WHERE a > 1) t1 + |JOIN (SELECT a, b, c, key FROM tt) t2 + | ON t1.key = t2.key + |""".stripMargin + val (origin, adaptive) = runAdaptiveAndVerifyResult(query) + assert(findTopLevelSortMergeJoin(origin).size == 1) + assert(findTopLevelBroadcastHashJoinTransform(adaptive).size == 1) + } + } + } + + testGluten( + "SPARK-36032: Use inputPlan instead of currentPhysicalPlan to initialize logical link") { + withTempView("v") { + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(i, i.toString)), 2) + .toDF("c1", "c2") + .createOrReplaceTempView("v") + + Seq("-1", "10000").foreach { + aqeBhj => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> aqeBhj, + SQLConf.SHUFFLE_PARTITIONS.key -> "1" + ) { + val (origin, adaptive) = runAdaptiveAndVerifyResult(""" + |SELECT * FROM v t1 JOIN ( + | SELECT c1 + 1 as c3 FROM v + |)t2 ON t1.c1 = t2.c3 + |SORT BY c1 + """.stripMargin) + if (aqeBhj.toInt < 0) { + // 1 sort since spark plan has no shuffle for SMJ + assert(findTopLevelSort(origin).size == 1) + // 2 sorts in SMJ + assert(findTopLevelSortTransform(adaptive).size == 2) + } else { + assert(findTopLevelSort(origin).size == 1) + // 1 sort at top node and BHJ has no sort + assert(findTopLevelSortTransform(adaptive).size == 1) + } + } + } + } + } + + testGluten("SPARK-37742: AQE reads invalid InMemoryRelation stats and mistakenly plans BHJ") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1048584", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + // Spark estimates a string column as 20 bytes so with 60k rows, these relations should be + // estimated at ~120m bytes which is greater than the broadcast join threshold. + val joinKeyOne = "00112233445566778899" + val joinKeyTwo = "11223344556677889900" + Seq + .fill(60000)(joinKeyOne) + .toDF("key") + .createOrReplaceTempView("temp") + Seq + .fill(60000)(joinKeyTwo) + .toDF("key") + .createOrReplaceTempView("temp2") + + Seq(joinKeyOne).toDF("key").createOrReplaceTempView("smallTemp") + spark.sql("SELECT key as newKey FROM temp").persist() + + // This query is trying to set up a situation where there are three joins. + // The first join will join the cached relation with a smaller relation. + // The first join is expected to be a broadcast join since the smaller relation will + // fit under the broadcast join threshold. + // The second join will join the first join with another relation and is expected + // to remain as a sort-merge join. + // The third join will join the cached relation with another relation and is expected + // to remain as a sort-merge join. + val query = + s""" + |SELECT t3.newKey + |FROM + | (SELECT t1.newKey + | FROM (SELECT key as newKey FROM temp) as t1 + | JOIN + | (SELECT key FROM smallTemp) as t2 + | ON t1.newKey = t2.key + | ) as t3 + | JOIN + | (SELECT key FROM temp2) as t4 + | ON t3.newKey = t4.key + |UNION + |SELECT t1.newKey + |FROM + | (SELECT key as newKey FROM temp) as t1 + | JOIN + | (SELECT key FROM temp2) as t2 + | ON t1.newKey = t2.key + |""".stripMargin + val df = spark.sql(query) + df.collect() + val adaptivePlan = df.queryExecution.executedPlan + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.length == 1) + } + } + + testGluten("test log level") { + def verifyLog(expectedLevel: Level): Unit = { + val logAppender = new LogAppender("adaptive execution") + logAppender.setThreshold(expectedLevel) + withLogAppender( + logAppender, + loggerNames = Seq(AdaptiveSparkPlanExec.getClass.getName.dropRight(1)), + level = Some(Level.TRACE)) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + sql("SELECT * FROM testData join testData2 ON key = a where value = '1'").collect() + } + } + Seq("Plan changed", "Final plan").foreach { + msg => + assert(logAppender.loggingEvents.exists { + event => + event.getMessage.getFormattedMessage.contains(msg) && event.getLevel == expectedLevel + }) + } + } + + // Verify default log level + verifyLog(Level.DEBUG) + + // Verify custom log level + val levels = Seq( + "TRACE" -> Level.TRACE, + "trace" -> Level.TRACE, + "DEBUG" -> Level.DEBUG, + "debug" -> Level.DEBUG, + "INFO" -> Level.INFO, + "info" -> Level.INFO, + "WARN" -> Level.WARN, + "warn" -> Level.WARN, + "ERROR" -> Level.ERROR, + "error" -> Level.ERROR, + "deBUG" -> Level.DEBUG + ) + + levels.foreach { + level => + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_LOG_LEVEL.key -> level._1) { + verifyLog(level._2) + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala new file mode 100644 index 000000000000..74d2835c4bd2 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/adaptive/velox/VeloxAdaptiveQueryExecSuite.scala @@ -0,0 +1,1551 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.adaptive.velox + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.{BroadcastHashJoinExecTransformerBase, ColumnarToCarrierRowExecBase, ShuffledHashJoinExecTransformerBase, SortExecTransformer, SortMergeJoinExecTransformer} + +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} +import org.apache.spark.sql.{Dataset, GlutenSQLTestsTrait, Row} +import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.datasources.noop.NoopDataSource +import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec +import org.apache.spark.sql.execution.exchange._ +import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter +import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.functions.when +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestData.TestData +import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.util.QueryExecutionListener + +import org.apache.logging.log4j.Level + +class VeloxAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQLTestsTrait { + import testImplicits._ + + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") + .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") + } + + private def runAdaptiveAndVerifyResult(query: String): (SparkPlan, SparkPlan) = { + var finalPlanCnt = 0 + val listener = new SparkListener { + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case SparkListenerSQLAdaptiveExecutionUpdate(_, _, sparkPlanInfo) => + if (sparkPlanInfo.simpleString.startsWith("AdaptiveSparkPlan isFinalPlan=true")) { + finalPlanCnt += 1 + } + case _ => // ignore other events + } + } + } + spark.sparkContext.addSparkListener(listener) + + val dfAdaptive = sql(query) + val planBefore = dfAdaptive.queryExecution.executedPlan + assert(planBefore.toString.startsWith("AdaptiveSparkPlan isFinalPlan=false")) + val result = dfAdaptive.collect() + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + val df = sql(query) + checkAnswer(df, result) + } + val planAfter = dfAdaptive.queryExecution.executedPlan + assert(planAfter.toString.startsWith("AdaptiveSparkPlan isFinalPlan=true")) + val adaptivePlan = planAfter.asInstanceOf[AdaptiveSparkPlanExec].executedPlan + + spark.sparkContext.listenerBus.waitUntilEmpty() + // AQE will post `SparkListenerSQLAdaptiveExecutionUpdate` twice in case of subqueries that + // exist out of query stages. + val expectedFinalPlanCnt = adaptivePlan.find(_.subqueries.nonEmpty).map(_ => 2).getOrElse(1) + assert(finalPlanCnt == expectedFinalPlanCnt) + spark.sparkContext.removeSparkListener(listener) + + val exchanges = adaptivePlan.collect { case e: Exchange => e } + assert(exchanges.isEmpty, "The final plan should not contain any Exchange node.") + (dfAdaptive.queryExecution.sparkPlan, adaptivePlan) + } + + private def broadcastHashJoinSize(plan: SparkPlan): Int = { + findTopLevelBroadcastHashJoinTransform(plan).size + findTopLevelBroadcastHashJoin(plan).size + } + + private def findTopLevelBroadcastHashJoinTransform( + plan: SparkPlan): Seq[BroadcastHashJoinExecTransformerBase] = { + collect(plan) { case j: BroadcastHashJoinExecTransformerBase => j } + } + + private def findTopLevelBroadcastHashJoin(plan: SparkPlan): Seq[BroadcastHashJoinExec] = { + collect(plan) { case j: BroadcastHashJoinExec => j } + } + + private def findTopLevelSortMergeJoin(plan: SparkPlan): Seq[SortMergeJoinExec] = { + collect(plan) { case j: SortMergeJoinExec => j } + } + + private def findTopLevelSortMergeJoinTransform( + plan: SparkPlan): Seq[SortMergeJoinExecTransformer] = { + collect(plan) { case j: SortMergeJoinExecTransformer => j } + } + + private def sortMergeJoinSize(plan: SparkPlan): Int = { + findTopLevelSortMergeJoinTransform(plan).size + findTopLevelSortMergeJoin(plan).size + } + + private def findTopLevelShuffledHashJoin(plan: SparkPlan): Seq[ShuffledHashJoinExec] = { + collect(plan) { case j: ShuffledHashJoinExec => j } + } + + private def findTopLevelShuffledHashJoinTransform( + plan: SparkPlan): Seq[ShuffledHashJoinExecTransformerBase] = { + collect(plan) { case j: ShuffledHashJoinExecTransformerBase => j } + } + + private def findTopLevelBaseJoin(plan: SparkPlan): Seq[BaseJoinExec] = { + collect(plan) { case j: BaseJoinExec => j } + } + + private def findTopLevelSort(plan: SparkPlan): Seq[SortExec] = { + collect(plan) { case s: SortExec => s } + } + + private def findTopLevelSortTransform(plan: SparkPlan): Seq[SortExecTransformer] = { + collect(plan) { case s: SortExecTransformer => s } + } + + private def findReusedExchange(plan: SparkPlan): Seq[ReusedExchangeExec] = { + collectWithSubqueries(plan) { + case ShuffleQueryStageExec(_, e: ReusedExchangeExec, _) => e + case BroadcastQueryStageExec(_, e: ReusedExchangeExec, _) => e + } + } + + private def findReusedSubquery(plan: SparkPlan): Seq[ReusedSubqueryExec] = { + collectWithSubqueries(plan) { case e: ReusedSubqueryExec => e } + } + + private def checkNumLocalShuffleReads( + plan: SparkPlan, + numShufflesWithoutLocalRead: Int = 0): Unit = { + val numShuffles = collect(plan) { case s: ShuffleQueryStageExec => s }.length + + val numLocalReads = collect(plan) { + case r: AQEShuffleReadExec if r.isLocalRead => r + } + // because columnar local reads cannot execute + numLocalReads.foreach { + r => + val rdd = r.executeColumnar() + val parts = rdd.partitions + assert(parts.forall(rdd.preferredLocations(_).nonEmpty)) + } + assert(numShuffles === (numLocalReads.length + numShufflesWithoutLocalRead)) + } + + testGluten("Change merge join to broadcast join") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300" + ) { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + checkNumLocalShuffleReads(adaptivePlan) + } + } + + testGluten("Change broadcast join to merge join") { + withTable("t1", "t2") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10000", + SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + sql("CREATE TABLE t1 USING PARQUET AS SELECT 1 c1") + sql("CREATE TABLE t2 USING PARQUET AS SELECT 1 c1") + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult(""" + |SELECT * FROM ( + | SELECT distinct c1 from t1 + | ) tmp1 JOIN ( + | SELECT distinct c1 from t2 + | ) tmp2 ON tmp1.c1 = tmp2.c1 + |""".stripMargin) + assert(broadcastHashJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 0) + assert(findTopLevelSortMergeJoinTransform(adaptivePlan).size == 1) + } + } + } + + testGluten("Reuse the parallelism of coalesced shuffle in local shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "10") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + val localReads = collect(adaptivePlan) { + case read: AQEShuffleReadExec if read.isLocalRead => read + } + assert(localReads.length == 2) + } + } + + testGluten("Reuse the default parallelism in local shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + val localReads = collect(adaptivePlan) { + case read: AQEShuffleReadExec if read.isLocalRead => read + } + assert(localReads.length == 2) + val localShuffleRDD0 = localReads(0) + .executeColumnar() + .asInstanceOf[ShuffledColumnarBatchRDD] + val localShuffleRDD1 = localReads(1) + .executeColumnar() + .asInstanceOf[ShuffledColumnarBatchRDD] + // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 + // and the partitions length is 2 * numMappers = 4 + assert(localShuffleRDD0.getPartitions.length == 4) + // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 + // and the partitions length is 2 * numMappers = 4 + assert(localShuffleRDD1.getPartitions.length == 4) + } + } + + testGluten("Empty stage coalesced to 1-partition RDD") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + val df1 = spark.range(10).withColumn("a", col("id")) + val df2 = spark.range(10).withColumn("b", col("id")) + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val testDf = df1 + .where(col("a") > 10) + .join(df2.where(col("b") > 10), Seq("id"), "left_outer") + .groupBy(col("a")) + .count() + checkAnswer(testDf, Seq()) + val plan = testDf.queryExecution.executedPlan + assert(find(plan)(_.isInstanceOf[SortMergeJoinExecTransformer]).isDefined) + } + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1") { + val testDf = df1 + .where(col("a") > 10) + .join(df2.where(col("b") > 10), Seq("id"), "left_outer") + .groupBy(col("a")) + .count() + checkAnswer(testDf, Seq()) + val plan = testDf.queryExecution.executedPlan + assert(find(plan)(_.isInstanceOf[BroadcastHashJoinExecTransformerBase]).isDefined) + val coalescedReads = collect(plan) { case r: AQEShuffleReadExec => r } + assert(coalescedReads.length == 3, s"$plan") + coalescedReads.foreach(r => assert(r.isLocalRead || r.partitionSpecs.length == 1)) + } + } + } + + testGluten("Scalar subquery") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a " + + "where value = (SELECT max(a) from testData3)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + } + } + + testGluten("Scalar subquery in later stages") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a " + + "where (value + a) = (SELECT max(a) from testData3)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + + checkNumLocalShuffleReads(adaptivePlan) + } + } + + testGluten("multiple joins") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + """ + |WITH t4 AS ( + | SELECT * FROM lowercaseData t2 JOIN testData3 t3 ON t2.n = t3.a where t2.n = '1' + |) + |SELECT * FROM testData + |JOIN testData2 t2 ON key = t2.a + |JOIN t4 ON t2.b = t4.a + |WHERE value = 1 + """.stripMargin) + assert(sortMergeJoinSize(plan) == 3) + assert(broadcastHashJoinSize(adaptivePlan) == 3) + + // A possible resulting query plan: + // BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastExchange + // +-LocalShuffleReader* + // +- ShuffleExchange + + // After applied the 'OptimizeShuffleWithLocalRead' rule, we can convert all the four + // shuffle read to local shuffle read in the bottom two Symbol("b")roadcastHashJoin'. + // For the top level Symbol("b")roadcastHashJoin', the probe side is not shuffle query stage + // and the build side shuffle query stage is also converted to local shuffle read. + checkNumLocalShuffleReads(adaptivePlan, 0) + } + } + + testGluten("multiple joins with aggregate") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = + runAdaptiveAndVerifyResult(""" + |WITH t4 AS ( + | SELECT * FROM lowercaseData t2 JOIN ( + | select a, sum(b) from testData3 group by a + | ) t3 ON t2.n = t3.a where t2.n = '1' + |) + |SELECT * FROM testData + |JOIN testData2 t2 ON key = t2.a + |JOIN t4 ON t2.b = t4.a + |WHERE value = 1 + """.stripMargin) + assert(sortMergeJoinSize(plan) == 3) + assert(broadcastHashJoinSize(adaptivePlan) == 3) + + // A possible resulting query plan: + // BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastExchange + // +-HashAggregate + // +- CoalescedShuffleReader + // +- ShuffleExchange + + // The shuffle added by Aggregate can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 1) + } + } + + testGluten("multiple joins with aggregate 2") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") { + val (plan, adaptivePlan) = + runAdaptiveAndVerifyResult(""" + |WITH t4 AS ( + | SELECT * FROM lowercaseData t2 JOIN ( + | select a, max(b) b from testData2 group by a + | ) t3 ON t2.n = t3.b + |) + |SELECT * FROM testData + |JOIN testData2 t2 ON key = t2.a + |JOIN t4 ON value = t4.a + |WHERE value = 1 + """.stripMargin) + assert(sortMergeJoinSize(plan) == 3) + assert(broadcastHashJoinSize(adaptivePlan) == 3) + + // A possible resulting query plan: + // BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- Filter + // +- HashAggregate + // +- CoalescedShuffleReader + // +- ShuffleExchange + // +- BroadcastExchange + // +-LocalShuffleReader* + // +- ShuffleExchange + + // The shuffle added by Aggregate can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 1) + } + } + + testGluten("Exchange reuse") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20" + ) { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT value FROM testData join testData2 ON key = a " + + "join (SELECT value v from testData join testData3 ON key = a) on value = v") + assert(sortMergeJoinSize(plan) == 3) + assert(broadcastHashJoinSize(adaptivePlan) == 2) + // There is still a SMJ, and its two shuffles can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 2) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.size == 1) + } + } + + testGluten("Exchange reuse with subqueries") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value = (SELECT max(a) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // // Even with local shuffle read, the query stage reuse can also work. + // gluten change the smj to bhj, stage is changed, so we cannot find the stage with old + // ReuseExchange from stageCache, then the reuse is removed + // https://github.com/apache/spark/pull/24706/ + // files#diff-ec42cd27662f3f528832c298a60fffa1d341feb04aa1d8c80044b70cbe0ebbfcR224 + // maybe vanilla spark should checkReuse rile again + // val ex = findReusedExchange(adaptivePlan) + // assert(ex.size == 1) + } + } + + testGluten("Exchange reuse across subqueries") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300", + SQLConf.SUBQUERY_REUSE_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value >= (SELECT max(a) from testData join testData2 ON key = a) " + + "and a <= (SELECT max(a) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.nonEmpty) + val sub = findReusedSubquery(adaptivePlan) + assert(sub.isEmpty) + } + } + + testGluten("Subquery reuse") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value >= (SELECT max(a) from testData join testData2 ON key = a) " + + "and a <= (SELECT max(a) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.isEmpty) + val sub = findReusedSubquery(adaptivePlan) + assert(sub.nonEmpty) + } + } + + testGluten("Broadcast exchange reuse across subqueries") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000", + SQLConf.SUBQUERY_REUSE_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value >= (" + + "SELECT /*+ broadcast(testData2) */ max(key) from testData join testData2 ON key = a) " + + "and a <= (" + + "SELECT /*+ broadcast(testData2) */ max(value) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.nonEmpty) + assert(ex.head.child.isInstanceOf[ColumnarBroadcastExchangeExec]) + val sub = findReusedSubquery(adaptivePlan) + assert(sub.isEmpty) + } + } + + // Cost is equal, not test cost is greater, need new test, but other test may contain cost change, + // so it maybe not essential + testGluten("Avoid plan change if cost is greater") {} + + testGluten("Change merge join to broadcast join without local shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.LOCAL_SHUFFLE_READER_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + """ + |SELECT * FROM testData t1 join testData2 t2 + |ON t1.key = t2.a join testData3 t3 on t2.a = t3.a + |where t1.value = 1 + """.stripMargin + ) + assert(sortMergeJoinSize(plan) == 2) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 2) + // There is still a SMJ, and its two shuffles can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 0) + } + } + + testGluten( + "Avoid changing merge join to broadcast join if too many empty partitions " + + "on build plan") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN.key -> "0.5", + // this config will make some empty partitions + SQLConf.SHUFFLE_PARTITIONS.key -> "5" + ) { + // `testData` is small enough to be broadcast but has empty partition ratio over the config. + // because testData2 in gluten sizeInBytes(from ColumnarShuffleExchangeExec plan stats) + // is 24B sometimes, so change the threshold from 80 to 20 + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + assert(sortMergeJoinSize(plan) == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.isEmpty) + } + // It is still possible to broadcast `testData2`. + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "2000") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + assert(sortMergeJoinSize(plan) == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + assert(bhj.head.joinBuildSide == BuildRight) + } + } + } + + testGluten("SPARK-30524: Do not optimize skew join if introduce additional shuffle") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "100", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "100" + ) { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 3 as key1", "id as value1") + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 1 as key2", "id as value2") + .createOrReplaceTempView("skewData2") + + def checkSkewJoin(query: String, optimizeSkewJoin: Boolean): Unit = { + val (_, innerAdaptivePlan) = runAdaptiveAndVerifyResult(query) + val innerSmj = findTopLevelSortMergeJoinTransform(innerAdaptivePlan) + assert(innerSmj.size == 1 && innerSmj.head.isSkewJoin == optimizeSkewJoin) + } + + // OptimizeSkewedJoin check the map status, because the + checkSkewJoin("SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2", true) + // Additional shuffle introduced, so disable the "OptimizeSkewedJoin" optimization + checkSkewJoin( + "SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2 GROUP BY key1", + false) + } + } + } + + testGluten("SPARK-29544: adaptive skew join with different join types") { + Seq("SHUFFLE_MERGE", "SHUFFLE_HASH").foreach { + joinHint => + def getJoinNode(plan: SparkPlan): Seq[BinaryExecNode] = if (joinHint == "SHUFFLE_MERGE") { + findTopLevelSortMergeJoinTransform(plan) + } else { + findTopLevelShuffledHashJoinTransform(plan) + } + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.SHUFFLE_PARTITIONS.key -> "100", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "800", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "800" + ) { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .select( + when(col("id") < 250, 249) + .when(col("id") >= 750, 1000) + .otherwise(col("id")) + .as("key1"), + col("id").as("value1")) + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .select( + when(col("id") < 250, 249) + .otherwise(col("id")) + .as("key2"), + col("id").as("value2")) + .createOrReplaceTempView("skewData2") + + def checkSkewJoin( + joins: Seq[BinaryExecNode], + leftSkewNum: Int, + rightSkewNum: Int): Unit = { + assert(joins.size == 1) + joins.head match { + case s: SortMergeJoinExecTransformer => assert(s.isSkewJoin) + case g: ShuffledHashJoinExecTransformerBase => assert(g.isSkewJoin) + case _ => assert(false) + } + assert( + joins.head.left + .collect { case r: AQEShuffleReadExec => r } + .head + .partitionSpecs + .collect { case p: PartialReducerPartitionSpec => p.reducerIndex } + .distinct + .length == leftSkewNum) + assert( + joins.head.right + .collect { case r: AQEShuffleReadExec => r } + .head + .partitionSpecs + .collect { case p: PartialReducerPartitionSpec => p.reducerIndex } + .distinct + .length == rightSkewNum) + } + + // skewed inner join optimization + val (_, innerAdaptivePlan) = runAdaptiveAndVerifyResult( + s"SELECT /*+ $joinHint(skewData1) */ * FROM skewData1 " + + "JOIN skewData2 ON key1 = key2") + val inner = getJoinNode(innerAdaptivePlan) + // checkSkewJoin(inner, 2, 1) + + // skewed left outer join optimization + val (_, leftAdaptivePlan) = runAdaptiveAndVerifyResult( + s"SELECT /*+ $joinHint(skewData2) */ * FROM skewData1 " + + "LEFT OUTER JOIN skewData2 ON key1 = key2") + val leftJoin = getJoinNode(leftAdaptivePlan) + // checkSkewJoin(leftJoin, 2, 0) + + // skewed right outer join optimization + val (_, rightAdaptivePlan) = runAdaptiveAndVerifyResult( + s"SELECT /*+ $joinHint(skewData1) */ * FROM skewData1 " + + "RIGHT OUTER JOIN skewData2 ON key1 = key2") + val rightJoin = getJoinNode(rightAdaptivePlan) + // checkSkewJoin(rightJoin, 0, 1) + } + } + } + } + + testGluten("SPARK-34682: AQEShuffleReadExec operating on canonicalized plan") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult("SELECT key FROM testData GROUP BY key") + val reads = collect(adaptivePlan) { case r: AQEShuffleReadExec => r } + assert(reads.length == 1) + val read = reads.head + val c = read.canonicalized.asInstanceOf[AQEShuffleReadExec] + // we can't just call execute() because that has separate checks for canonicalized plans + val ex = intercept[IllegalStateException] { + val doExecute = PrivateMethod[Unit](Symbol("doExecuteColumnar")) + c.invokePrivate(doExecute()) + } + assert(ex.getMessage === "operating on canonicalized plan") + } + } + + testGluten("metrics of the shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.SHUFFLE_PARTITIONS.key -> "5") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult("SELECT key FROM testData GROUP BY key") + val reads = collect(adaptivePlan) { case r: AQEShuffleReadExec => r } + assert(reads.length == 1) + val read = reads.head + assert(!read.isLocalRead) + assert(!read.hasSkewedPartition) + assert(read.hasCoalescedPartition) + assert( + read.metrics.keys.toSeq.sorted == Seq( + "numCoalescedPartitions", + "numPartitions", + "partitionDataSize")) + assert(read.metrics("numCoalescedPartitions").value == 1) + assert(read.metrics("numPartitions").value == read.partitionSpecs.length) + assert(read.metrics("partitionDataSize").value > 0) + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val join = collect(adaptivePlan) { case j: BroadcastHashJoinExecTransformerBase => j }.head + assert(join.joinBuildSide == BuildLeft) + + val reads = collect(join.right) { case r: AQEShuffleReadExec => r } + assert(reads.length == 1) + val read = reads.head + assert(read.isLocalRead) + assert(read.metrics.keys.toSeq == Seq("numPartitions")) + assert(read.metrics("numPartitions").value == read.partitionSpecs.length) + } + + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SHUFFLE_PARTITIONS.key -> "100", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "800", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "1000" + ) { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .select( + when(col("id") < 250, 249) + .when(col("id") >= 750, 1000) + .otherwise(col("id")) + .as("key1"), + col("id").as("value1")) + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .select( + when(col("id") < 250, 249) + .otherwise(col("id")) + .as("key2"), + col("id").as("value2")) + .createOrReplaceTempView("skewData2") + val (_, adaptivePlan) = + runAdaptiveAndVerifyResult("SELECT * FROM skewData1 join skewData2 ON key1 = key2") + } + } + } + } + + // because gluten use columnar format, which cannot execute to get rowIterator, then get the key + // null status + ignore( + GLUTEN_TEST + "SPARK-32573: Eliminate NAAJ when BuildSide is HashedRelationWithAllNullKeys") {} + + // EmptyRelation case + ignore( + GLUTEN_TEST + "SPARK-35455: Unify empty relation optimization " + + "between normal and AQE optimizer - single join") {} + + testGluten("SPARK-32753: Only copy tags to node with no tags") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + withTempView("v1") { + spark.range(10).union(spark.range(10)).createOrReplaceTempView("v1") + + val (_, adaptivePlan) = + runAdaptiveAndVerifyResult("SELECT id FROM v1 GROUP BY id DISTRIBUTE BY id") + assert(collect(adaptivePlan) { case s: ColumnarShuffleExchangeExec => s }.length == 1) + } + } + } + + testGluten("Logging plan changes for AQE") { + val testAppender = new LogAppender("plan changes") + withLogAppender(testAppender) { + withSQLConf( + // this test default level is WARN, so we should check warn level + SQLConf.PLAN_CHANGE_LOG_LEVEL.key -> "WARN", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80" + ) { + sql( + "SELECT * FROM testData JOIN testData2 ON key = a " + + "WHERE value = (SELECT max(a) FROM testData3)").collect() + } + Seq( + "=== Result of Batch AQE Preparations ===", + "=== Result of Batch AQE Post Stage Creation ===", + "=== Result of Batch AQE Replanning ===", + "=== Result of Batch AQE Query Stage Optimization ===" + ).foreach { + expectedMsg => + assert( + testAppender.loggingEvents.exists( + _.getMessage.getFormattedMessage.contains(expectedMsg))) + } + } + } + + testGluten("SPARK-33551: Do not use AQE shuffle read for repartition") { + def hasRepartitionShuffle(plan: SparkPlan): Boolean = { + find(plan) { + case s: ShuffleExchangeLike => + s.shuffleOrigin == REPARTITION_BY_COL || s.shuffleOrigin == REPARTITION_BY_NUM + case _ => false + }.isDefined + } + + def checkBHJ( + df: Dataset[Row], + optimizeOutRepartition: Boolean, + probeSideLocalRead: Boolean, + probeSideCoalescedRead: Boolean): Unit = { + df.collect() + val plan = df.queryExecution.executedPlan + // There should be only one shuffle that can't do local read, which is either the top shuffle + // from repartition, or BHJ probe side shuffle. + checkNumLocalShuffleReads(plan, 1) + assert(hasRepartitionShuffle(plan) == !optimizeOutRepartition) + val bhj = findTopLevelBroadcastHashJoinTransform(plan) + assert(bhj.length == 1) + + // Build side should do local read. + val buildSide = find(bhj.head.left)(_.isInstanceOf[AQEShuffleReadExec]) + assert(buildSide.isDefined) + assert(buildSide.get.asInstanceOf[AQEShuffleReadExec].isLocalRead) + + val probeSide = find(bhj.head.right)(_.isInstanceOf[AQEShuffleReadExec]) + if (probeSideLocalRead || probeSideCoalescedRead) { + assert(probeSide.isDefined) + if (probeSideLocalRead) { + assert(probeSide.get.asInstanceOf[AQEShuffleReadExec].isLocalRead) + } else { + assert(probeSide.get.asInstanceOf[AQEShuffleReadExec].hasCoalescedPartition) + } + } else { + assert(probeSide.isEmpty) + } + } + + def checkSMJ( + df: Dataset[Row], + optimizeOutRepartition: Boolean, + optimizeSkewJoin: Boolean, + coalescedRead: Boolean): Unit = { + df.collect() + val plan = df.queryExecution.executedPlan + assert(hasRepartitionShuffle(plan) == !optimizeOutRepartition) + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.length == 1) + assert(smj.head.isSkewJoin == optimizeSkewJoin) + val aqeReads = collect(smj.head) { case c: AQEShuffleReadExec => c } + if (coalescedRead || optimizeSkewJoin) { + assert(aqeReads.length == 2) + if (coalescedRead) assert(aqeReads.forall(_.hasCoalescedPartition)) + } else { + assert(aqeReads.isEmpty) + } + } + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.SHUFFLE_PARTITIONS.key -> "5") { + val df = sql(""" + |SELECT * FROM ( + | SELECT * FROM testData WHERE key = 1 + |) + |RIGHT OUTER JOIN testData2 + |ON value = b + """.stripMargin) + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + // Repartition with no partition num specified. + checkBHJ( + df.repartition(col("b")), + // The top shuffle from repartition is optimized out. + optimizeOutRepartition = true, + probeSideLocalRead = false, + probeSideCoalescedRead = true + ) + + // Repartition with default partition num (5 in test env) specified. + checkBHJ( + df.repartition(5, col("b")), + // The top shuffle from repartition is optimized out + // The final plan must have 5 partitions, no optimization can be made to the probe side. + optimizeOutRepartition = true, + probeSideLocalRead = false, + probeSideCoalescedRead = false + ) + + // Repartition with non-default partition num specified. + checkBHJ( + df.repartition(4, col("b")), + // The top shuffle from repartition is not optimized out + optimizeOutRepartition = false, + probeSideLocalRead = true, + probeSideCoalescedRead = true + ) + + // Repartition by col and project away the partition cols + checkBHJ( + df.repartition(col("b")).select(col("key")), + // The top shuffle from repartition is not optimized out + optimizeOutRepartition = false, + probeSideLocalRead = true, + probeSideCoalescedRead = true + ) + } + + // Force skew join + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SKEW_JOIN_ENABLED.key -> "true", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "1", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_FACTOR.key -> "0", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "10" + ) { + // Repartition with no partition num specified. + checkSMJ( + df.repartition(col("b")), + // The top shuffle from repartition is optimized out. + optimizeOutRepartition = true, + optimizeSkewJoin = false, + coalescedRead = true + ) + + // Repartition with default partition num (5 in test env) specified. + checkSMJ( + df.repartition(5, col("b")), + // The top shuffle from repartition is optimized out. + // The final plan must have 5 partitions, can't do coalesced read. + optimizeOutRepartition = true, + optimizeSkewJoin = false, + coalescedRead = false + ) + + // Repartition with non-default partition num specified. + checkSMJ( + df.repartition(4, col("b")), + // The top shuffle from repartition is not optimized out. + optimizeOutRepartition = false, + optimizeSkewJoin = true, + coalescedRead = false + ) + + // Repartition by col and project away the partition cols + checkSMJ( + df.repartition(col("b")).select(col("key")), + // The top shuffle from repartition is not optimized out. + optimizeOutRepartition = false, + optimizeSkewJoin = true, + coalescedRead = false + ) + } + } + } + + testGluten("SPARK-34091: Batch shuffle fetch in AQE partition coalescing") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.SHUFFLE_PARTITIONS.key -> "10", + SQLConf.FETCH_SHUFFLE_BLOCKS_IN_BATCH.key -> "true") { + withTable("t1") { + spark.range(100).selectExpr("id + 1 as a").write.format("parquet").saveAsTable("t1") + val query = "SELECT SUM(a) FROM t1 GROUP BY a" + val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) + val metricName = SQLShuffleReadMetricsReporter.LOCAL_BLOCKS_FETCHED + val blocksFetchedMetric = collectFirst(adaptivePlan) { + case p if p.metrics.contains(metricName) => p.metrics(metricName) + } + assert(blocksFetchedMetric.isDefined) + val blocksFetched = blocksFetchedMetric.get.value + withSQLConf(SQLConf.FETCH_SHUFFLE_BLOCKS_IN_BATCH.key -> "false") { + val (_, adaptivePlan2) = runAdaptiveAndVerifyResult(query) + val blocksFetchedMetric2 = collectFirst(adaptivePlan2) { + case p if p.metrics.contains(metricName) => p.metrics(metricName) + } + assert(blocksFetchedMetric2.isDefined) + val blocksFetched2 = blocksFetchedMetric2.get.value + assert(blocksFetched == blocksFetched2) + } + } + } + } + + testGluten("SPARK-34899: Use origin plan if we can not coalesce shuffle partition") { + def checkNoCoalescePartitions(ds: Dataset[Row], origin: ShuffleOrigin): Unit = { + assert(collect(ds.queryExecution.executedPlan) { + case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s + }.size == 1) + ds.collect() + val plan = ds.queryExecution.executedPlan + assert(collect(plan) { + case s: ColumnarShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => + s + }.size == 1) + checkAnswer(ds, testData) + } + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + // Pick a small value so that no coalesce can happen. + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "100", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.SHUFFLE_PARTITIONS.key -> "2" + ) { + val df = + spark.sparkContext.parallelize((1 to 100).map(i => TestData(i, i.toString)), 10).toDF() + + // partition size [1420, 1420] + checkNoCoalescePartitions(df.repartition($"key"), REPARTITION_BY_COL) + // partition size [1140, 1119] + checkNoCoalescePartitions(df.sort($"key"), ENSURE_REQUIREMENTS) + } + } + + testGluten("SPARK-35239: Coalesce shuffle partition should handle empty input RDD") { + withTable("t") { + withSQLConf( + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.SHUFFLE_PARTITIONS.key -> "2", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + spark.sql("CREATE TABLE t (c1 int) USING PARQUET") + val (_, adaptive) = runAdaptiveAndVerifyResult("SELECT c1, count(*) FROM t GROUP BY c1") + assert( + collect(adaptive) { + case c @ AQEShuffleReadExec(_, partitionSpecs) if partitionSpecs.length == 1 => + assert(c.hasCoalescedPartition) + c + }.length == 1 + ) + } + } + } + + testGluten("SPARK-35264: Support AQE side broadcastJoin threshold") { + withTempView("t1", "t2") { + def checkJoinStrategy(shouldBroadcast: Boolean): Unit = { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val (origin, adaptive) = + runAdaptiveAndVerifyResult("SELECT t1.c1, t2.c1 FROM t1 JOIN t2 ON t1.c1 = t2.c1") + assert(findTopLevelSortMergeJoin(origin).size == 1) + if (shouldBroadcast) { + assert(findTopLevelBroadcastHashJoinTransform(adaptive).size == 1) + } else { + assert(findTopLevelSortMergeJoinTransform(adaptive).size == 1) + } + } + } + + // t1: 1600 bytes + // t2: 160 bytes + spark.sparkContext + .parallelize((1 to 100).map(i => TestData(i, i.toString)), 10) + .toDF("c1", "c2") + .createOrReplaceTempView("t1") + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(i, i.toString)), 5) + .toDF("c1", "c2") + .createOrReplaceTempView("t2") + + checkJoinStrategy(false) + withSQLConf(SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + checkJoinStrategy(false) + } + + withSQLConf(SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "400") { + checkJoinStrategy(true) + } + } + } + + // table partition size is different with spark + testGluten("SPARK-35264: Support AQE side shuffled hash join formula") { + withTempView("t1", "t2") { + def checkJoinStrategy(shouldShuffleHashJoin: Boolean): Unit = { + Seq("100", "100000").foreach { + size => + withSQLConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> size) { + val (origin1, adaptive1) = + runAdaptiveAndVerifyResult("SELECT t1.c1, t2.c1 FROM t1 JOIN t2 ON t1.c1 = t2.c1") + assert(findTopLevelSortMergeJoin(origin1).size === 1) + if (shouldShuffleHashJoin && size.toInt < 100000) { + val shj = findTopLevelShuffledHashJoinTransform(adaptive1) + assert(shj.size === 1) + assert(shj.head.joinBuildSide == BuildRight) + } else { + assert(findTopLevelSortMergeJoinTransform(adaptive1).size === 1) + } + } + } + // respect user specified join hint + val (origin2, adaptive2) = runAdaptiveAndVerifyResult( + "SELECT /*+ MERGE(t1) */ t1.c1, t2.c1 FROM t1 JOIN t2 ON t1.c1 = t2.c1") + assert(findTopLevelSortMergeJoin(origin2).size === 1) + assert(findTopLevelSortMergeJoinTransform(adaptive2).size === 1) + } + + spark.sparkContext + .parallelize((1 to 100).map(i => TestData(i, i.toString)), 10) + .toDF("c1", "c2") + .createOrReplaceTempView("t1") + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(i, i.toString)), 5) + .toDF("c1", "c2") + .createOrReplaceTempView("t2") + + // t1 partition size: [395, 316, 313] + // t2 partition size: [140, 50, 0] + withSQLConf( + SQLConf.SHUFFLE_PARTITIONS.key -> "3", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.PREFER_SORTMERGEJOIN.key -> "true") { + // check default value + checkJoinStrategy(false) + // t1 no hint. + // t2 partition size are all smaller than 200, t2 has SHJ hint. The result is true. + withSQLConf(SQLConf.ADAPTIVE_MAX_SHUFFLE_HASH_JOIN_LOCAL_MAP_THRESHOLD.key -> "200") { + checkJoinStrategy(true) + } + // t1 no hint. + // Not all partition size of t2 are smaller than 100, t2 no hint. The result is false. + withSQLConf(SQLConf.ADAPTIVE_MAX_SHUFFLE_HASH_JOIN_LOCAL_MAP_THRESHOLD.key -> "100") { + checkJoinStrategy(false) + } + // t1, t2 partition size are all smaller than 1000, t1 and t2 can use SHJ. + // The result is true. + withSQLConf(SQLConf.ADAPTIVE_MAX_SHUFFLE_HASH_JOIN_LOCAL_MAP_THRESHOLD.key -> "1000") { + checkJoinStrategy(true) + } + } + } + } + + testGluten( + "SPARK-30953: InsertAdaptiveSparkPlan should apply AQE on child plan of v2 write commands") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY.key -> "true") { + var plan: SparkPlan = null + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + plan = qe.executedPlan + } + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + withTable("t1") { + val format = classOf[NoopDataSource].getName + Seq((0, 1)).toDF("x", "y").write.format(format).mode("overwrite").save() + + sparkContext.listenerBus.waitUntilEmpty() + assert(plan.isInstanceOf[V2TableWriteExec]) + val childPlan = plan.asInstanceOf[V2TableWriteExec].child + assert(childPlan.isInstanceOf[ColumnarToCarrierRowExecBase]) + assert( + childPlan + .asInstanceOf[ColumnarToCarrierRowExecBase] + .child + .isInstanceOf[AdaptiveSparkPlanExec]) + + spark.listenerManager.unregister(listener) + } + } + } + + testGluten("SPARK-35650: Coalesce number of partitions by AEQ") { + withSQLConf(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1") { + Seq("REPARTITION", "REBALANCE(key)") + .foreach { + repartition => + val query = s"SELECT /*+ $repartition */ * FROM testData" + val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) + collect(adaptivePlan) { case r: AQEShuffleReadExec => r } match { + case Seq(aqeShuffleRead) => + assert(aqeShuffleRead.partitionSpecs.size === 1) + assert(!aqeShuffleRead.isLocalRead) + case _ => + fail("There should be a AQEShuffleReadExec") + } + } + } + } + + testGluten("SPARK-35650: Use local shuffle read if can not coalesce number of partitions") { + withSQLConf(SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "false") { + val query = "SELECT /*+ REPARTITION */ * FROM testData" + val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) + collect(adaptivePlan) { case r: AQEShuffleReadExec => r } match { + case Seq(aqeShuffleRead) => + assert(aqeShuffleRead.partitionSpecs.size === 4) + assert(aqeShuffleRead.isLocalRead) + case _ => + fail("There should be a AQEShuffleReadExec") + } + } + } + + testGluten("SPARK-35725: Support optimize skewed partitions in RebalancePartitions") { + withTempView("v") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1" + ) { + + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(if (i > 4) 5 else i, i.toString)), 3) + .toDF("c1", "c2") + .createOrReplaceTempView("v") + + def checkPartitionNumber( + query: String, + skewedPartitionNumber: Int, + totalNumber: Int): Unit = { + val (_, adaptive) = runAdaptiveAndVerifyResult(query) + val read = collect(adaptive) { case read: AQEShuffleReadExec => read } + assert(read.size == 1) + assert( + read.head.partitionSpecs.count(_.isInstanceOf[PartialReducerPartitionSpec]) == + skewedPartitionNumber) + assert(read.head.partitionSpecs.size == totalNumber) + } + + // Changed ADVISORY_PARTITION_SIZE_IN_BYTES from 150 to 120 because Gluten has smaller + // partition size. + withSQLConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "120") { + // partition size [0,208,54,54,54] + checkPartitionNumber("SELECT /*+ REBALANCE(c1) */ * FROM v", 2, 4) + // partition size [108, 54, 60, 108, 54, 108, 54] + checkPartitionNumber("SELECT /*+ REBALANCE */ * FROM v", 6, 7) + } + + // no skewed partition should be optimized + withSQLConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "10000") { + checkPartitionNumber("SELECT /*+ REBALANCE(c1) */ * FROM v", 0, 1) + } + } + } + } + + testGluten("SPARK-35888: join with a 0-partition table") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + withTempView("t2") { + // create a temp view with 0 partition + spark + .createDataFrame(sparkContext.emptyRDD[Row], new StructType().add("b", IntegerType)) + .createOrReplaceTempView("t2") + val (_, adaptive) = + runAdaptiveAndVerifyResult("SELECT * FROM testData2 t1 left semi join t2 ON t1.a=t2.b") + val aqeReads = collect(adaptive) { case c: AQEShuffleReadExec => c } + assert(aqeReads.length == 2) + aqeReads.foreach { + c => + val stats = c.child.asInstanceOf[QueryStageExec].getRuntimeStatistics + assert(stats.sizeInBytes >= 0) + assert(stats.rowCount.get >= 0) + } + } + } + } + + testGluten("SPARK-35968: AQE coalescing should not produce too small partitions by default") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val (_, adaptive) = + runAdaptiveAndVerifyResult("SELECT sum(id) FROM RANGE(10) GROUP BY id % 3") + val coalesceRead = collect(adaptive) { + case r: AQEShuffleReadExec if r.hasCoalescedPartition => r + } + assert(coalesceRead.length == 1) + // RANGE(10) is a very small dataset and AQE coalescing should produce one partition. + assert(coalesceRead.head.partitionSpecs.length == 1) + } + } + + testGluten("SPARK-35794: Allow custom plugin for cost evaluator") { + CostEvaluator.instantiate( + classOf[SimpleShuffleSortCostEvaluator].getCanonicalName, + spark.sparkContext.getConf) + intercept[IllegalArgumentException] { + CostEvaluator.instantiate( + classOf[InvalidCostEvaluator].getCanonicalName, + spark.sparkContext.getConf) + } + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val query = "SELECT * FROM testData join testData2 ON key = a where value = '1'" + + withSQLConf( + SQLConf.ADAPTIVE_CUSTOM_COST_EVALUATOR_CLASS.key -> + "org.apache.spark.sql.execution.adaptive.SimpleShuffleSortCostEvaluator") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult(query) + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + checkNumLocalShuffleReads(adaptivePlan) + } + + withSQLConf( + SQLConf.ADAPTIVE_CUSTOM_COST_EVALUATOR_CLASS.key -> + "org.apache.spark.sql.execution.adaptive.InvalidCostEvaluator") { + intercept[IllegalArgumentException] { + runAdaptiveAndVerifyResult(query) + } + } + } + } + + testGluten("SPARK-36020: Check logical link in remove redundant projects") { + withTempView("t") { + spark + .range(10) + .selectExpr( + "id % 10 as key", + "cast(id * 2 as int) as a", + "cast(id * 3 as int) as b", + "array(id, id + 1, id + 3) as c") + .createOrReplaceTempView("t") + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "800") { + val query = + """ + |WITH tt AS ( + | SELECT key, a, b, explode(c) AS c FROM t + |) + |SELECT t1.key, t1.c, t2.key, t2.c + |FROM (SELECT a, b, c, key FROM tt WHERE a > 1) t1 + |JOIN (SELECT a, b, c, key FROM tt) t2 + | ON t1.key = t2.key + |""".stripMargin + val (origin, adaptive) = runAdaptiveAndVerifyResult(query) + assert(findTopLevelSortMergeJoin(origin).size == 1) + assert(findTopLevelBroadcastHashJoinTransform(adaptive).size == 1) + } + } + } + + testGluten( + "SPARK-36032: Use inputPlan instead of currentPhysicalPlan to initialize logical link") { + withTempView("v") { + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(i, i.toString)), 2) + .toDF("c1", "c2") + .createOrReplaceTempView("v") + + Seq("-1", "10000").foreach { + aqeBhj => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> aqeBhj, + SQLConf.SHUFFLE_PARTITIONS.key -> "1" + ) { + val (origin, adaptive) = runAdaptiveAndVerifyResult(""" + |SELECT * FROM v t1 JOIN ( + | SELECT c1 + 1 as c3 FROM v + |)t2 ON t1.c1 = t2.c3 + |SORT BY c1 + """.stripMargin) + if (aqeBhj.toInt < 0) { + // 1 sort since spark plan has no shuffle for SMJ + assert(findTopLevelSort(origin).size == 1) + // 2 sorts in SMJ + assert(findTopLevelSortTransform(adaptive).size == 2) + } else { + assert(findTopLevelSort(origin).size == 1) + // 1 sort at top node and BHJ has no sort + assert(findTopLevelSortTransform(adaptive).size == 1) + } + } + } + } + } + + testGluten("SPARK-37742: AQE reads invalid InMemoryRelation stats and mistakenly plans BHJ") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1048584", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + // Spark estimates a string column as 20 bytes so with 60k rows, these relations should be + // estimated at ~120m bytes which is greater than the broadcast join threshold. + val joinKeyOne = "00112233445566778899" + val joinKeyTwo = "11223344556677889900" + Seq + .fill(60000)(joinKeyOne) + .toDF("key") + .createOrReplaceTempView("temp") + Seq + .fill(60000)(joinKeyTwo) + .toDF("key") + .createOrReplaceTempView("temp2") + + Seq(joinKeyOne).toDF("key").createOrReplaceTempView("smallTemp") + spark.sql("SELECT key as newKey FROM temp").persist() + + // This query is trying to set up a situation where there are three joins. + // The first join will join the cached relation with a smaller relation. + // The first join is expected to be a broadcast join since the smaller relation will + // fit under the broadcast join threshold. + // The second join will join the first join with another relation and is expected + // to remain as a sort-merge join. + // The third join will join the cached relation with another relation and is expected + // to remain as a sort-merge join. + val query = + s""" + |SELECT t3.newKey + |FROM + | (SELECT t1.newKey + | FROM (SELECT key as newKey FROM temp) as t1 + | JOIN + | (SELECT key FROM smallTemp) as t2 + | ON t1.newKey = t2.key + | ) as t3 + | JOIN + | (SELECT key FROM temp2) as t4 + | ON t3.newKey = t4.key + |UNION + |SELECT t1.newKey + |FROM + | (SELECT key as newKey FROM temp) as t1 + | JOIN + | (SELECT key FROM temp2) as t2 + | ON t1.newKey = t2.key + |""".stripMargin + val df = spark.sql(query) + df.collect() + val adaptivePlan = df.queryExecution.executedPlan + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.length == 1) + } + } + + testGluten("test log level") { + def verifyLog(expectedLevel: Level): Unit = { + val logAppender = new LogAppender("adaptive execution") + logAppender.setThreshold(expectedLevel) + withLogAppender( + logAppender, + loggerNames = Seq(AdaptiveSparkPlanExec.getClass.getName.dropRight(1)), + level = Some(Level.TRACE)) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + sql("SELECT * FROM testData join testData2 ON key = a where value = '1'").collect() + } + } + Seq("Plan changed", "Final plan").foreach { + msg => + assert(logAppender.loggingEvents.exists { + event => + event.getMessage.getFormattedMessage.contains(msg) && event.getLevel == expectedLevel + }) + } + } + + // Verify default log level + verifyLog(Level.DEBUG) + + // Verify custom log level + val levels = Seq( + "TRACE" -> Level.TRACE, + "trace" -> Level.TRACE, + "DEBUG" -> Level.DEBUG, + "debug" -> Level.DEBUG, + "INFO" -> Level.INFO, + "info" -> Level.INFO, + "WARN" -> Level.WARN, + "warn" -> Level.WARN, + "ERROR" -> Level.ERROR, + "error" -> Level.ERROR, + "deBUG" -> Level.DEBUG + ) + + levels.foreach { + level => + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_LOG_LEVEL.key -> level._1) { + verifyLog(level._2) + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenBucketingUtilsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenBucketingUtilsSuite.scala new file mode 100644 index 000000000000..37a786e34c53 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenBucketingUtilsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenBucketingUtilsSuite extends BucketingUtilsSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenDataSourceStrategySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenDataSourceStrategySuite.scala new file mode 100644 index 000000000000..eeb63436c1e1 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenDataSourceStrategySuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceStrategySuite extends DataSourceStrategySuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenDataSourceSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenDataSourceSuite.scala new file mode 100644 index 000000000000..6435d17de2ab --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenDataSourceSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceSuite extends DataSourceSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileFormatWriterSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileFormatWriterSuite.scala new file mode 100644 index 000000000000..c0ba24f2be1f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileFormatWriterSuite.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest + +class GlutenFileFormatWriterSuite + extends FileFormatWriterSuite + with GlutenSQLTestsBaseTrait + with CodegenInterpretedPlanTest {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileIndexSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileIndexSuite.scala new file mode 100644 index 000000000000..c1c57eaa9145 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileIndexSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenFileIndexSuite extends FileIndexSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileMetadataStructSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileMetadataStructSuite.scala new file mode 100644 index 000000000000..ed347d024c1c --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileMetadataStructSuite.scala @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.{FileSourceScanExecTransformer, FilterExecTransformer} + +import org.apache.spark.sql.{Column, DataFrame, Row} +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructField, StructType} + +import java.io.File +import java.sql.Timestamp + +import scala.reflect.ClassTag + +class GlutenFileMetadataStructSuite extends FileMetadataStructSuite with GlutenSQLTestsBaseTrait { + + val schemaWithFilePathField: StructType = new StructType() + .add(StructField("file_path", StringType)) + .add(StructField("age", IntegerType)) + .add( + StructField( + "info", + new StructType() + .add(StructField("id", LongType)) + .add(StructField("university", StringType)))) + + private val METADATA_FILE_PATH = "_metadata.file_path" + private val METADATA_FILE_NAME = "_metadata.file_name" + private val METADATA_FILE_SIZE = "_metadata.file_size" + private val METADATA_FILE_MODIFICATION_TIME = "_metadata.file_modification_time" + + private def getMetadataForFile(f: File): Map[String, Any] = { + Map( + METADATA_FILE_PATH -> f.toURI.toString, + METADATA_FILE_NAME -> f.getName, + METADATA_FILE_SIZE -> f.length(), + METADATA_FILE_MODIFICATION_TIME -> new Timestamp(f.lastModified()) + ) + } + + private def metadataColumnsNativeTest(testName: String, fileSchema: StructType)( + f: (DataFrame, Map[String, Any], Map[String, Any]) => Unit): Unit = { + Seq("parquet").foreach { + testFileFormat => + testGluten(s"metadata struct ($testFileFormat): " + testName) { + withTempDir { + dir => + import scala.collection.JavaConverters._ + + // 1. create df0 and df1 and save under /data/f0 and /data/f1 + val df0 = spark.createDataFrame(data0.asJava, fileSchema) + val f0 = new File(dir, "data/f0").getCanonicalPath + df0.coalesce(1).write.format(testFileFormat).save(f0) + + val df1 = spark.createDataFrame(data1.asJava, fileSchema) + val f1 = new File(dir, "data/f1 gluten").getCanonicalPath + df1.coalesce(1).write.format(testFileFormat).save(f1) + + // 2. read both f0 and f1 + val df = spark.read + .format(testFileFormat) + .schema(fileSchema) + .load(new File(dir, "data").getCanonicalPath + "/*") + val realF0 = new File(dir, "data/f0") + .listFiles() + .filter(_.getName.endsWith(s".$testFileFormat")) + .head + val realF1 = new File(dir, "data/f1 gluten") + .listFiles() + .filter(_.getName.endsWith(s".$testFileFormat")) + .head + f(df, getMetadataForFile(realF0), getMetadataForFile(realF1)) + } + } + } + } + + def checkOperatorMatch[T](df: DataFrame)(implicit tag: ClassTag[T]): Unit = { + val executedPlan = getExecutedPlan(df) + assert(executedPlan.exists(plan => plan.getClass == tag.runtimeClass)) + } + + metadataColumnsNativeTest( + "plan check with metadata and user data select", + schemaWithFilePathField) { + (df, f0, f1) => + var dfWithMetadata = df.select( + METADATA_FILE_NAME, + METADATA_FILE_PATH, + METADATA_FILE_SIZE, + METADATA_FILE_MODIFICATION_TIME, + "age") + dfWithMetadata.collect + if (BackendsApiManager.getSettings.supportNativeMetadataColumns()) { + checkOperatorMatch[FileSourceScanExecTransformer](dfWithMetadata) + } else { + checkOperatorMatch[FileSourceScanExec](dfWithMetadata) + } + + // would fallback + dfWithMetadata = df.select(METADATA_FILE_PATH, "file_path") + checkAnswer( + dfWithMetadata, + Seq( + Row(f0(METADATA_FILE_PATH), "jack"), + Row(f1(METADATA_FILE_PATH), "lily") + ) + ) + checkOperatorMatch[FileSourceScanExec](dfWithMetadata) + } + + metadataColumnsNativeTest("plan check with metadata filter", schemaWithFilePathField) { + (df, f0, f1) => + var filterDF = df + .select("file_path", "age", METADATA_FILE_NAME) + .where(Column(METADATA_FILE_NAME) === f0((METADATA_FILE_NAME))) + val ret = filterDF.collect + assert(ret.size == 1) + if (BackendsApiManager.getSettings.supportNativeMetadataColumns()) { + checkOperatorMatch[FileSourceScanExecTransformer](filterDF) + } else { + checkOperatorMatch[FileSourceScanExec](filterDF) + } + checkOperatorMatch[FilterExecTransformer](filterDF) + + // case to check if file_path is URI string + filterDF = + df.select(METADATA_FILE_PATH).where(Column(METADATA_FILE_NAME) === f1((METADATA_FILE_NAME))) + checkAnswer( + filterDF, + Seq( + Row(f1(METADATA_FILE_PATH)) + ) + ) + if (BackendsApiManager.getSettings.supportNativeMetadataColumns()) { + checkOperatorMatch[FileSourceScanExecTransformer](filterDF) + } else { + checkOperatorMatch[FileSourceScanExec](filterDF) + } + checkOperatorMatch[FilterExecTransformer](filterDF) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceAggregatePushDownSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceAggregatePushDownSuite.scala new file mode 100644 index 000000000000..54138564f95f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceAggregatePushDownSuite.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetV1AggregatePushDownSuite + extends ParquetV1AggregatePushDownSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenParquetV2AggregatePushDownSuite + extends ParquetV2AggregatePushDownSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenOrcV1AggregatePushDownSuite + extends OrcV1AggregatePushDownSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenOrcV2AggregatePushDownSuite + extends OrcV2AggregatePushDownSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceCodecSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceCodecSuite.scala new file mode 100644 index 000000000000..8790bed64e1f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceCodecSuite.scala @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import org.apache.gluten.config.GlutenConfig + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetCodecSuite extends ParquetCodecSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = + super.sparkConf.set(GlutenConfig.PARQUET_UNEXPECTED_METADATA_FALLBACK_ENABLED.key, "true") +} + +class GlutenOrcCodecSuite extends OrcCodecSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceCustomMetadataStructSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceCustomMetadataStructSuite.scala new file mode 100644 index 000000000000..9aed8f6d6541 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceCustomMetadataStructSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenFileSourceCustomMetadataStructSuite + extends FileSourceCustomMetadataStructSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceStrategySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceStrategySuite.scala new file mode 100644 index 000000000000..171a27e31c47 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceStrategySuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql._ + +class GlutenFileSourceStrategySuite extends FileSourceStrategySuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenHadoopFileLinesReaderSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenHadoopFileLinesReaderSuite.scala new file mode 100644 index 000000000000..b283d44b03a4 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenHadoopFileLinesReaderSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenHadoopFileLinesReaderSuite + extends HadoopFileLinesReaderSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPathFilterStrategySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPathFilterStrategySuite.scala new file mode 100644 index 000000000000..f3554eb1cb09 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPathFilterStrategySuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenPathFilterStrategySuite extends PathFilterStrategySuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPathFilterSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPathFilterSuite.scala new file mode 100644 index 000000000000..4f4f9c76ee4a --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPathFilterSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenPathFilterSuite extends PathFilterSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPruneFileSourcePartitionsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPruneFileSourcePartitionsSuite.scala new file mode 100644 index 000000000000..a108c4fe1ecf --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPruneFileSourcePartitionsSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenPruneFileSourcePartitionsSuite + extends PruneFileSourcePartitionsSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenReadSchemaSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenReadSchemaSuite.scala new file mode 100644 index 000000000000..6ea7dd910194 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenReadSchemaSuite.scala @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import org.apache.gluten.config.GlutenConfig + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.internal.SQLConf + +import java.io.File + +class GlutenCSVReadSchemaSuite extends CSVReadSchemaSuite with GlutenSQLTestsBaseTrait { + + override def sparkConf: SparkConf = + super.sparkConf + .set(GlutenConfig.NATIVE_ARROW_READER_ENABLED.key, "true") +} + +class GlutenHeaderCSVReadSchemaSuite extends HeaderCSVReadSchemaSuite with GlutenSQLTestsBaseTrait { + + override def sparkConf: SparkConf = + super.sparkConf + .set(GlutenConfig.NATIVE_ARROW_READER_ENABLED.key, "true") +} + +class GlutenJsonReadSchemaSuite extends JsonReadSchemaSuite with GlutenSQLTestsBaseTrait {} + +class GlutenOrcReadSchemaSuite extends OrcReadSchemaSuite with GlutenSQLTestsBaseTrait {} + +class GlutenVectorizedOrcReadSchemaSuite + extends VectorizedOrcReadSchemaSuite + with GlutenSQLTestsBaseTrait { + + import testImplicits._ + + private lazy val values = 1 to 10 + private lazy val floatDF = values.map(_.toFloat).toDF("col1") + private lazy val doubleDF = values.map(_.toDouble).toDF("col1") + private lazy val unionDF = floatDF.union(doubleDF) + + testGluten("change column position") { + withTempPath { + dir => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { + val path = dir.getCanonicalPath + + val df1 = Seq(("1", "a"), ("2", "b"), ("3", "c")).toDF("col1", "col2") + val df2 = Seq(("d", "4"), ("e", "5"), ("f", "6")).toDF("col2", "col1") + val unionDF = df1.unionByName(df2) + + val dir1 = s"$path${File.separator}part=one" + val dir2 = s"$path${File.separator}part=two" + + df1.write.format(format).options(options).save(dir1) + df2.write.format(format).options(options).save(dir2) + + val df = spark.read + .schema(unionDF.schema) + .format(format) + .options(options) + .load(path) + .select("col1", "col2") + + checkAnswer(df, unionDF) + } + } + } + + testGluten("read byte, int, short, long together") { + withTempPath { + dir => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { + val path = dir.getCanonicalPath + + val byteDF = (Byte.MaxValue - 2 to Byte.MaxValue).map(_.toByte).toDF("col1") + val shortDF = (Short.MaxValue - 2 to Short.MaxValue).map(_.toShort).toDF("col1") + val intDF = (Int.MaxValue - 2 to Int.MaxValue).toDF("col1") + val longDF = (Long.MaxValue - 2 to Long.MaxValue).toDF("col1") + val unionDF = byteDF.union(shortDF).union(intDF).union(longDF) + + val byteDir = s"$path${File.separator}part=byte" + val shortDir = s"$path${File.separator}part=short" + val intDir = s"$path${File.separator}part=int" + val longDir = s"$path${File.separator}part=long" + + byteDF.write.format(format).options(options).save(byteDir) + shortDF.write.format(format).options(options).save(shortDir) + intDF.write.format(format).options(options).save(intDir) + longDF.write.format(format).options(options).save(longDir) + + val df = spark.read + .schema(unionDF.schema) + .format(format) + .options(options) + .load(path) + .select("col1") + + checkAnswer(df, unionDF) + } + } + } + + testGluten("read float and double together") { + withTempPath { + dir => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { + val path = dir.getCanonicalPath + + val floatDir = s"$path${File.separator}part=float" + val doubleDir = s"$path${File.separator}part=double" + + floatDF.write.format(format).options(options).save(floatDir) + doubleDF.write.format(format).options(options).save(doubleDir) + + val df = spark.read + .schema(unionDF.schema) + .format(format) + .options(options) + .load(path) + .select("col1") + + checkAnswer(df, unionDF) + } + } + } +} + +class GlutenMergedOrcReadSchemaSuite + extends MergedOrcReadSchemaSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenParquetReadSchemaSuite extends ParquetReadSchemaSuite with GlutenSQLTestsBaseTrait {} + +class GlutenVectorizedParquetReadSchemaSuite + extends VectorizedParquetReadSchemaSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenMergedParquetReadSchemaSuite + extends MergedParquetReadSchemaSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenTableLocationSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenTableLocationSuite.scala new file mode 100644 index 000000000000..879b0badf1dd --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenTableLocationSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenTableLocationSuite extends TableLocationSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenV1WriteCommandSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenV1WriteCommandSuite.scala new file mode 100644 index 000000000000..a287f5fffb62 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenV1WriteCommandSuite.scala @@ -0,0 +1,268 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import org.apache.gluten.GlutenColumnarWriteTestSupport +import org.apache.gluten.execution.SortExecTransformer + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.catalyst.expressions.{Ascending, AttributeReference, NullsFirst, SortOrder} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Sort} +import org.apache.spark.sql.execution.{QueryExecution, SortExec} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{IntegerType, StringType} +import org.apache.spark.sql.util.QueryExecutionListener + +trait GlutenV1WriteCommandSuiteBase extends V1WriteCommandSuiteBase { + + override def beforeAll(): Unit = { + super.beforeAll() + } + + override def afterAll(): Unit = { + super.afterAll() + } + + override def executeAndCheckOrdering( + hasLogicalSort: Boolean, + orderingMatched: Boolean, + hasEmpty2Null: Boolean = false)(query: => Unit): Unit = { + var optimizedPlan: LogicalPlan = null + + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + qe.optimizedPlan match { + case w: V1WriteCommand => + if (hasLogicalSort && conf.getConf(SQLConf.PLANNED_WRITE_ENABLED)) { + assert(w.query.isInstanceOf[WriteFiles]) + assert(w.partitionColumns == w.query.asInstanceOf[WriteFiles].partitionColumns) + optimizedPlan = w.query.asInstanceOf[WriteFiles].child + } else { + optimizedPlan = w.query + } + case _ => + } + } + override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = {} + } + spark.listenerManager.register(listener) + + query + + // RemoveNativeWriteFilesSortAndProject remove SortExec or SortExecTransformer, + // thus, FileFormatWriter.outputOrderingMatched is false. + if (!conf.getConf(SQLConf.PLANNED_WRITE_ENABLED)) { + // Check whether the output ordering is matched before FileFormatWriter executes rdd. + assert( + FileFormatWriter.outputOrderingMatched == orderingMatched, + s"Expect: $orderingMatched, Actual: ${FileFormatWriter.outputOrderingMatched}") + } + + sparkContext.listenerBus.waitUntilEmpty() + + assert(optimizedPlan != null) + // Check whether exists a logical sort node of the write query. + // If user specified sort matches required ordering, the sort node may not at the top of query. + assert( + optimizedPlan.exists(_.isInstanceOf[Sort]) == hasLogicalSort, + s"Expect hasLogicalSort: $hasLogicalSort," + + s"Actual: ${optimizedPlan.exists(_.isInstanceOf[Sort])}" + ) + + // Check empty2null conversion. + val empty2nullExpr = optimizedPlan.exists(p => V1WritesUtils.hasEmptyToNull(p.expressions)) + assert( + empty2nullExpr == hasEmpty2Null, + s"Expect hasEmpty2Null: $hasEmpty2Null, Actual: $empty2nullExpr. Plan:\n$optimizedPlan") + + spark.listenerManager.unregister(listener) + } +} + +class GlutenV1WriteCommandSuite + extends V1WriteCommandSuite + with GlutenV1WriteCommandSuiteBase + with GlutenSQLTestsBaseTrait + with GlutenColumnarWriteTestSupport { + + // TODO: fix in Spark-4.0 + ignoreGluten( + "SPARK-41914: v1 write with AQE and in-partition sorted - non-string partition column") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + withPlannedWrite { + enabled => + withTable("t") { + sql(""" + |CREATE TABLE t(b INT, value STRING) USING PARQUET + |PARTITIONED BY (key INT) + |""".stripMargin) + executeAndCheckOrdering(hasLogicalSort = true, orderingMatched = true) { + sql(""" + |INSERT INTO t + |SELECT b, value, key + |FROM testData JOIN testData2 ON key = a + |SORT BY key, value + |""".stripMargin) + } + + // inspect the actually executed plan (that is different to executeAndCheckOrdering) + assert(FileFormatWriter.executedPlan.isDefined) + val executedPlan = FileFormatWriter.executedPlan.get + + val plan = if (enabled) { + checkWriteFilesAndGetChild(executedPlan) + } else { + executedPlan.transformDown { case a: AdaptiveSparkPlanExec => a.executedPlan } + } + + // assert the outer most sort in the executed plan + assert( + plan + .collectFirst { + case s: SortExec => s + case ns: SortExecTransformer => ns + } + .exists { + case SortExec( + Seq( + SortOrder( + AttributeReference("key", IntegerType, _, _), + Ascending, + NullsFirst, + _), + SortOrder( + AttributeReference("value", StringType, _, _), + Ascending, + NullsFirst, + _) + ), + false, + _, + _ + ) => + true + case SortExecTransformer( + Seq( + SortOrder( + AttributeReference("key", IntegerType, _, _), + Ascending, + NullsFirst, + _), + SortOrder( + AttributeReference("value", StringType, _, _), + Ascending, + NullsFirst, + _) + ), + false, + _, + _ + ) => + true + case _ => false + }, + plan + ) + } + } + } + } + + // TODO: fix in Spark-4.0 + ignoreGluten("SPARK-41914: v1 write with AQE and in-partition sorted - string partition column") { + withPlannedWrite { + enabled => + withTable("t") { + sql(""" + |CREATE TABLE t(key INT, b INT) USING PARQUET + |PARTITIONED BY (value STRING) + |""".stripMargin) + executeAndCheckOrdering( + hasLogicalSort = true, + orderingMatched = true, + hasEmpty2Null = enabled) { + sql(""" + |INSERT INTO t + |SELECT key, b, value + |FROM testData JOIN testData2 ON key = a + |SORT BY value, key + |""".stripMargin) + } + + // inspect the actually executed plan (that is different to executeAndCheckOrdering) + assert(FileFormatWriter.executedPlan.isDefined) + val executedPlan = FileFormatWriter.executedPlan.get + + val plan = if (enabled) { + checkWriteFilesAndGetChild(executedPlan) + } else { + executedPlan.transformDown { case a: AdaptiveSparkPlanExec => a.executedPlan } + } + + // assert the outer most sort in the executed plan + assert( + plan + .collectFirst { + case s: SortExec => s + case ns: SortExecTransformer => ns + } + .exists { + case SortExec( + Seq( + SortOrder( + AttributeReference("value", StringType, _, _), + Ascending, + NullsFirst, + _), + SortOrder( + AttributeReference("key", IntegerType, _, _), + Ascending, + NullsFirst, + _) + ), + false, + _, + _ + ) => + true + case SortExecTransformer( + Seq( + SortOrder( + AttributeReference("value", StringType, _, _), + Ascending, + NullsFirst, + _), + SortOrder( + AttributeReference("key", IntegerType, _, _), + Ascending, + NullsFirst, + _) + ), + false, + _, + _ + ) => + true + case _ => false + }, + plan + ) + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/GlutenBinaryFileFormatSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/GlutenBinaryFileFormatSuite.scala new file mode 100644 index 000000000000..ee6ec1bea1af --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/GlutenBinaryFileFormatSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.binaryfile + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenBinaryFileFormatSuite extends BinaryFileFormatSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/csv/GlutenCSVSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/csv/GlutenCSVSuite.scala new file mode 100644 index 000000000000..63f0327e077f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/csv/GlutenCSVSuite.scala @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.csv + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.exception.GlutenException + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} +import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} +import org.apache.spark.sql.types.{DateType, IntegerType, StructType, TimestampType} + +import org.scalatest.exceptions.TestFailedException + +import java.sql.{Date, Timestamp} + +class GlutenCSVSuite extends CSVSuite with GlutenSQLTestsBaseTrait { + + override def sparkConf: SparkConf = + super.sparkConf + .set(GlutenConfig.NATIVE_ARROW_READER_ENABLED.key, "true") + + /** Returns full path to the given file in the resource folder */ + override protected def testFile(fileName: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + fileName + } +} + +class GlutenCSVv1Suite extends GlutenCSVSuite { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "csv") +} + +class GlutenCSVv2Suite extends GlutenCSVSuite { + + import testImplicits._ + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") + .set(GlutenConfig.NATIVE_ARROW_READER_ENABLED.key, "true") + + override def testNameBlackList: Seq[String] = Seq( + // overwritten with different test + "test for FAILFAST parsing mode", + "SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern" + ) + + testGluten("test for FAILFAST parsing mode") { + Seq(false, true).foreach { + multiLine => + val exception = intercept[SparkException] { + spark.read + .format("csv") + .option("multiLine", multiLine) + .options(Map("header" -> "true", "mode" -> "failfast")) + .load(testFile(carsFile)) + .collect() + } + + assert(exception.getCause.isInstanceOf[GlutenException]) + assert( + exception.getMessage.contains( + "[MALFORMED_RECORD_IN_PARSING] Malformed records are detected in record parsing: " + + "[2015,Chevy,Volt,null,null]")) + } + } + + testGluten("SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern") { + withTempPath { + path => + Seq("1,2020011,2020011", "2,20201203,20201203") + .toDF() + .repartition(1) + .write + .text(path.getAbsolutePath) + val schema = new StructType() + .add("id", IntegerType) + .add("date", DateType) + .add("ts", TimestampType) + val output = spark.read + .schema(schema) + .option("dateFormat", "yyyyMMdd") + .option("timestampFormat", "yyyyMMdd") + .csv(path.getAbsolutePath) + + def check(mode: String, res: Seq[Row]): Unit = { + withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> mode) { + checkAnswer(output, res) + } + } + + check( + "legacy", + Seq( + Row(1, Date.valueOf("2020-01-01"), Timestamp.valueOf("2020-01-01 00:00:00")), + Row(2, Date.valueOf("2020-12-03"), Timestamp.valueOf("2020-12-03 00:00:00")) + ) + ) + + check( + "corrected", + Seq( + Row(1, null, null), + Row(2, Date.valueOf("2020-12-03"), Timestamp.valueOf("2020-12-03 00:00:00")) + ) + ) + + val err = intercept[TestFailedException] { + check("exception", Nil) + } + assert(err.message.get.contains("org.apache.spark.SparkUpgradeException")) + } + } +} + +class GlutenCSVLegacyTimeParserSuite extends GlutenCSVSuite { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.LEGACY_TIME_PARSER_POLICY, LegacyBehaviorPolicy.LEGACY) +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/json/GlutenJsonSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/json/GlutenJsonSuite.scala new file mode 100644 index 000000000000..3fefc9bdd44d --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/json/GlutenJsonSuite.scala @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.json + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{sources, GlutenSQLTestsBaseTrait, Row} +import org.apache.spark.sql.execution.datasources.{InMemoryFileIndex, NoopCache} +import org.apache.spark.sql.execution.datasources.v2.json.JsonScanBuilder +import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} +import org.apache.spark.sql.types.{DateType, IntegerType, StructType, TimestampType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +import org.scalatest.exceptions.TestFailedException + +import java.sql.{Date, Timestamp} + +class GlutenJsonSuite extends JsonSuite with GlutenSQLTestsBaseTrait { + + /** Returns full path to the given file in the resource folder */ + override protected def testFile(fileName: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + fileName + } +} + +class GlutenJsonV1Suite extends GlutenJsonSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "json") +} + +class GlutenJsonV2Suite extends GlutenJsonSuite with GlutenSQLTestsBaseTrait { + + import testImplicits._ + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") + + testGluten("get pushed filters") { + val attr = "col" + def getBuilder(path: String): JsonScanBuilder = { + val fileIndex = new InMemoryFileIndex( + spark, + Seq(new org.apache.hadoop.fs.Path(path, "file.json")), + Map.empty, + None, + NoopCache) + val schema = new StructType().add(attr, IntegerType) + val options = CaseInsensitiveStringMap.empty() + new JsonScanBuilder(spark, fileIndex, schema, schema, options) + } + val filters: Array[sources.Filter] = Array(sources.IsNotNull(attr)) + withSQLConf(SQLConf.JSON_FILTER_PUSHDOWN_ENABLED.key -> "true") { + withTempPath { + file => + val scanBuilder = getBuilder(file.getCanonicalPath) + assert(scanBuilder.pushDataFilters(filters) === filters) + } + } + + withSQLConf(SQLConf.JSON_FILTER_PUSHDOWN_ENABLED.key -> "false") { + withTempPath { + file => + val scanBuilder = getBuilder(file.getCanonicalPath) + assert(scanBuilder.pushDataFilters(filters) === Array.empty[sources.Filter]) + } + } + } + + testGluten("SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern") { + withTempPath { + path => + Seq( + """{"date": "2020011", "ts": "2020011"}""", + """{"date": "20201203", "ts": "20201203"}""") + .toDF() + .repartition(1) + .write + .text(path.getAbsolutePath) + val schema = new StructType() + .add("date", DateType) + .add("ts", TimestampType) + val output = spark.read + .schema(schema) + .option("dateFormat", "yyyyMMdd") + .option("timestampFormat", "yyyyMMdd") + .json(path.getAbsolutePath) + + def check(mode: String, res: Seq[Row]): Unit = { + withSQLConf(SQLConf.LEGACY_TIME_PARSER_POLICY.key -> mode) { + checkAnswer(output, res) + } + } + + check( + "legacy", + Seq( + Row(Date.valueOf("2020-01-01"), Timestamp.valueOf("2020-01-01 00:00:00")), + Row(Date.valueOf("2020-12-03"), Timestamp.valueOf("2020-12-03 00:00:00")) + ) + ) + + check( + "corrected", + Seq( + Row(null, null), + Row(Date.valueOf("2020-12-03"), Timestamp.valueOf("2020-12-03 00:00:00")) + ) + ) + + val err = intercept[TestFailedException] { + check("exception", Nil) + } + assert(err.message.get.contains("org.apache.spark.SparkUpgradeException")) + } + } +} + +class GlutenJsonLegacyTimeParserSuite extends GlutenJsonSuite with GlutenSQLTestsBaseTrait { + + override def excluded: Seq[String] = + Seq("Write timestamps correctly in ISO8601 format by default") + + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.LEGACY_TIME_PARSER_POLICY, LegacyBehaviorPolicy.LEGACY) +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcColumnarBatchReaderSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcColumnarBatchReaderSuite.scala new file mode 100644 index 000000000000..e2e3818aad9c --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcColumnarBatchReaderSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.orc + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenOrcColumnarBatchReaderSuite + extends OrcColumnarBatchReaderSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcFilterSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcFilterSuite.scala new file mode 100644 index 000000000000..f5a8db3395d6 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcFilterSuite.scala @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.orc + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +/** A test suite that tests Apache ORC filter API based filter pushdown optimization. */ +class GlutenOrcFilterSuite extends OrcFilterSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcPartitionDiscoverySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcPartitionDiscoverySuite.scala new file mode 100644 index 000000000000..a9848b7f444d --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcPartitionDiscoverySuite.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.orc + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenOrcPartitionDiscoverySuite + extends OrcPartitionDiscoveryTest + with GlutenSQLTestsBaseTrait {} + +class GlutenOrcV1PartitionDiscoverySuite + extends OrcV1PartitionDiscoverySuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcQuerySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcQuerySuite.scala new file mode 100644 index 000000000000..f186695b88b8 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcQuerySuite.scala @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.orc + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.internal.SQLConf + +class GlutenOrcQuerySuite extends OrcQuerySuite with GlutenSQLTestsBaseTrait { + testGluten("Simple selection form ORC table") { + val data = (1 to 10).map { + i => Person(s"name_$i", i, (0 to 1).map(m => Contact(s"contact_$m", s"phone_$m"))) + } + + withOrcTable(data, "t") { + withSQLConf("spark.sql.orc.enableVectorizedReader" -> "false") { + // ppd: + // leaf-0 = (LESS_THAN_EQUALS age 5) + // expr = leaf-0 + assert(sql("SELECT name FROM t WHERE age <= 5").count() === 5) + + // ppd: + // leaf-0 = (LESS_THAN_EQUALS age 5) + // expr = (not leaf-0) + assertResult(10) { + sql("SELECT name, contacts FROM t where age > 5").rdd + .flatMap(_.getAs[scala.collection.Seq[_]]("contacts")) + .count() + } + + // ppd: + // leaf-0 = (LESS_THAN_EQUALS age 5) + // leaf-1 = (LESS_THAN age 8) + // expr = (and (not leaf-0) leaf-1) + { + val df = sql("SELECT name, contacts FROM t WHERE age > 5 AND age < 8") + assert(df.count() === 2) + assertResult(4) { + df.rdd.flatMap(_.getAs[scala.collection.Seq[_]]("contacts")).count() + } + } + + // ppd: + // leaf-0 = (LESS_THAN age 2) + // leaf-1 = (LESS_THAN_EQUALS age 8) + // expr = (or leaf-0 (not leaf-1)) + { + val df = sql("SELECT name, contacts FROM t WHERE age < 2 OR age > 8") + assert(df.count() === 3) + assertResult(6) { + df.rdd.flatMap(_.getAs[scala.collection.Seq[_]]("contacts")).count() + } + } + } + } + } + + testGluten("simple select queries") { + withOrcTable((0 until 10).map(i => (i, i.toString)), "t") { + withSQLConf("spark.sql.orc.enableVectorizedReader" -> "false") { + checkAnswer(sql("SELECT `_1` FROM t where t.`_1` > 5"), (6 until 10).map(Row.apply(_))) + + checkAnswer( + sql("SELECT `_1` FROM t as tmp where tmp.`_1` < 5"), + (0 until 5).map(Row.apply(_))) + } + } + } + + testGluten("overwriting") { + val data = (0 until 10).map(i => (i, i.toString)) + spark.createDataFrame(data).toDF("c1", "c2").createOrReplaceTempView("tmp") + withOrcTable(data, "t") { + withSQLConf("spark.sql.orc.enableVectorizedReader" -> "false") { + sql("INSERT OVERWRITE TABLE t SELECT * FROM tmp") + checkAnswer(spark.table("t"), data.map(Row.fromTuple)) + } + } + spark.sessionState.catalog.dropTable( + TableIdentifier("tmp"), + ignoreIfNotExists = true, + purge = false) + } + + testGluten("self-join") { + // 4 rows, cells of column 1 of row 2 and row 4 are null + val data = (1 to 4).map { + i => + val maybeInt = if (i % 2 == 0) None else Some(i) + (maybeInt, i.toString) + } + + withOrcTable(data, "t") { + withSQLConf("spark.sql.orc.enableVectorizedReader" -> "false") { + val selfJoin = sql("SELECT * FROM t x JOIN t y WHERE x.`_1` = y.`_1`") + val queryOutput = selfJoin.queryExecution.analyzed.output + + assertResult(4, "Field count mismatches")(queryOutput.size) + assertResult(2, s"Duplicated expression ID in query plan:\n $selfJoin") { + queryOutput.filter(_.name == "_1").map(_.exprId).size + } + + checkAnswer(selfJoin, List(Row(1, "1", 1, "1"), Row(3, "3", 3, "3"))) + } + } + } + + testGluten("columns only referenced by pushed down filters should remain") { + withOrcTable((1 to 10).map(Tuple1.apply), "t") { + withSQLConf("spark.sql.orc.enableVectorizedReader" -> "false") { + checkAnswer(sql("SELECT `_1` FROM t WHERE `_1` < 10"), (1 to 9).map(Row.apply(_))) + } + } + } + + testGluten("SPARK-5309 strings stored using dictionary compression in orc") { + withOrcTable((0 until 1000).map(i => ("same", "run_" + i / 100, 1)), "t") { + withSQLConf("spark.sql.orc.enableVectorizedReader" -> "false") { + checkAnswer( + sql("SELECT `_1`, `_2`, SUM(`_3`) FROM t GROUP BY `_1`, `_2`"), + (0 until 10).map(i => Row("same", "run_" + i, 100))) + + checkAnswer( + sql("SELECT `_1`, `_2`, SUM(`_3`) FROM t WHERE `_2` = 'run_5' GROUP BY `_1`, `_2`"), + List(Row("same", "run_5", 100))) + } + } + } +} + +class GlutenOrcV1QuerySuite extends GlutenOrcQuerySuite { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "orc") +} + +class GlutenOrcV2QuerySuite extends GlutenOrcQuerySuite { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcSourceSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcSourceSuite.scala new file mode 100644 index 000000000000..3413ff72c100 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcSourceSuite.scala @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.orc + +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, GlutenTestConstants, Row} +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{DayTimeIntervalType, IntegerType, StructField, StructType, YearMonthIntervalType} + +import java.sql.Date +import java.time.{Duration, Period} + +class GlutenOrcSourceSuite extends OrcSourceSuite with GlutenSQLTestsBaseTrait { + import testImplicits._ + + override def withAllNativeOrcReaders(code: => Unit): Unit = { + // test the row-based reader + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false")(code) + } + + testGluten("SPARK-31238: compatibility with Spark 2.4 in reading dates") { + Seq(false).foreach { + vectorized => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + checkAnswer( + readResourceOrcFile("test-data/before_1582_date_v2_4.snappy.orc"), + Row(java.sql.Date.valueOf("1200-01-01"))) + } + } + } + + testGluten("SPARK-31238, SPARK-31423: rebasing dates in write") { + withTempPath { + dir => + val path = dir.getAbsolutePath + Seq("1001-01-01", "1582-10-10") + .toDF("dateS") + .select($"dateS".cast("date").as("date")) + .write + .orc(path) + + Seq(false).foreach { + vectorized => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + checkAnswer( + spark.read.orc(path), + Seq(Row(Date.valueOf("1001-01-01")), Row(Date.valueOf("1582-10-15")))) + } + } + } + } + + testGluten("SPARK-31284: compatibility with Spark 2.4 in reading timestamps") { + Seq(false).foreach { + vectorized => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + checkAnswer( + readResourceOrcFile("test-data/before_1582_ts_v2_4.snappy.orc"), + Row(java.sql.Timestamp.valueOf("1001-01-01 01:02:03.123456"))) + } + } + } + + testGluten("SPARK-31284, SPARK-31423: rebasing timestamps in write") { + withTempPath { + dir => + val path = dir.getAbsolutePath + Seq("1001-01-01 01:02:03.123456", "1582-10-10 11:12:13.654321") + .toDF("tsS") + .select($"tsS".cast("timestamp").as("ts")) + .write + .orc(path) + + Seq(false).foreach { + vectorized => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + checkAnswer( + spark.read.orc(path), + Seq( + Row(java.sql.Timestamp.valueOf("1001-01-01 01:02:03.123456")), + Row(java.sql.Timestamp.valueOf("1582-10-15 11:12:13.654321")))) + } + } + } + } + + testGluten("SPARK-34862: Support ORC vectorized reader for nested column") { + withTempPath { + dir => + val path = dir.getCanonicalPath + val df = spark + .range(10) + .map { + x => + val stringColumn = s"$x" * 10 + val structColumn = (x, s"$x" * 100) + val arrayColumn = (0 until 5).map(i => (x + i, s"$x" * 5)) + val mapColumn = Map( + s"$x" -> (x * 0.1, (x, s"$x" * 100)), + (s"$x" * 2) -> (x * 0.2, (x, s"$x" * 200)), + (s"$x" * 3) -> (x * 0.3, (x, s"$x" * 300))) + (x, stringColumn, structColumn, arrayColumn, mapColumn) + } + .toDF("int_col", "string_col", "struct_col", "array_col", "map_col") + df.write.format("orc").save(path) + + // Rewrite because Gluten does not support Spark's vectorized reading. + withSQLConf(SQLConf.ORC_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key -> "false") { + val readDf = spark.read.orc(path) + val vectorizationEnabled = readDf.queryExecution.executedPlan.find { + case scan: FileSourceScanExec => scan.supportsColumnar + case _ => false + }.isDefined + assert(!vectorizationEnabled) + checkAnswer(readDf, df) + } + } + } + withAllNativeOrcReaders { + Seq(false).foreach { + vecReaderNestedColEnabled => + val vecReaderEnabled = SQLConf.get.orcVectorizedReaderEnabled + test( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-36931: Support reading and writing ANSI intervals (" + + s"${SQLConf.ORC_VECTORIZED_READER_ENABLED.key}=$vecReaderEnabled, " + + s"${SQLConf.ORC_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key}" + + s"=$vecReaderNestedColEnabled)") { + + withSQLConf( + SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> + vecReaderEnabled.toString, + SQLConf.ORC_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key -> + vecReaderNestedColEnabled.toString + ) { + Seq( + YearMonthIntervalType() -> ((i: Int) => Period.of(i, i, 0)), + DayTimeIntervalType() -> ((i: Int) => Duration.ofDays(i).plusSeconds(i)) + ).foreach { + case (it, f) => + val data = (1 to 10).map(i => Row(i, f(i))) + val schema = StructType( + Array(StructField("d", IntegerType, false), StructField("i", it, false))) + withTempPath { + file => + val df = spark.createDataFrame(sparkContext.parallelize(data), schema) + df.write.orc(file.getCanonicalPath) + val df2 = spark.read.orc(file.getCanonicalPath) + checkAnswer(df2, df.collect().toSeq) + } + } + + // Tests for ANSI intervals in complex types. + withTempPath { + file => + val df = spark.sql("""SELECT + | named_struct('interval', interval '1-2' year to month) a, + | array(interval '1 2:3' day to minute) b, + | map('key', interval '10' year) c, + | map(interval '20' second, 'value') d""".stripMargin) + df.write.orc(file.getCanonicalPath) + val df2 = spark.read.orc(file.getCanonicalPath) + checkAnswer(df2, df.collect().toSeq) + } + } + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV1FilterSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV1FilterSuite.scala new file mode 100644 index 000000000000..3c2fb0b318f1 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV1FilterSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.orc + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenOrcV1FilterSuite extends OrcV1FilterSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV1SchemaPruningSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV1SchemaPruningSuite.scala new file mode 100644 index 000000000000..c142d33bdc22 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV1SchemaPruningSuite.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.orc + +import org.apache.gluten.execution.FileSourceScanExecTransformer + +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.tags.ExtendedSQLTest + +@ExtendedSQLTest +class GlutenOrcV1SchemaPruningSuite extends OrcV1SchemaPruningSuite with GlutenSQLTestsBaseTrait { + // disable column reader for nested type + override protected val vectorizedReaderNestedEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key + "_DISABLED" + + override def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { + val fileSourceScanSchemata = + collect(df.queryExecution.executedPlan) { + case scan: FileSourceScanExec => scan.requiredSchema + case scan: FileSourceScanExecTransformer => scan.requiredSchema + } + assert( + fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, + s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + + s"but expected $expectedSchemaCatalogStrings" + ) + fileSourceScanSchemata.zip(expectedSchemaCatalogStrings).foreach { + case (scanSchema, expectedScanSchemaCatalogString) => + val expectedScanSchema = CatalystSqlParser.parseDataType(expectedScanSchemaCatalogString) + implicit val equality = schemaEquality + assert(scanSchema === expectedScanSchema) + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV2SchemaPruningSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV2SchemaPruningSuite.scala new file mode 100644 index 000000000000..76a9a6ef956b --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV2SchemaPruningSuite.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.orc + +import org.apache.gluten.execution.BatchScanExecTransformer + +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.tags.ExtendedSQLTest + +@ExtendedSQLTest +class GlutenOrcV2SchemaPruningSuite extends OrcV2SchemaPruningSuite with GlutenSQLTestsBaseTrait { + // disable column reader for nested type + override protected val vectorizedReaderNestedEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key + "_DISABLED" + + override def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { + val fileSourceScanSchemata = + collect(df.queryExecution.executedPlan) { + case b: BatchScanExec if b.scan.isInstanceOf[OrcScan] => + b.scan.asInstanceOf[OrcScan].readDataSchema + case b: BatchScanExecTransformer if b.scan.isInstanceOf[OrcScan] => + b.scan.asInstanceOf[OrcScan].readDataSchema + } + assert( + fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, + s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + + s"but expected $expectedSchemaCatalogStrings" + ) + fileSourceScanSchemata.zip(expectedSchemaCatalogStrings).foreach { + case (scanSchema, expectedScanSchemaCatalogString) => + val expectedScanSchema = CatalystSqlParser.parseDataType(expectedScanSchemaCatalogString) + implicit val equality = schemaEquality + assert(scanSchema === expectedScanSchema) + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetColumnIndexSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetColumnIndexSuite.scala new file mode 100644 index 000000000000..4bb8e964553e --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetColumnIndexSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetColumnIndexSuite extends ParquetColumnIndexSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetCompressionCodecPrecedenceSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetCompressionCodecPrecedenceSuite.scala new file mode 100644 index 000000000000..ac938d4eaf4f --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetCompressionCodecPrecedenceSuite.scala @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +class GlutenParquetCompressionCodecPrecedenceSuite + extends ParquetCompressionCodecPrecedenceSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaByteArrayEncodingSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaByteArrayEncodingSuite.scala new file mode 100644 index 000000000000..166f3255efd5 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaByteArrayEncodingSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetDeltaByteArrayEncodingSuite + extends ParquetDeltaLengthByteArrayEncodingSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaEncodingSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaEncodingSuite.scala new file mode 100644 index 000000000000..ccb69819a3a3 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaEncodingSuite.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetDeltaEncodingInteger + extends ParquetDeltaEncodingInteger + with GlutenSQLTestsBaseTrait {} + +class GlutenParquetDeltaEncodingLong + extends ParquetDeltaEncodingLong + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaLengthByteArrayEncodingSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaLengthByteArrayEncodingSuite.scala new file mode 100644 index 000000000000..36928cee001d --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaLengthByteArrayEncodingSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetDeltaLengthByteArrayEncodingSuite + extends ParquetDeltaLengthByteArrayEncodingSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetEncodingSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetEncodingSuite.scala new file mode 100644 index 000000000000..6c69c700becc --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetEncodingSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +// TODO: this needs a lot more testing but it's currently not easy to test with the parquet +// writer abstractions. Revisit. +class GlutenParquetEncodingSuite extends ParquetEncodingSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala new file mode 100644 index 000000000000..bd1c269843fb --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} + +class GlutenParquetFieldIdIOSuite extends ParquetFieldIdIOSuite with GlutenSQLTestsBaseTrait { + testGluten("Parquet writer with ARRAY and MAP") { + spark.sql(""" + |CREATE TABLE T1 ( + | a INT, + | b ARRAY, + | c MAP + |) + |USING PARQUET + |""".stripMargin) + + spark.sql(""" + | INSERT OVERWRITE T1 VALUES + | (1, ARRAY(1, 2, 3), MAP("key1","value1")) + |""".stripMargin) + + checkAnswer( + spark.sql("SELECT * FROM T1"), + Row(1, Array("1", "2", "3"), Map("key1" -> "value1")) :: Nil + ) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFileFormatSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFileFormatSuite.scala new file mode 100644 index 000000000000..b60850df2401 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFileFormatSuite.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetFileFormatV1Suite + extends ParquetFileFormatV1Suite + with GlutenSQLTestsBaseTrait {} + +class GlutenParquetFileFormatV2Suite + extends ParquetFileFormatV2Suite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFileMetadataStructRowIndexSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFileMetadataStructRowIndexSuite.scala new file mode 100644 index 000000000000..9d5291471b33 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFileMetadataStructRowIndexSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetFileMetadataStructRowIndexSuite + extends ParquetFileMetadataStructRowIndexSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala new file mode 100644 index 000000000000..3a039d975f30 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala @@ -0,0 +1,580 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.SparkConf +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.classic.ClassicConversions._ +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.parseColumnPath +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation, PushableColumnAndNestedColumn} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +import org.apache.spark.sql.internal.LegacyBehaviorPolicy.{CORRECTED, LEGACY} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType.INT96 +import org.apache.spark.sql.types._ +import org.apache.spark.tags.ExtendedSQLTest +import org.apache.spark.util.Utils + +import org.apache.hadoop.fs.Path +import org.apache.parquet.filter2.predicate.{FilterApi, FilterPredicate, Operators} +import org.apache.parquet.filter2.predicate.FilterApi._ +import org.apache.parquet.filter2.predicate.Operators.{Column => _, Eq, Gt, GtEq, Lt, LtEq, NotEq} +import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetOutputFormat} +import org.apache.parquet.hadoop.util.HadoopInputFile + +import java.sql.{Date, Timestamp} +import java.time.LocalDate + +import scala.reflect.ClassTag +import scala.reflect.runtime.universe.TypeTag + +abstract class GlutenParquetFilterSuite extends ParquetFilterSuite with GlutenSQLTestsBaseTrait { + protected def checkFilterPredicate( + predicate: Predicate, + filterClass: Class[_ <: FilterPredicate], + expected: Seq[Row])(implicit df: DataFrame): Unit = { + checkFilterPredicate(df, predicate, filterClass, checkAnswer(_, _: Seq[Row]), expected) + } + + protected def checkFilterPredicate[T]( + predicate: Predicate, + filterClass: Class[_ <: FilterPredicate], + expected: T)(implicit df: DataFrame): Unit = { + checkFilterPredicate(predicate, filterClass, Seq(Row(expected)))(df) + } + + override protected def readResourceParquetFile(name: String): DataFrame = { + spark.read.parquet( + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + name) + } + + testGluten("filter pushdown - timestamp") { + Seq(true, false).foreach { + java8Api => + Seq(CORRECTED, LEGACY).foreach { + rebaseMode => + val millisData = Seq( + "1000-06-14 08:28:53.123", + "1582-06-15 08:28:53.001", + "1900-06-16 08:28:53.0", + "2018-06-17 08:28:53.999") + // INT96 doesn't support pushdown + withSQLConf( + SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString, + SQLConf.PARQUET_INT96_REBASE_MODE_IN_WRITE.key -> rebaseMode.toString, + SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> INT96.toString + ) { + import testImplicits._ + withTempPath { + file => + millisData + .map(i => Tuple1(Timestamp.valueOf(i))) + .toDF + .write + .format(dataSourceName) + .save(file.getCanonicalPath) + readParquetFile(file.getCanonicalPath) { + df => + val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) + assertResult(None) { + createParquetFilters(schema).createFilter(sources.IsNull("_1")) + } + } + } + } + } + } + } + + testGluten("SPARK-12218: 'Not' is included in Parquet filter pushdown") { + import testImplicits._ + + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") { + withTempPath { + dir => + val path = s"${dir.getCanonicalPath}/table1" + val df = (1 to 5).map(i => (i, (i % 2).toString)).toDF("a", "b") + df.show() + df.write.parquet(path) + + checkAnswer( + spark.read.parquet(path).where("not (a = 2) or not(b in ('1'))"), + (1 to 5).map(i => Row(i, (i % 2).toString))) + + checkAnswer( + spark.read.parquet(path).where("not (a = 2 and b in ('1'))"), + (1 to 5).map(i => Row(i, (i % 2).toString))) + } + } + } + + testGluten("SPARK-23852: Broken Parquet push-down for partially-written stats") { + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") { + // parquet-1217.parquet contains a single column with values -1, 0, 1, 2 and null. + // The row-group statistics include null counts, but not min and max values, which + // triggers PARQUET-1217. + + val df = readResourceParquetFile("test-data/parquet-1217.parquet") + + // Will return 0 rows if PARQUET-1217 is not fixed. + assert(df.where("col > 0").count() === 2) + } + } + + testGluten("SPARK-17091: Convert IN predicate to Parquet filter push-down") { + val schema = StructType( + Seq( + StructField("a", IntegerType, nullable = false) + )) + + val parquetSchema = new SparkToParquetSchemaConverter(conf).convert(schema) + val parquetFilters = createParquetFilters(parquetSchema) + assertResult(Some(FilterApi.eq(intColumn("a"), null: Integer))) { + parquetFilters.createFilter(sources.In("a", Array(null))) + } + + assertResult(Some(FilterApi.eq(intColumn("a"), 10: Integer))) { + parquetFilters.createFilter(sources.In("a", Array(10))) + } + + // Remove duplicates + assertResult(Some(FilterApi.eq(intColumn("a"), 10: Integer))) { + parquetFilters.createFilter(sources.In("a", Array(10, 10))) + } + + assertResult( + Some( + or( + or(FilterApi.eq(intColumn("a"), 10: Integer), FilterApi.eq(intColumn("a"), 20: Integer)), + FilterApi.eq(intColumn("a"), 30: Integer)))) { + parquetFilters.createFilter(sources.In("a", Array(10, 20, 30))) + } + + Seq(0, 10).foreach { + threshold => + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_INFILTERTHRESHOLD.key -> threshold.toString) { + assert( + createParquetFilters(parquetSchema) + .createFilter(sources.In("a", Array(10, 20, 30))) + .nonEmpty === threshold > 0) + } + } + + import testImplicits._ + withTempPath { + path => + val data = 0 to 1024 + data + .toDF("a") + .selectExpr("if (a = 1024, null, a) AS a") // convert 1024 to null + .coalesce(1) + .write + .option("parquet.block.size", 512) + .parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath) + Seq(true, false).foreach { + pushEnabled => + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> pushEnabled.toString) { + Seq(1, 5, 10, 11, 100).foreach { + count => + val filter = s"a in(${Range(0, count).mkString(",")})" + assert(df.where(filter).count() === count) + val actual = stripSparkFilter(df.where(filter)).collect().length + assert(actual === count) + } + assert(df.where("a in(null)").count() === 0) + assert(df.where("a = null").count() === 0) + assert(df.where("a is null").count() === 1) + } + } + } + } + + // Velox doesn't support ParquetOutputFormat.PAGE_SIZE and ParquetOutputFormat.BLOCK_SIZE. + ignoreGluten("Support Parquet column index") { + // block 1: + // null count min max + // page-0 0 0 99 + // page-1 0 100 199 + // page-2 0 200 299 + // page-3 0 300 399 + // page-4 0 400 449 + // + // block 2: + // null count min max + // page-0 0 450 549 + // page-1 0 550 649 + // page-2 0 650 749 + // page-3 0 750 849 + // page-4 0 850 899 + withTempPath { + path => + spark + .range(900) + .repartition(1) + .write + .option(ParquetOutputFormat.PAGE_SIZE, "500") + .option(ParquetOutputFormat.BLOCK_SIZE, "2000") + .parquet(path.getCanonicalPath) + + val parquetFile = path.listFiles().filter(_.getName.startsWith("part")).last + val in = HadoopInputFile.fromPath( + new Path(parquetFile.getCanonicalPath), + spark.sessionState.newHadoopConf()) + + Utils.tryWithResource(ParquetFileReader.open(in)) { + reader => + val blocks = reader.getFooter.getBlocks + assert(blocks.size() > 1) + val columns = blocks.get(0).getColumns + assert(columns.size() === 1) + val columnIndex = reader.readColumnIndex(columns.get(0)) + assert(columnIndex.getMinValues.size() > 1) + + val rowGroupCnt = blocks.get(0).getRowCount + // Page count = Second page min value - first page min value + val pageCnt = columnIndex.getMinValues.get(1).asLongBuffer().get() - + columnIndex.getMinValues.get(0).asLongBuffer().get() + assert(pageCnt < rowGroupCnt) + Seq(true, false).foreach { + columnIndex => + withSQLConf(ParquetInputFormat.COLUMN_INDEX_FILTERING_ENABLED -> s"$columnIndex") { + val df = spark.read.parquet(parquetFile.getCanonicalPath).where("id = 1") + df.collect() + val plan = df.queryExecution.executedPlan + // Ignore metrics comparison. + /* + val metrics = plan.collectLeaves().head.metrics + val numOutputRows = metrics("numOutputRows").value + + if (columnIndex) { + assert(numOutputRows === pageCnt) + } else { + assert(numOutputRows === rowGroupCnt) + } + */ + } + } + } + } + } +} + +@ExtendedSQLTest +class GlutenParquetV1FilterSuite extends GlutenParquetFilterSuite with GlutenSQLTestsBaseTrait { + // TODO: enable Parquet V2 write path after file source V2 writers are workable. + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "parquet") + override def checkFilterPredicate( + df: DataFrame, + predicate: Predicate, + filterClass: Class[_ <: FilterPredicate], + checker: (DataFrame, Seq[Row]) => Unit, + expected: Seq[Row]): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + + Seq(("parquet", true), ("", false)).foreach { + case (pushdownDsList, nestedPredicatePushdown) => + withSQLConf( + SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_DATE_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_DECIMAL_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_STRING_STARTSWITH_ENABLED.key -> "true", + // Disable adding filters from constraints because it adds, for instance, + // is-not-null to pushed filters, which makes it hard to test if the pushed + // filter is expected or not (this had to be fixed with SPARK-13495). + SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> InferFiltersFromConstraints.ruleName, + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false", + SQLConf.NESTED_PREDICATE_PUSHDOWN_FILE_SOURCE_LIST.key -> pushdownDsList + ) { + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + val nestedOrAttributes = predicate.collectFirst { + case g: GetStructField => g + case a: Attribute => a + } + assert(nestedOrAttributes.isDefined, "No GetStructField nor Attribute is detected.") + + val parsed = + parseColumnPath(PushableColumnAndNestedColumn.unapply(nestedOrAttributes.get).get) + + val containsNestedColumnOrDot = parsed.length > 1 || parsed(0).contains(".") + + var maybeRelation: Option[HadoopFsRelation] = None + val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan + .collect { + case PhysicalOperation( + _, + filters, + LogicalRelation(relation: HadoopFsRelation, _, _, _, _)) => + maybeRelation = Some(relation) + filters + } + .flatten + .reduceLeftOption(_ && _) + assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") + + val (_, selectedFilters, _) = + DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) + // If predicates contains nested column or dot, we push down the predicates only if + // "parquet" is in `NESTED_PREDICATE_PUSHDOWN_V1_SOURCE_LIST`. + if (nestedPredicatePushdown || !containsNestedColumnOrDot) { + assert(selectedFilters.nonEmpty, "No filter is pushed down") + val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) + val parquetFilters = createParquetFilters(schema) + // In this test suite, all the simple predicates are convertible here. + assert(parquetFilters.convertibleFilters(selectedFilters) === selectedFilters) + val pushedParquetFilters = selectedFilters.map { + pred => + val maybeFilter = parquetFilters.createFilter(pred) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pred") + maybeFilter.get + } + // Doesn't bother checking type parameters here (e.g. `Eq[Integer]`) + assert( + pushedParquetFilters.exists(_.getClass === filterClass), + s"${pushedParquetFilters.map(_.getClass).toList} did not contain $filterClass.") + + checker(stripSparkFilter(query), expected) + } else { + assert(selectedFilters.isEmpty, "There is filter pushed down") + } + } + } + } +} + +@ExtendedSQLTest +class GlutenParquetV2FilterSuite extends GlutenParquetFilterSuite with GlutenSQLTestsBaseTrait { + // TODO: enable Parquet V2 write path after file source V2 writers are workable. + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") + + override def checkFilterPredicate( + df: DataFrame, + predicate: Predicate, + filterClass: Class[_ <: FilterPredicate], + checker: (DataFrame, Seq[Row]) => Unit, + expected: Seq[Row]): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + + withSQLConf( + SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_DATE_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_DECIMAL_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_STRING_STARTSWITH_ENABLED.key -> "true", + // Disable adding filters from constraints because it adds, for instance, + // is-not-null to pushed filters, which makes it hard to test if the pushed + // filter is expected or not (this had to be fixed with SPARK-13495). + SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> InferFiltersFromConstraints.ruleName, + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false" + ) { + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + query.queryExecution.optimizedPlan.collectFirst { + case PhysicalOperation( + _, + filters, + DataSourceV2ScanRelation(_, scan: ParquetScan, _, None, None)) => + assert(filters.nonEmpty, "No filter is analyzed from the given query") + val sourceFilters = filters.flatMap(DataSourceStrategy.translateFilter(_, true)).toArray + val pushedFilters = scan.pushedFilters + assert(pushedFilters.nonEmpty, "No filter is pushed down") + val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) + val parquetFilters = createParquetFilters(schema) + // In this test suite, all the simple predicates are convertible here. + assert(parquetFilters.convertibleFilters(sourceFilters) === pushedFilters) + val pushedParquetFilters = pushedFilters.map { + pred => + val maybeFilter = parquetFilters.createFilter(pred) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pred") + maybeFilter.get + } + // Doesn't bother checking type parameters here (e.g. `Eq[Integer]`) + assert( + pushedParquetFilters.exists(_.getClass === filterClass), + s"${pushedParquetFilters.map(_.getClass).toList} did not contain $filterClass.") + + checker(stripSparkFilter(query), expected) + + case _ => assert(false, "Can not match ParquetTable in the query.") + } + } + } + + /** + * Takes a sequence of products `data` to generate multi-level nested dataframes as new test data. + * It tests both non-nested and nested dataframes which are written and read back with Parquet + * datasource. + * + * This is different from [[ParquetTest.withParquetDataFrame]] which does not test nested cases. + */ + private def withNestedParquetDataFrame[T <: Product: ClassTag: TypeTag](data: Seq[T])( + runTest: (DataFrame, String, Any => Any) => Unit): Unit = + withNestedParquetDataFrame(spark.createDataFrame(data))(runTest) + + private def withNestedParquetDataFrame(inputDF: DataFrame)( + runTest: (DataFrame, String, Any => Any) => Unit): Unit = { + withNestedDataFrame(inputDF).foreach { + case (newDF, colName, resultFun) => + withTempPath { + file => + newDF.write.format(dataSourceName).save(file.getCanonicalPath) + readParquetFile(file.getCanonicalPath)(df => runTest(df, colName, resultFun)) + } + } + } + + testGluten("filter pushdown - date") { + implicit class StringToDate(s: String) { + def date: Date = Date.valueOf(s) + } + + val data = Seq("1000-01-01", "2018-03-19", "2018-03-20", "2018-03-21") + import testImplicits._ + + // Velox backend does not support rebaseMode being LEGACY. + Seq(false, true).foreach { + java8Api => + Seq(CORRECTED).foreach { + rebaseMode => + withSQLConf( + SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString, + SQLConf.PARQUET_REBASE_MODE_IN_WRITE.key -> rebaseMode.toString) { + val dates = data.map(i => Tuple1(Date.valueOf(i))).toDF() + withNestedParquetDataFrame(dates) { + case (inputDF, colName, fun) => + implicit val df: DataFrame = inputDF + + def resultFun(dateStr: String): Any = { + val parsed = if (java8Api) LocalDate.parse(dateStr) else Date.valueOf(dateStr) + fun(parsed) + } + + val dateAttr: Expression = df(colName).expr + assert(df(colName).expr.dataType === DateType) + + checkFilterPredicate(dateAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate( + dateAttr.isNotNull, + classOf[NotEq[_]], + data.map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate( + dateAttr === "1000-01-01".date, + classOf[Eq[_]], + resultFun("1000-01-01")) + logWarning(s"java8Api: $java8Api, rebaseMode, $rebaseMode") + checkFilterPredicate( + dateAttr <=> "1000-01-01".date, + classOf[Eq[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + dateAttr =!= "1000-01-01".date, + classOf[NotEq[_]], + Seq("2018-03-19", "2018-03-20", "2018-03-21").map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate( + dateAttr < "2018-03-19".date, + classOf[Lt[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + dateAttr > "2018-03-20".date, + classOf[Gt[_]], + resultFun("2018-03-21")) + checkFilterPredicate( + dateAttr <= "1000-01-01".date, + classOf[LtEq[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + dateAttr >= "2018-03-21".date, + classOf[GtEq[_]], + resultFun("2018-03-21")) + + checkFilterPredicate( + Literal("1000-01-01".date) === dateAttr, + classOf[Eq[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + Literal("1000-01-01".date) <=> dateAttr, + classOf[Eq[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + Literal("2018-03-19".date) > dateAttr, + classOf[Lt[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + Literal("2018-03-20".date) < dateAttr, + classOf[Gt[_]], + resultFun("2018-03-21")) + checkFilterPredicate( + Literal("1000-01-01".date) >= dateAttr, + classOf[LtEq[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + Literal("2018-03-21".date) <= dateAttr, + classOf[GtEq[_]], + resultFun("2018-03-21")) + + checkFilterPredicate( + !(dateAttr < "2018-03-21".date), + classOf[GtEq[_]], + resultFun("2018-03-21")) + checkFilterPredicate( + dateAttr < "2018-03-19".date || dateAttr > "2018-03-20".date, + classOf[Operators.Or], + Seq(Row(resultFun("1000-01-01")), Row(resultFun("2018-03-21")))) + + Seq(3, 20).foreach { + threshold => + withSQLConf( + SQLConf.PARQUET_FILTER_PUSHDOWN_INFILTERTHRESHOLD.key -> s"$threshold") { + checkFilterPredicate( + In( + dateAttr, + Array( + "2018-03-19".date, + "2018-03-20".date, + "2018-03-21".date, + "2018-03-22".date).map(Literal.apply)), + if (threshold == 3) classOf[Operators.In[_]] else classOf[Operators.Or], + Seq( + Row(resultFun("2018-03-19")), + Row(resultFun("2018-03-20")), + Row(resultFun("2018-03-21"))) + ) + } + } + } + } + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetIOSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetIOSuite.scala new file mode 100644 index 000000000000..ad1ae40f928c --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetIOSuite.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql._ + +/** A test suite that tests basic Parquet I/O. */ +class GlutenParquetIOSuite extends ParquetIOSuite with GlutenSQLTestsBaseTrait { + override protected def testFile(fileName: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + fileName + } + + override protected def readResourceParquetFile(name: String): DataFrame = { + spark.read.parquet(testFile(name)) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetInteroperabilitySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetInteroperabilitySuite.scala new file mode 100644 index 000000000000..051343dafb06 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetInteroperabilitySuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetInteroperabilitySuite + extends ParquetInteroperabilitySuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetPartitionDiscoverySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetPartitionDiscoverySuite.scala new file mode 100644 index 000000000000..5af8fa48c538 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetPartitionDiscoverySuite.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql._ + +class GlutenParquetV1PartitionDiscoverySuite + extends ParquetV1PartitionDiscoverySuite + with GlutenSQLTestsBaseTrait {} + +class GlutenParquetV2PartitionDiscoverySuite + extends ParquetV2PartitionDiscoverySuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetProtobufCompatibilitySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetProtobufCompatibilitySuite.scala new file mode 100644 index 000000000000..f175910792bd --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetProtobufCompatibilitySuite.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait} + +class GlutenParquetProtobufCompatibilitySuite + extends ParquetProtobufCompatibilitySuite + with GlutenSQLTestsBaseTrait { + override protected def readResourceParquetFile(name: String): DataFrame = { + spark.read.parquet( + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + name) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetQuerySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetQuerySuite.scala new file mode 100644 index 000000000000..07ef21208cd0 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetQuerySuite.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql._ + +/** A test suite that tests various Parquet queries. */ +class GlutenParquetV1QuerySuite extends ParquetV1QuerySuite with GlutenSQLTestsBaseTrait { + import testImplicits._ + + testGluten( + "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") { + withAllParquetReaders { + withTempPath { + path => + // Repeated values for dictionary encoding. + Seq(Some("A"), Some("A"), None).toDF.repartition(1).write.parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath) + checkAnswer(stripSparkFilter(df.where("NOT (value <=> 'A')")), Seq(null: String).toDF) + } + } + } +} + +class GlutenParquetV2QuerySuite extends ParquetV2QuerySuite with GlutenSQLTestsBaseTrait { + import testImplicits._ + + testGluten( + "SPARK-26677: negated null-safe equality comparison " + + "should not filter matched row groups") { + withAllParquetReaders { + withTempPath { + path => + // Repeated values for dictionary encoding. + Seq(Some("A"), Some("A"), None).toDF.repartition(1).write.parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath) + checkAnswer(stripSparkFilter(df.where("NOT (value <=> 'A')")), Seq(null: String).toDF) + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetRebaseDatetimeSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetRebaseDatetimeSuite.scala new file mode 100644 index 000000000000..dbc2dac3df00 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetRebaseDatetimeSuite.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} +import org.apache.spark.sql.internal.LegacyBehaviorPolicy +import org.apache.spark.sql.internal.LegacyBehaviorPolicy.{CORRECTED, EXCEPTION, LEGACY} +import org.apache.spark.sql.internal.SQLConf + +import java.sql.Date + +class GlutenParquetRebaseDatetimeV1Suite + extends ParquetRebaseDatetimeV1Suite + with GlutenSQLTestsBaseTrait { + + import testImplicits._ + + override protected def getResourceParquetFilePath(name: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + name + } + + private def inReadConfToOptions( + conf: String, + mode: LegacyBehaviorPolicy.Value): Map[String, String] = conf match { + case SQLConf.PARQUET_INT96_REBASE_MODE_IN_READ.key => + Map(ParquetOptions.INT96_REBASE_MODE -> mode.toString) + case _ => Map(ParquetOptions.DATETIME_REBASE_MODE -> mode.toString) + } + + private def runInMode(conf: String, modes: Seq[LegacyBehaviorPolicy.Value])( + f: Map[String, String] => Unit): Unit = { + modes.foreach(mode => withSQLConf(conf -> mode.toString)(f(Map.empty))) + withSQLConf(conf -> EXCEPTION.toString) { + modes.foreach(mode => f(inReadConfToOptions(conf, mode))) + } + } + + // gluten does not consider file metadata which indicates needs rebase or not + // it only supports write the parquet file as CORRECTED + testGluten("SPARK-31159: rebasing dates in write") { + val N = 8 + Seq(false, true).foreach { + dictionaryEncoding => + withTempPath { + dir => + val path = dir.getAbsolutePath + withSQLConf(SQLConf.PARQUET_REBASE_MODE_IN_WRITE.key -> CORRECTED.toString) { + Seq + .tabulate(N)(_ => "1001-01-01") + .toDF("dateS") + .select($"dateS".cast("date").as("date")) + .repartition(1) + .write + .option("parquet.enable.dictionary", dictionaryEncoding) + .parquet(path) + } + + withAllParquetReaders { + // The file metadata indicates if it needs rebase or not, so we can always get the + // correct result regardless of the "rebase mode" config. + runInMode( + SQLConf.PARQUET_REBASE_MODE_IN_READ.key, + Seq(LEGACY, CORRECTED, EXCEPTION)) { + options => + checkAnswer( + spark.read.options(options).parquet(path), + Seq.tabulate(N)(_ => Row(Date.valueOf("1001-01-01")))) + } + + // Force to not rebase to prove the written datetime values are rebased + // and we will get wrong result if we don't rebase while reading. + // gluten not support this mode +// withSQLConf("spark.test.forceNoRebase" -> "true") { +// checkAnswer( +// spark.read.parquet(path), +// Seq.tabulate(N)(_ => Row(Date.valueOf("1001-01-07")))) +// } + } + } + } + } +} + +class GlutenParquetRebaseDatetimeV2Suite + extends ParquetRebaseDatetimeV2Suite + with GlutenSQLTestsBaseTrait { + + override protected def getResourceParquetFilePath(name: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + name + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetRowIndexSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetRowIndexSuite.scala new file mode 100644 index 000000000000..570b6d5e0c1a --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetRowIndexSuite.scala @@ -0,0 +1,361 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.{BatchScanExecTransformer, FileSourceScanExecTransformer} + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetDataSourceV2 +import org.apache.spark.sql.functions.{col, max, min} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{LongType, StringType} + +import org.apache.hadoop.fs.Path +import org.apache.parquet.column.ParquetProperties._ +import org.apache.parquet.format.converter.ParquetMetadataConverter +import org.apache.parquet.hadoop.ParquetOutputFormat +import org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE + +import java.io.File + +import scala.collection.JavaConverters._ + +class GlutenParquetRowIndexSuite extends ParquetRowIndexSuite with GlutenSQLTestsBaseTrait { + import testImplicits._ + + private def readRowGroupRowCounts(path: String): Seq[Long] = { + ParquetFooterReaderShim + .readFooter( + spark.sessionState.newHadoopConf(), + new Path(path), + ParquetMetadataConverter.NO_FILTER) + .getBlocks + .asScala + .map(_.getRowCount) + .toSeq + } + + private def readRowGroupRowCounts(dir: File): Seq[Seq[Long]] = { + assert(dir.isDirectory) + dir + .listFiles() + .filter(f => f.isFile && f.getName.endsWith("parquet")) + .map(f => readRowGroupRowCounts(f.getAbsolutePath)) + .toSeq + } + + /** Do the files contain exactly one row group? */ + private def assertOneRowGroup(dir: File): Unit = { + readRowGroupRowCounts(dir).foreach { + rcs => assert(rcs.length == 1, "expected one row group per file") + } + } + + /** + * Do the files have a good layout to test row group skipping (both range metadata filter, and by + * using min/max). + */ + private def assertTinyRowGroups(dir: File): Unit = { + readRowGroupRowCounts(dir).foreach { + rcs => + assert(rcs.length > 1, "expected multiple row groups per file") + assert(rcs.last <= DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK) + assert( + rcs.reverse.tail.distinct == Seq(DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK), + "expected row groups with minimal row count") + } + } + + /** + * Do the files have a good layout to test a combination of page skipping and row group skipping? + */ + private def assertIntermediateRowGroups(dir: File): Unit = { + readRowGroupRowCounts(dir).foreach { + rcs => + assert(rcs.length >= 3, "expected at least 3 row groups per file") + rcs.reverse.tail.foreach { + rc => + assert( + rc > DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK, + "expected row groups larger than minimal row count") + } + } + } + + case class GlutenRowIndexTestConf( + numRows: Long = 10000L, + useMultipleFiles: Boolean = false, + useVectorizedReader: Boolean = true, + useSmallPages: Boolean = false, + useSmallRowGroups: Boolean = false, + useSmallSplits: Boolean = false, + useFilter: Boolean = false, + useDataSourceV2: Boolean = false) { + + val NUM_MULTIPLE_FILES = 4 + // The test doesn't work correctly if the number of records per file is uneven. + assert(!useMultipleFiles || (numRows % NUM_MULTIPLE_FILES == 0)) + + def numFiles: Int = if (useMultipleFiles) { NUM_MULTIPLE_FILES } + else { 1 } + + def rowGroupSize: Long = if (useSmallRowGroups) { + if (useSmallPages) { + // Each file will contain multiple row groups. All of them (except for the last one) + // will contain more than DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK, so that individual + // pages within the row group can be skipped. + 2048L + } else { + // Each file will contain multiple row groups. All of them (except for the last one) + // will contain exactly DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK records. + 64L + } + } else { + // Each file will contain a single row group. + DEFAULT_BLOCK_SIZE + } + + def pageSize: Long = if (useSmallPages) { + // Each page (except for the last one for each column) will contain exactly + // DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK records. + 64L + } else { + DEFAULT_PAGE_SIZE + } + + def writeFormat: String = "parquet" + def readFormat: String = if (useDataSourceV2) { + classOf[ParquetDataSourceV2].getCanonicalName + } else { + "parquet" + } + + assert(useSmallRowGroups || !useSmallSplits) + def filesMaxPartitionBytes: Long = if (useSmallSplits) { + 256L + } else { + SQLConf.FILES_MAX_PARTITION_BYTES.defaultValue.get + } + + def desc: String = { + { if (useVectorizedReader) Seq("vectorized reader") else Seq("parquet-mr reader") } ++ { + if (useMultipleFiles) Seq("many files") else Seq.empty[String] + } ++ { if (useFilter) Seq("filtered") else Seq.empty[String] } ++ { + if (useSmallPages) Seq("small pages") else Seq.empty[String] + } ++ { if (useSmallRowGroups) Seq("small row groups") else Seq.empty[String] } ++ { + if (useSmallSplits) Seq("small splits") else Seq.empty[String] + } ++ { if (useDataSourceV2) Seq("datasource v2") else Seq.empty[String] } + }.mkString(", ") + + def sqlConfs: Seq[(String, String)] = Seq( + // TODO: remove this change after customized parquet options as `block_size`, `page_size` + // been fully supported. + GlutenConfig.NATIVE_WRITER_ENABLED.key -> "false", + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> useVectorizedReader.toString, + SQLConf.FILES_MAX_PARTITION_BYTES.key -> filesMaxPartitionBytes.toString + ) ++ { if (useDataSourceV2) Seq(SQLConf.USE_V1_SOURCE_LIST.key -> "") else Seq.empty } + } + + for (useVectorizedReader <- Seq(true, false)) + for (useDataSourceV2 <- Seq(true, false)) + for (useSmallRowGroups <- Seq(true, false)) + for (useSmallPages <- Seq(true, false)) + for (useFilter <- Seq(true, false)) + for (useSmallSplits <- Seq(useSmallRowGroups, false).distinct) { + val conf = GlutenRowIndexTestConf( + useVectorizedReader = useVectorizedReader, + useDataSourceV2 = useDataSourceV2, + useSmallRowGroups = useSmallRowGroups, + useSmallPages = useSmallPages, + useFilter = useFilter, + useSmallSplits = useSmallSplits + ) + testRowIndexGeneration("row index generation", conf) + } + + private def testRowIndexGeneration(label: String, conf: GlutenRowIndexTestConf): Unit = { + testGluten(s"$label - ${conf.desc}") { + withSQLConf(conf.sqlConfs: _*) { + withTempPath { + path => + // Read row index using _metadata.row_index if that is supported by the file format. + val rowIndexMetadataColumnSupported = conf.readFormat match { + case "parquet" => true + case _ => false + } + val rowIndexColName = if (rowIndexMetadataColumnSupported) { + s"${FileFormat.METADATA_NAME}.${ParquetFileFormat.ROW_INDEX}" + } else { + ParquetFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME + } + val numRecordsPerFile = conf.numRows / conf.numFiles + val (skipCentileFirst, skipCentileMidLeft, skipCentileMidRight, skipCentileLast) = + (0.2, 0.4, 0.6, 0.8) + val expectedRowIdxCol = "expected_rowIdx_col" + val df = spark + .range(0, conf.numRows, 1, conf.numFiles) + .toDF("id") + .withColumn("dummy_col", ($"id" / 55).cast("int")) + .withColumn(expectedRowIdxCol, ($"id" % numRecordsPerFile).cast("int")) + + // Add row index to schema if required. + val schemaWithRowIdx = if (rowIndexMetadataColumnSupported) { + df.schema + } else { + df.schema.add(rowIndexColName, LongType, nullable = true) + } + + df.write + .format(conf.writeFormat) + .option(ParquetOutputFormat.BLOCK_SIZE, conf.rowGroupSize) + .option(ParquetOutputFormat.PAGE_SIZE, conf.pageSize) + .option(ParquetOutputFormat.DICTIONARY_PAGE_SIZE, conf.pageSize) + .save(path.getAbsolutePath) + val dfRead = spark.read + .format(conf.readFormat) + .schema(schemaWithRowIdx) + .load(path.getAbsolutePath) + // Verify that the produced files are laid out as expected. + if (conf.useSmallRowGroups) { + if (conf.useSmallPages) { + assertIntermediateRowGroups(path) + } else { + assertTinyRowGroups(path) + } + } else { + assertOneRowGroup(path) + } + + val dfToAssert = if (conf.useFilter) { + // Add a filter such that we skip 60% of the records: + // [0%, 20%], [40%, 60%], [80%, 100%] + dfRead.filter( + ($"id" >= (skipCentileFirst * conf.numRows).toInt && + $"id" < (skipCentileMidLeft * conf.numRows).toInt) || + ($"id" >= (skipCentileMidRight * conf.numRows).toInt && + $"id" < (skipCentileLast * conf.numRows).toInt)) + } else { + dfRead + } + + var numPartitions: Long = 0 + var numOutputRows: Long = 0 + dfToAssert.collect() + logInfo(dfToAssert.queryExecution.executedPlan.toString()) + dfToAssert.queryExecution.executedPlan.foreach { + case a: BatchScanExec => + numPartitions += a.inputRDD.partitions.length + numOutputRows += a.metrics("numOutputRows").value + case b: FileSourceScanExec => + numPartitions += b.inputRDD.partitions.length + numOutputRows += b.metrics("numOutputRows").value + case c: BatchScanExecTransformer => + numPartitions += c.inputRDD.partitions.length + numOutputRows += c.metrics("numOutputRows").value + case f: FileSourceScanExecTransformer => + numPartitions += f.inputRDD.partitions.length + numOutputRows += f.metrics("numOutputRows").value + case _ => + } + assert(numPartitions > 0) + assert(numOutputRows > 0) + + if (conf.useSmallSplits) { + assert(numPartitions >= 2 * conf.numFiles) + } + + // Assert that every rowIdx value matches the value in `expectedRowIdx`. + assert( + dfToAssert + .filter(s"$rowIndexColName != $expectedRowIdxCol") + .count() == 0) + + if (conf.useFilter) { + if (conf.useSmallRowGroups) { + assert(numOutputRows < conf.numRows) + } + + val minMaxRowIndexes = + dfToAssert.select(max(col(rowIndexColName)), min(col(rowIndexColName))).collect() + val (expectedMaxRowIdx, expectedMinRowIdx) = if (conf.numFiles == 1) { + // When there is a single file, we still have row group skipping, + // but that should not affect the produced rowIdx. + (conf.numRows * skipCentileLast - 1, conf.numRows * skipCentileFirst) + } else { + // For simplicity, the chosen filter skips the whole files. + // Thus all unskipped files will have the same max and min rowIdx values. + (numRecordsPerFile - 1, 0) + } + assert(minMaxRowIndexes(0).get(0) == expectedMaxRowIdx) + assert(minMaxRowIndexes(0).get(1) == expectedMinRowIdx) + if (!conf.useMultipleFiles) { + val skippedValues = List.range(0, (skipCentileFirst * conf.numRows).toInt) ++ + List.range( + (skipCentileMidLeft * conf.numRows).toInt, + (skipCentileMidRight * conf.numRows).toInt) ++ + List.range((skipCentileLast * conf.numRows).toInt, conf.numRows) + // rowIdx column should not have any of the `skippedValues`. + assert( + dfToAssert + .filter(col(rowIndexColName).isin(skippedValues: _*)) + .count() == 0) + } + } else { + // assert(numOutputRows == conf.numRows) + // When there is no filter, the rowIdx values should be in range + // [0-`numRecordsPerFile`]. + val expectedRowIdxValues = List.range(0, numRecordsPerFile) + assert( + dfToAssert + .filter(col(rowIndexColName).isin(expectedRowIdxValues: _*)) + .count() == conf.numRows) + } + } + } + } + } + for (useDataSourceV2 <- Seq(true, false)) { + val conf = GlutenRowIndexTestConf(useDataSourceV2 = useDataSourceV2) + + testGluten(s"invalid row index column type - ${conf.desc}") { + withSQLConf(conf.sqlConfs: _*) { + withTempPath { + path => + val df = spark.range(0, 10, 1, 1).toDF("id") + val schemaWithRowIdx = df.schema + .add(ParquetFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME, StringType) + + df.write + .format(conf.writeFormat) + .save(path.getAbsolutePath) + + val dfRead = spark.read + .format(conf.readFormat) + .schema(schemaWithRowIdx) + .load(path.getAbsolutePath) + + val exception = intercept[Exception](dfRead.collect()) + assert(exception.getMessage.contains(ParquetFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME)) + } + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetSchemaPruningSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetSchemaPruningSuite.scala new file mode 100644 index 000000000000..854961e48f0a --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetSchemaPruningSuite.scala @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet +import org.apache.gluten.execution.{BatchScanExecTransformer, FileSourceScanExecTransformer} + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.tags.ExtendedSQLTest + +@ExtendedSQLTest +class GlutenParquetV1SchemaPruningSuite + extends ParquetV1SchemaPruningSuite + with GlutenSQLTestsBaseTrait { + // disable column reader for nested type + override protected val vectorizedReaderNestedEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key + "_DISABLED" + override def sparkConf: SparkConf = { + super.sparkConf.set("spark.memory.offHeap.size", "3g") + } + + override def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { + val fileSourceScanSchemata = + collect(df.queryExecution.executedPlan) { + case scan: FileSourceScanExec => scan.requiredSchema + case scan: FileSourceScanExecTransformer => scan.requiredSchema + } + assert( + fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, + s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + + s"but expected $expectedSchemaCatalogStrings" + ) + fileSourceScanSchemata.zip(expectedSchemaCatalogStrings).foreach { + case (scanSchema, expectedScanSchemaCatalogString) => + val expectedScanSchema = CatalystSqlParser.parseDataType(expectedScanSchemaCatalogString) + implicit val equality = schemaEquality + assert(scanSchema === expectedScanSchema) + } + } +} + +@ExtendedSQLTest +class GlutenParquetV2SchemaPruningSuite + extends ParquetV2SchemaPruningSuite + with GlutenSQLTestsBaseTrait { + override protected val vectorizedReaderNestedEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key + "_DISABLED" + override def sparkConf: SparkConf = { + super.sparkConf.set("spark.memory.offHeap.size", "3g") + } + + override def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { + val fileSourceScanSchemata = + collect(df.queryExecution.executedPlan) { + case scan: BatchScanExec => scan.scan.asInstanceOf[ParquetScan].readDataSchema + case scan: BatchScanExecTransformer => scan.scan.asInstanceOf[ParquetScan].readDataSchema + } + assert( + fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, + s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + + s"but expected $expectedSchemaCatalogStrings" + ) + fileSourceScanSchemata.zip(expectedSchemaCatalogStrings).foreach { + case (scanSchema, expectedScanSchemaCatalogString) => + val expectedScanSchema = CatalystSqlParser.parseDataType(expectedScanSchemaCatalogString) + implicit val equality = schemaEquality + assert(scanSchema === expectedScanSchema) + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetSchemaSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetSchemaSuite.scala new file mode 100644 index 000000000000..b4eedeab3939 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetSchemaSuite.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.SparkException +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.Cast.toSQLType +import org.apache.spark.sql.types.{IntegerType, LongType, StructField, StructType} + +class GlutenParquetSchemaInferenceSuite + extends ParquetSchemaInferenceSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenParquetSchemaSuite extends ParquetSchemaSuite with GlutenSQLTestsBaseTrait { + + override protected def testFile(fileName: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + fileName + } + + testGluten("CANNOT_MERGE_SCHEMAS: Failed merging schemas") { + import testImplicits._ + + withTempPath { + dir => + val path = dir.getCanonicalPath + + // Note: Velox backend always generates Parquet files with nullable = true, + // regardless of whether nullable is set to false or true in the schema. + // Before https://github.com/apache/spark/pull/44644, `StructField.sql` would not + // return the `NOT NULL` qualifier. This is why this test succeeds in Spark 3.5. + val schema1 = StructType(Seq(StructField("id", LongType, nullable = true))) + val df1 = spark.createDataFrame( + spark.sparkContext.parallelize(Seq(Row(0L), Row(1L), Row(2L))), + schema1) + df1.write.parquet(s"$path/p=1") + val df2 = df1.select($"id".cast(IntegerType).as(Symbol("id"))) + df2.write.parquet(s"$path/p=2") + + checkError( + exception = intercept[SparkException] { + spark.read.option("mergeSchema", "true").parquet(path) + }, + condition = "CANNOT_MERGE_SCHEMAS", + sqlState = "42KD9", + parameters = Map("left" -> toSQLType(df1.schema), "right" -> toSQLType(df2.schema)) + ) + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetThriftCompatibilitySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetThriftCompatibilitySuite.scala new file mode 100644 index 000000000000..6dc41c090244 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetThriftCompatibilitySuite.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} + +class GlutenParquetThriftCompatibilitySuite + extends ParquetThriftCompatibilitySuite + with GlutenSQLTestsBaseTrait { + + private val parquetFilePath = + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + + "/test-data/parquet-thrift-compat.snappy.parquet" + + testGluten("Read Parquet file generated by parquet-thrift") { + logInfo(s"""Schema of the Parquet file written by parquet-thrift: + |${readParquetSchema(parquetFilePath.toString)} + """.stripMargin) + + checkAnswer( + spark.read.parquet(parquetFilePath.toString), + (0 until 10).map { + i => + val suits = Array("SPADES", "HEARTS", "DIAMONDS", "CLUBS") + + val nonNullablePrimitiveValues = Seq( + i % 2 == 0, + i.toByte, + (i + 1).toShort, + i + 2, + i.toLong * 10, + i.toDouble + 0.2d, + // Thrift `BINARY` values are actually unencoded `STRING` values, and thus are always + // treated as `BINARY (UTF8)` in parquet-thrift, since parquet-thrift always assume + // Thrift `STRING`s are encoded using UTF-8. + s"val_$i", + s"val_$i", + // Thrift ENUM values are converted to Parquet binaries containing UTF-8 strings + suits(i % 4) + ) + + val nullablePrimitiveValues = if (i % 3 == 0) { + Seq.fill(nonNullablePrimitiveValues.length)(null) + } else { + nonNullablePrimitiveValues + } + + val complexValues = Seq( + Seq.tabulate(3)(n => s"arr_${i + n}"), + // Thrift `SET`s are converted to Parquet `LIST`s + Seq(i), + Seq.tabulate(3)(n => (i + n: Integer) -> s"val_${i + n}").toMap, + Seq + .tabulate(3) { + n => + (i + n) -> Seq.tabulate(3) { + m => Row(Seq.tabulate(3)(j => i + j + m), s"val_${i + m}") + } + } + .toMap + ) + + Row(nonNullablePrimitiveValues ++ nullablePrimitiveValues ++ complexValues: _*) + } + ) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetVectorizedSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetVectorizedSuite.scala new file mode 100644 index 000000000000..a0cf738e52a6 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetVectorizedSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetVectorizedSuite extends ParquetVectorizedSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/text/GlutenTextSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/text/GlutenTextSuite.scala new file mode 100644 index 000000000000..845af160c74d --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/text/GlutenTextSuite.scala @@ -0,0 +1,290 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.text + +import org.apache.spark.{SparkConf, SparkIllegalArgumentException, TestUtils} +import org.apache.spark.sql.{AnalysisException, DataFrame, GlutenSQLTestsBaseTrait, QueryTest, Row, SaveMode} +import org.apache.spark.sql.execution.datasources.CommonFileDataSourceSuite +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.util.Utils + +import org.apache.hadoop.io.SequenceFile.CompressionType +import org.apache.hadoop.io.compress.GzipCodec + +import java.io.File +import java.nio.charset.StandardCharsets +import java.nio.file.Files + +abstract class GlutenTextSuite + extends QueryTest + with SharedSparkSession + with CommonFileDataSourceSuite + with GlutenSQLTestsBaseTrait { + import testImplicits._ + + override protected def dataSourceFormat = "text" + + testGluten("reading text file") { + verifyFrame(spark.read.format("text").load(testFile)) + } + + testGluten("SQLContext.read.text() API") { + verifyFrame(spark.read.text(testFile)) + } + + testGluten("SPARK-12562 verify write.text() can handle column name beyond `value`") { + val df = spark.read.text(testFile).withColumnRenamed("value", "adwrasdf") + + val tempFile = Utils.createTempDir() + tempFile.delete() + df.write.text(tempFile.getCanonicalPath) + verifyFrame(spark.read.text(tempFile.getCanonicalPath)) + + Utils.deleteRecursively(tempFile) + } + + testGluten("error handling for invalid schema") { + val tempFile = Utils.createTempDir() + tempFile.delete() + + val df = spark.range(2) + intercept[AnalysisException] { + df.write.text(tempFile.getCanonicalPath) + } + + intercept[AnalysisException] { + spark.range(2).select(df("id"), df("id") + 1).write.text(tempFile.getCanonicalPath) + } + } + + testGluten("reading partitioned data using read.textFile()") { + val ds = spark.read.textFile(textPartitioned) + val data = ds.collect() + + assert(ds.schema == new StructType().add("value", StringType)) + assert(data.length == 2) + } + + testGluten("support for partitioned reading using read.text()") { + val df = spark.read.text(textPartitioned) + val data = df.filter("year = '2015'").select("value").collect() + + assert(data(0) == Row("2015-test")) + assert(data.length == 1) + } + + testGluten("SPARK-13503 Support to specify the option for compression codec for TEXT") { + val testDf = spark.read.text(testFile) + val extensionNameMap = Map("bzip2" -> ".bz2", "deflate" -> ".deflate", "gzip" -> ".gz") + extensionNameMap.foreach { + case (codecName, extension) => + val tempDir = Utils.createTempDir() + val tempDirPath = tempDir.getAbsolutePath + testDf.write.option("compression", codecName).mode(SaveMode.Overwrite).text(tempDirPath) + val compressedFiles = new File(tempDirPath).listFiles() + assert(compressedFiles.exists(_.getName.endsWith(s".txt$extension"))) + verifyFrame(spark.read.text(tempDirPath)) + } + + withTempDir { + dir => + checkError( + exception = intercept[SparkIllegalArgumentException] { + testDf.write + .option("compression", "illegal") + .mode(SaveMode.Overwrite) + .text(dir.getAbsolutePath) + }, + condition = "CODEC_NOT_AVAILABLE.WITH_AVAILABLE_CODECS_SUGGESTION", + parameters = Map( + "codecName" -> "illegal", + "availableCodecs" -> "bzip2, deflate, uncompressed, snappy, none, lz4, gzip") + ) + } + } + + testGluten("SPARK-13543 Write the output as uncompressed via option()") { + val extraOptions = Map[String, String]( + "mapreduce.output.fileoutputformat.compress" -> "true", + "mapreduce.output.fileoutputformat.compress.type" -> CompressionType.BLOCK.toString, + "mapreduce.map.output.compress" -> "true", + "mapreduce.output.fileoutputformat.compress.codec" -> classOf[GzipCodec].getName, + "mapreduce.map.output.compress.codec" -> classOf[GzipCodec].getName + ) + withTempDir { + dir => + val testDf = spark.read.text(testFile) + val tempDirPath = dir.getAbsolutePath + testDf.write + .option("compression", "none") + .options(extraOptions) + .mode(SaveMode.Overwrite) + .text(tempDirPath) + val compressedFiles = new File(tempDirPath).listFiles() + assert(compressedFiles.exists(!_.getName.endsWith(".txt.gz"))) + verifyFrame(spark.read.options(extraOptions).text(tempDirPath)) + } + } + + testGluten("case insensitive option") { + val extraOptions = Map[String, String]( + "mApReDuCe.output.fileoutputformat.compress" -> "true", + "mApReDuCe.output.fileoutputformat.compress.type" -> CompressionType.BLOCK.toString, + "mApReDuCe.map.output.compress" -> "true", + "mApReDuCe.output.fileoutputformat.compress.codec" -> classOf[GzipCodec].getName, + "mApReDuCe.map.output.compress.codec" -> classOf[GzipCodec].getName + ) + withTempDir { + dir => + val testDf = spark.read.text(testFile) + val tempDirPath = dir.getAbsolutePath + testDf.write + .option("CoMpReSsIoN", "none") + .options(extraOptions) + .mode(SaveMode.Overwrite) + .text(tempDirPath) + val compressedFiles = new File(tempDirPath).listFiles() + assert(compressedFiles.exists(!_.getName.endsWith(".txt.gz"))) + verifyFrame(spark.read.options(extraOptions).text(tempDirPath)) + } + } + + testGluten("SPARK-14343: select partitioning column") { + withTempPath { + dir => + val path = dir.getCanonicalPath + val ds1 = spark.range(1).selectExpr("CONCAT('val_', id)") + ds1.write.text(s"$path/part=a") + ds1.write.text(s"$path/part=b") + + checkAnswer( + spark.read.format("text").load(path).select($"part"), + Row("a") :: Row("b") :: Nil) + } + } + + testGluten("SPARK-15654: should not split gz files") { + withTempDir { + dir => + val path = dir.getCanonicalPath + val df1 = spark.range(0, 1000).selectExpr("CAST(id AS STRING) AS s") + df1.write.option("compression", "gzip").mode("overwrite").text(path) + + val expected = df1.collect() + Seq(10, 100, 1000).foreach { + bytes => + withSQLConf(SQLConf.FILES_MAX_PARTITION_BYTES.key -> bytes.toString) { + val df2 = spark.read.format("text").load(path) + checkAnswer(df2, expected) + } + } + } + } + + def testLineSeparator(lineSep: String): Unit = { + test(s"SPARK-23577: Support line separator - lineSep: '$lineSep'") { + // Read + val values = Seq("a", "b", "\nc") + val data = values.mkString(lineSep) + val dataWithTrailingLineSep = s"$data$lineSep" + Seq(data, dataWithTrailingLineSep).foreach { + lines => + withTempPath { + path => + Files.write(path.toPath, lines.getBytes(StandardCharsets.UTF_8)) + val df = spark.read.option("lineSep", lineSep).text(path.getAbsolutePath) + checkAnswer(df, Seq("a", "b", "\nc").toDF()) + } + } + + // Write + withTempPath { + path => + values.toDF().coalesce(1).write.option("lineSep", lineSep).text(path.getAbsolutePath) + val partFile = + TestUtils.recursiveList(path).filter(f => f.getName.startsWith("part-")).head + val readBack = new String(Files.readAllBytes(partFile.toPath), StandardCharsets.UTF_8) + assert(readBack === s"a${lineSep}b$lineSep\nc$lineSep") + } + + // Roundtrip + withTempPath { + path => + val df = values.toDF() + df.write.option("lineSep", lineSep).text(path.getAbsolutePath) + val readBack = spark.read.option("lineSep", lineSep).text(path.getAbsolutePath) + checkAnswer(df, readBack) + } + } + } + + // scalastyle:off nonascii + Seq("|", "^", "::", "!!!@3", 0x1e.toChar.toString, "아").foreach { + lineSep => testLineSeparator(lineSep) + } + // scalastyle:on nonascii + + // Rewrite for file locating. + private def testFile: String = { + getWorkspaceFilePath( + "sql", + "core", + "src", + "test", + "resources").toString + "/test-data/text-suite.txt" + } + + // Added for file locating. + private def textPartitioned: String = { + getWorkspaceFilePath( + "sql", + "core", + "src", + "test", + "resources").toString + "/test-data/text-partitioned" + } + + /** Verifies data and schema. */ + private def verifyFrame(df: DataFrame): Unit = { + // schema + assert(df.schema == new StructType().add("value", StringType)) + + // verify content + val data = df.collect() + assert(data(0) == Row("This is a test file for the text data source")) + assert(data(1) == Row("1+1")) + // scalastyle:off nonascii + assert(data(2) == Row("数据砖头")) + // scalastyle:on nonascii + assert(data(3) == Row("\"doh\"")) + assert(data.length == 4) + } +} + +class GlutenTextV1Suite extends GlutenTextSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "text") +} + +class GlutenTextV2Suite extends GlutenTextSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenDataSourceV2StrategySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenDataSourceV2StrategySuite.scala new file mode 100644 index 000000000000..f6d7db3849e9 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenDataSourceV2StrategySuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceV2StrategySuite + extends DataSourceV2StrategySuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenFileTableSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenFileTableSuite.scala new file mode 100644 index 000000000000..bc6fcc3c0e9b --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenFileTableSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenFileTableSuite extends FileTableSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenV2PredicateSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenV2PredicateSuite.scala new file mode 100644 index 000000000000..e2d8186f6874 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenV2PredicateSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenV2PredicateSuite extends V2PredicateSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/exchange/GlutenEnsureRequirementsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/exchange/GlutenEnsureRequirementsSuite.scala new file mode 100644 index 000000000000..1409b598ac63 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/exchange/GlutenEnsureRequirementsSuite.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.exchange + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenEnsureRequirementsSuite extends EnsureRequirementsSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = { + // Native SQL configs + super.sparkConf + .set("spark.sql.shuffle.partitions", "5") + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/exchange/GlutenValidateRequirementsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/exchange/GlutenValidateRequirementsSuite.scala new file mode 100644 index 000000000000..a1467af2c5bb --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/exchange/GlutenValidateRequirementsSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.exchange + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenValidateRequirementsSuite + extends ValidateRequirementsSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala new file mode 100644 index 000000000000..8062596bdf6e --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.joins + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.utils.BackendTestUtils + +import org.apache.spark.sql.{GlutenTestsCommonTrait, SparkSession} +import org.apache.spark.sql.catalyst.optimizer.{ConstantFolding, ConvertToLocalRelation, NullPropagation} +import org.apache.spark.sql.classic.ClassicConversions._ +import org.apache.spark.sql.internal.SQLConf + +/** + * This test needs setting for spark test home (its source code), e.g., appending the following + * setting for `mvn test`: -DargLine="-Dspark.test.home=/home/sparkuser/spark/". + * + * In addition, you also need build spark source code before running this test, e.g., with + * `./build/mvn -DskipTests clean package`. + */ +class GlutenBroadcastJoinSuite extends BroadcastJoinSuite with GlutenTestsCommonTrait { + + /** + * Create a new [[SparkSession]] running in local-cluster mode with unsafe and codegen enabled. + */ + override def beforeAll(): Unit = { + super.beforeAll() + val sparkBuilder = SparkSession + .builder() + .master("local-cluster[2,1,1024]") + .appName("Gluten-UT") + .master(s"local[2]") + .config(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) + .config("spark.driver.memory", "1G") + .config("spark.sql.adaptive.enabled", "true") + .config("spark.sql.shuffle.partitions", "1") + .config("spark.sql.files.maxPartitionBytes", "134217728") + .config("spark.memory.offHeap.enabled", "true") + .config("spark.memory.offHeap.size", "1024MB") + .config("spark.plugins", "org.apache.gluten.GlutenPlugin") + .config("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .config("spark.sql.warehouse.dir", warehouse) + // Avoid static evaluation for literal input by spark catalyst. + .config( + "spark.sql.optimizer.excludedRules", + ConstantFolding.ruleName + "," + + NullPropagation.ruleName) + // Avoid the code size overflow error in Spark code generation. + .config("spark.sql.codegen.wholeStage", "false") + + spark = if (BackendTestUtils.isCHBackendLoaded()) { + sparkBuilder + .config("spark.io.compression.codec", "LZ4") + .config("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .config(GlutenConfig.NATIVE_VALIDATION_ENABLED.key, "false") + .config("spark.sql.files.openCostInBytes", "134217728") + .config("spark.unsafe.exceptionOnMemoryLeak", "true") + .getOrCreate() + } else { + sparkBuilder + .config("spark.unsafe.exceptionOnMemoryLeak", "true") + .getOrCreate() + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/joins/GlutenExistenceJoinSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/joins/GlutenExistenceJoinSuite.scala new file mode 100644 index 000000000000..309af61a43ae --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/joins/GlutenExistenceJoinSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.joins + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenExistenceJoinSuite extends ExistenceJoinSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala new file mode 100644 index 000000000000..94d5629107df --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.joins + +import org.apache.gluten.config.GlutenConfig + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenInnerJoinSuiteForceShjOn extends InnerJoinSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "true") + } +} + +class GlutenInnerJoinSuiteForceShjOff extends InnerJoinSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala new file mode 100644 index 000000000000..04a555cfce30 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.joins + +import org.apache.gluten.config.GlutenConfig + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenOuterJoinSuiteForceShjOn extends OuterJoinSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "true") + } +} + +class GlutenOuterJoinSuiteForceShjOff extends OuterJoinSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/python/GlutenBatchEvalPythonExecSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/python/GlutenBatchEvalPythonExecSuite.scala new file mode 100644 index 000000000000..d364262be588 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/python/GlutenBatchEvalPythonExecSuite.scala @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.python + +import org.apache.gluten.execution.{ColumnarToRowExecBase, FilterExecTransformer, RowToColumnarExecBase, WholeStageTransformer} + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, GreaterThan, In} +import org.apache.spark.sql.execution.{ColumnarInputAdapter, InputIteratorTransformer} + +class GlutenBatchEvalPythonExecSuite extends BatchEvalPythonExecSuite with GlutenSQLTestsBaseTrait { + + import testImplicits._ + + testGluten("Python UDF: push down deterministic FilterExecTransformer predicates") { + val df = Seq(("Hello", 4)) + .toDF("a", "b") + .where("dummyPythonUDF(b) and dummyPythonUDF(a) and a in (3, 4)") + val qualifiedPlanNodes = df.queryExecution.executedPlan.collect { + case f @ FilterExecTransformer( + And(_: AttributeReference, _: AttributeReference), + InputIteratorTransformer(ColumnarInputAdapter(r: RowToColumnarExecBase))) + if r.child.isInstanceOf[BatchEvalPythonExec] => + f + case b @ BatchEvalPythonExec(_, _, c: ColumnarToRowExecBase) => + c.child match { + case WholeStageTransformer(FilterExecTransformer(_: In, _), _) => b + } + } + assert(qualifiedPlanNodes.size == 2) + } + + testGluten("Nested Python UDF: push down deterministic FilterExecTransformer predicates") { + val df = Seq(("Hello", 4)) + .toDF("a", "b") + .where("dummyPythonUDF(a, dummyPythonUDF(a, b)) and a in (3, 4)") + val qualifiedPlanNodes = df.queryExecution.executedPlan.collect { + case f @ FilterExecTransformer( + _: AttributeReference, + InputIteratorTransformer(ColumnarInputAdapter(r: RowToColumnarExecBase))) + if r.child.isInstanceOf[BatchEvalPythonExec] => + f + case b @ BatchEvalPythonExec(_, _, c: ColumnarToRowExecBase) => + c.child match { + case WholeStageTransformer(FilterExecTransformer(_: In, _), _) => b + } + } + assert(qualifiedPlanNodes.size == 2) + } + + testGluten("Python UDF: no push down on non-deterministic") { + val df = Seq(("Hello", 4)) + .toDF("a", "b") + .where("b > 4 and dummyPythonUDF(a) and rand() > 0.3") + val qualifiedPlanNodes = df.queryExecution.executedPlan.collect { + case f @ FilterExecTransformer( + And(_: AttributeReference, _: GreaterThan), + InputIteratorTransformer(ColumnarInputAdapter(r: RowToColumnarExecBase))) + if r.child.isInstanceOf[BatchEvalPythonExec] => + f + case b @ BatchEvalPythonExec(_, _, c: ColumnarToRowExecBase) => + c.child match { + case WholeStageTransformer(_: FilterExecTransformer, _) => b + } + } + assert(qualifiedPlanNodes.size == 2) + } + + testGluten( + "Python UDF: push down on deterministic predicates after the first non-deterministic") { + val df = Seq(("Hello", 4)) + .toDF("a", "b") + .where("dummyPythonUDF(a) and rand() > 0.3 and b > 4") + + val qualifiedPlanNodes = df.queryExecution.executedPlan.collect { + case f @ FilterExecTransformer( + And(_: AttributeReference, _: GreaterThan), + InputIteratorTransformer(ColumnarInputAdapter(r: RowToColumnarExecBase))) + if r.child.isInstanceOf[BatchEvalPythonExec] => + f + case b @ BatchEvalPythonExec(_, _, c: ColumnarToRowExecBase) => + c.child match { + case WholeStageTransformer(_: FilterExecTransformer, _) => b + } + } + assert(qualifiedPlanNodes.size == 2) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/python/GlutenExtractPythonUDFsSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/python/GlutenExtractPythonUDFsSuite.scala new file mode 100644 index 000000000000..1cd34bbf7852 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/execution/python/GlutenExtractPythonUDFsSuite.scala @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.python + +import org.apache.gluten.execution.{BatchScanExecTransformer, FileSourceScanExecTransformer} + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.internal.SQLConf + +class GlutenExtractPythonUDFsSuite extends ExtractPythonUDFsSuite with GlutenSQLTestsBaseTrait { + + import testImplicits._ + + def collectBatchExec(plan: SparkPlan): Seq[BatchEvalPythonExec] = plan.collect { + case b: BatchEvalPythonExec => b + } + + def collectColumnarArrowExec(plan: SparkPlan): Seq[EvalPythonExec] = plan.collect { + // To check for ColumnarArrowEvalPythonExec + case b: EvalPythonExec + if !b.isInstanceOf[ArrowEvalPythonExec] && !b.isInstanceOf[BatchEvalPythonExec] => + b + } + + testGluten("Chained Scalar Pandas UDFs should be combined to a single physical node") { + val df = Seq(("Hello", 4)).toDF("a", "b") + val df2 = df + .withColumn("c", scalarPandasUDF(col("a"))) + .withColumn("d", scalarPandasUDF(col("c"))) + val arrowEvalNodes = collectColumnarArrowExec(df2.queryExecution.executedPlan) + assert(arrowEvalNodes.size == 1) + } + + testGluten("Mixed Batched Python UDFs and Pandas UDF should be separate physical node") { + val df = Seq(("Hello", 4)).toDF("a", "b") + val df2 = df + .withColumn("c", batchedPythonUDF(col("a"))) + .withColumn("d", scalarPandasUDF(col("b"))) + + val pythonEvalNodes = collectBatchExec(df2.queryExecution.executedPlan) + val arrowEvalNodes = collectColumnarArrowExec(df2.queryExecution.executedPlan) + assert(pythonEvalNodes.size == 1) + assert(arrowEvalNodes.size == 1) + } + + testGluten( + "Independent Batched Python UDFs and Scalar Pandas UDFs should be combined separately") { + val df = Seq(("Hello", 4)).toDF("a", "b") + val df2 = df + .withColumn("c1", batchedPythonUDF(col("a"))) + .withColumn("c2", batchedPythonUDF(col("c1"))) + .withColumn("d1", scalarPandasUDF(col("a"))) + .withColumn("d2", scalarPandasUDF(col("d1"))) + + val pythonEvalNodes = collectBatchExec(df2.queryExecution.executedPlan) + val arrowEvalNodes = collectColumnarArrowExec(df2.queryExecution.executedPlan) + assert(pythonEvalNodes.size == 1) + assert(arrowEvalNodes.size == 1) + } + + testGluten("Dependent Batched Python UDFs and Scalar Pandas UDFs should not be combined") { + val df = Seq(("Hello", 4)).toDF("a", "b") + val df2 = df + .withColumn("c1", batchedPythonUDF(col("a"))) + .withColumn("d1", scalarPandasUDF(col("c1"))) + .withColumn("c2", batchedPythonUDF(col("d1"))) + .withColumn("d2", scalarPandasUDF(col("c2"))) + + val pythonEvalNodes = collectBatchExec(df2.queryExecution.executedPlan) + val arrowEvalNodes = collectColumnarArrowExec(df2.queryExecution.executedPlan) + assert(pythonEvalNodes.size == 2) + assert(arrowEvalNodes.size == 2) + } + + testGluten("Python UDF should not break column pruning/filter pushdown -- Parquet V2") { + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { + withTempPath { + f => + spark.range(10).select($"id".as("a"), $"id".as("b")).write.parquet(f.getCanonicalPath) + val df = spark.read.parquet(f.getCanonicalPath) + + withClue("column pruning") { + val query = df.filter(batchedPythonUDF($"a")).select($"a") + + val pythonEvalNodes = collectBatchExec(query.queryExecution.executedPlan) + assert(pythonEvalNodes.length == 1) + + val scanNodes = query.queryExecution.executedPlan.collect { + case scan: BatchScanExecTransformer => scan + } + assert(scanNodes.length == 1) + assert(scanNodes.head.output.map(_.name) == Seq("a")) + } + + withClue("filter pushdown") { + val query = df.filter($"a" > 1 && batchedPythonUDF($"a")) + val pythonEvalNodes = collectBatchExec(query.queryExecution.executedPlan) + assert(pythonEvalNodes.length == 1) + + val scanNodes = query.queryExecution.executedPlan.collect { + case scan: BatchScanExecTransformer => scan + } + assert(scanNodes.length == 1) + // $"a" is not null and $"a" > 1 + val filters = scanNodes.head.scan.asInstanceOf[ParquetScan].pushedFilters + assert(filters.length == 2) + assert(filters.flatMap(_.references).distinct === Array("a")) + } + } + } + } + + testGluten("Python UDF should not break column pruning/filter pushdown -- Parquet V1") { + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "parquet") { + withTempPath { + f => + spark.range(10).select($"id".as("a"), $"id".as("b")).write.parquet(f.getCanonicalPath) + val df = spark.read.parquet(f.getCanonicalPath) + + withClue("column pruning") { + val query = df.filter(batchedPythonUDF($"a")).select($"a") + + val pythonEvalNodes = collectBatchExec(query.queryExecution.executedPlan) + assert(pythonEvalNodes.length == 1) + + val scanNodes = query.queryExecution.executedPlan.collect { + case scan: FileSourceScanExecTransformer => scan + } + assert(scanNodes.length == 1) + assert(scanNodes.head.output.map(_.name) == Seq("a")) + } + + withClue("filter pushdown") { + val query = df.filter($"a" > 1 && batchedPythonUDF($"a")) + val pythonEvalNodes = collectBatchExec(query.queryExecution.executedPlan) + assert(pythonEvalNodes.length == 1) + + val scanNodes = query.queryExecution.executedPlan.collect { + case scan: FileSourceScanExecTransformer => scan + } + assert(scanNodes.length == 1) + // $"a" is not null and $"a" > 1 + assert(scanNodes.head.dataFilters.length == 2) + assert( + scanNodes.head.dataFilters.flatMap(_.references.map(_.name)).distinct == Seq("a")) + } + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/extension/CustomerColumnarPreRules.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/extension/CustomerColumnarPreRules.scala new file mode 100644 index 000000000000..5de35daaedf9 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/extension/CustomerColumnarPreRules.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.extension + +import org.apache.gluten.sql.shims.SparkShimLoader + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} + +case class CustomerColumnarPreRules(session: SparkSession) extends Rule[SparkPlan] { + + override def apply(plan: SparkPlan): SparkPlan = plan.transformDown { + case fileSourceScan: FileSourceScanExec => + val transformer = new TestFileSourceScanExecTransformer( + fileSourceScan.relation, + SparkShimLoader.getSparkShims.getFileSourceScanStream(fileSourceScan), + fileSourceScan.output, + fileSourceScan.requiredSchema, + fileSourceScan.partitionFilters, + fileSourceScan.optionalBucketSet, + fileSourceScan.optionalNumCoalescedBuckets, + fileSourceScan.dataFilters, + fileSourceScan.tableIdentifier, + fileSourceScan.disableBucketedScan + ) + if (transformer.doValidate().ok()) { + transformer + } else { + plan + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/extension/GlutenCollapseProjectExecTransformerSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/extension/GlutenCollapseProjectExecTransformerSuite.scala new file mode 100644 index 000000000000..49e4187babf4 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/extension/GlutenCollapseProjectExecTransformerSuite.scala @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.extension + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.ProjectExecTransformer + +import org.apache.spark.sql.GlutenSQLTestsTrait +import org.apache.spark.sql.Row +import org.apache.spark.sql.execution.SparkPlan + +class GlutenCollapseProjectExecTransformerSuite extends GlutenSQLTestsTrait { + + import testImplicits._ + + testGluten("Support ProjectExecTransformer collapse") { + val query = + """ + |SELECT + | o_orderpriority + |FROM + | orders + |WHERE + | o_shippriority >= 0 + | AND EXISTS ( + | SELECT + | * + | FROM + | lineitem + | WHERE + | l_orderkey = o_orderkey + | AND l_linenumber < 10 + | ) + |ORDER BY + | o_orderpriority + |LIMIT + | 100; + |""".stripMargin + + val ordersData = Seq[(Int, Int, String)]( + (30340, 1, "3-MEDIUM"), + (31140, 1, "1-URGENT"), + (31940, 1, "2-HIGH"), + (32740, 1, "3-MEDIUM"), + (33540, 1, "5-LOW"), + (34340, 1, "2-HIGH"), + (35140, 1, "3-MEDIUM"), + (35940, 1, "1-URGENT"), + (36740, 1, "3-MEDIUM"), + (37540, 1, "4-NOT SPECIFIED") + ) + val lineitemData = Seq[(Int, Int, String)]( + (30340, 1, "F"), + (31140, 4, "F"), + (31940, 7, "O"), + (32740, 6, "O"), + (33540, 2, "F"), + (34340, 3, "F"), + (35140, 1, "O"), + (35940, 2, "F"), + (36740, 3, "F"), + (37540, 5, "O") + ) + withTable("orders", "lineitem") { + ordersData + .toDF("o_orderkey", "o_shippriority", "o_orderpriority") + .write + .format("parquet") + .saveAsTable("orders") + lineitemData + .toDF("l_orderkey", "l_linenumber", "l_linestatus") + .write + .format("parquet") + .saveAsTable("lineitem") + Seq(true, false).foreach { + collapsed => + withSQLConf( + GlutenConfig.ENABLE_COLUMNAR_PROJECT_COLLAPSE.key -> collapsed.toString, + "spark.sql.autoBroadcastJoinThreshold" -> "-1") { + val df = sql(query) + checkAnswer( + df, + Seq( + Row("1-URGENT"), + Row("1-URGENT"), + Row("2-HIGH"), + Row("2-HIGH"), + Row("3-MEDIUM"), + Row("3-MEDIUM"), + Row("3-MEDIUM"), + Row("3-MEDIUM"), + Row("4-NOT SPECIFIED"), + Row("5-LOW") + ) + ) + assert( + getExecutedPlan(df).exists { + case _ @ProjectExecTransformer(_, _: ProjectExecTransformer) => true + case _ => false + } == !collapsed + ) + if (collapsed) { + val projectPlan = getExecutedPlan(df).collect { + case plan: ProjectExecTransformer => plan + }.head + assert(projectPlan.getTagValue(SparkPlan.LOGICAL_PLAN_TAG).isDefined) + } + } + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala new file mode 100644 index 000000000000..6dc265bc4fd0 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.extension + +import org.apache.gluten.config.GlutenConfig + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsTrait + +class GlutenCustomerExtensionSuite extends GlutenSQLTestsTrait { + + override def sparkConf: SparkConf = { + super.sparkConf + .set("spark.sql.adaptive.enabled", "false") + .set( + GlutenConfig.EXTENDED_COLUMNAR_TRANSFORM_RULES.key, + "org.apache.spark.sql" + + ".extension.CustomerColumnarPreRules") + .set(GlutenConfig.EXTENDED_COLUMNAR_POST_RULES.key, "") + } + + testGluten("test customer column rules") { + withSQLConf((GlutenConfig.GLUTEN_ENABLED.key, "false")) { + sql("create table my_parquet(id int) using parquet") + sql("insert into my_parquet values (1)") + sql("insert into my_parquet values (2)") + } + withSQLConf((GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key, "false")) { + val df = sql("select * from my_parquet") + val testFileSourceScanExecTransformer = df.queryExecution.executedPlan.collect { + case f: TestFileSourceScanExecTransformer => f + } + assert(testFileSourceScanExecTransformer.nonEmpty) + assert(testFileSourceScanExecTransformer.head.nodeNamePrefix.equals("TestFile")) + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala new file mode 100644 index 000000000000..4924ee4c4f56 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.extension + +import org.apache.gluten.extension.injector.InjectorControl +import org.apache.gluten.utils.BackendTestUtils + +import org.apache.spark.SparkConf +import org.apache.spark.sql._ +import org.apache.spark.sql.internal.StaticSQLConf.SPARK_SESSION_EXTENSIONS + +class GlutenSessionExtensionSuite extends GlutenSQLTestsTrait { + + override def sparkConf: SparkConf = { + super.sparkConf + .set(SPARK_SESSION_EXTENSIONS.key, classOf[MyExtensions].getCanonicalName) + } + + testGluten("test gluten extensions") { + assert( + spark.sessionState.columnarRules + .exists(_.isInstanceOf[InjectorControl.DisablerAware])) + + assert(spark.sessionState.planner.strategies.contains(MySparkStrategy(spark))) + assert(spark.sessionState.analyzer.extendedResolutionRules.contains(MyRule(spark))) + assert(spark.sessionState.analyzer.postHocResolutionRules.contains(MyRule(spark))) + assert(spark.sessionState.analyzer.extendedCheckRules.contains(MyCheckRule(spark))) + assert(spark.sessionState.optimizer.batches.flatMap(_.rules).contains(MyRule(spark))) + if (BackendTestUtils.isCHBackendLoaded()) { + assert(spark.sessionState.sqlParser.isInstanceOf[InjectorControl.DisablerAware]) + } else { + assert(spark.sessionState.sqlParser.isInstanceOf[MyParser]) + } + assert( + spark.sessionState.functionRegistry + .lookupFunction(MyExtensions.myFunction._1) + .isDefined) + assert( + spark.sessionState.columnarRules.contains( + MyColumnarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule()))) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala new file mode 100644 index 000000000000..18c5709bf51a --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.extension + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.{BasicScanExecTransformer, FileSourceScanExecTransformerBase} +import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat + +import org.apache.spark.Partition +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.connector.read.streaming.SparkDataStream +import org.apache.spark.sql.execution.datasources.HadoopFsRelation +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.collection.BitSet + +/** Test for customer column rules */ +case class TestFileSourceScanExecTransformer( + @transient override val relation: HadoopFsRelation, + @transient val stream: Option[SparkDataStream], + override val output: Seq[Attribute], + override val requiredSchema: StructType, + override val partitionFilters: Seq[Expression], + override val optionalBucketSet: Option[BitSet], + override val optionalNumCoalescedBuckets: Option[Int], + override val dataFilters: Seq[Expression], + override val tableIdentifier: Option[TableIdentifier], + override val disableBucketedScan: Boolean = false, + override val pushDownFilters: Option[Seq[Expression]] = None) + extends FileSourceScanExecTransformerBase( + relation, + stream, + output, + requiredSchema, + partitionFilters, + optionalBucketSet, + optionalNumCoalescedBuckets, + dataFilters, + tableIdentifier, + disableBucketedScan) { + + override def getPartitions: Seq[Partition] = + BackendsApiManager.getTransformerApiInstance.genPartitionSeq( + relation, + requiredSchema, + getPartitionArray, + output, + bucketedScan, + optionalBucketSet, + optionalNumCoalescedBuckets, + disableBucketedScan) + + override def getPartitionWithReadFileFormats: Seq[(Partition, ReadFileFormat)] = + getPartitions.map((_, fileFormat)) + + override val nodeNamePrefix: String = "TestFile" + + override def withNewPushdownFilters(filters: Seq[Expression]): BasicScanExecTransformer = + copy(pushDownFilters = Some(filters)) +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackStrategiesSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackStrategiesSuite.scala new file mode 100644 index 000000000000..50beded8defd --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackStrategiesSuite.scala @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.gluten + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.{BasicScanExecTransformer, GlutenPlan} +import org.apache.gluten.extension.GlutenSessionExtensions +import org.apache.gluten.extension.caller.CallerInfo +import org.apache.gluten.extension.columnar.{FallbackTags, RemoveFallbackTagRule} +import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall +import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow +import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} +import org.apache.gluten.extension.columnar.transition.{Convention, InsertBackendTransitions} + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{GlutenSQLTestsTrait, SparkSession} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution._ + +class GlutenFallbackStrategiesSuite extends GlutenSQLTestsTrait { + import GlutenFallbackStrategiesSuite._ + + testGluten("Fall back the whole query if one unsupported") { + withSQLConf((GlutenConfig.COLUMNAR_QUERY_FALLBACK_THRESHOLD.key, "1")) { + val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp())))) + val rule = newRuleApplier( + spark, + List( + _ => + _ => { + UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) + }, + c => InsertBackendTransitions(c.outputsColumnar))) + val outputPlan = rule.apply(originalPlan, false) + // Expect to fall back the entire plan. + assert(outputPlan == originalPlan) + } + } + + testGluten("Fall back the whole plan if meeting the configured threshold") { + withSQLConf((GlutenConfig.COLUMNAR_WHOLESTAGE_FALLBACK_THRESHOLD.key, "1")) { + CallerInfo.withLocalValue(isAqe = true, isCache = false) { + val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp())))) + val rule = newRuleApplier( + spark, + List( + _ => + _ => { + UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) + }, + c => InsertBackendTransitions(c.outputsColumnar))) + val outputPlan = rule.apply(originalPlan, false) + // Expect to fall back the entire plan. + assert(outputPlan == originalPlan) + } + } + } + + testGluten("Don't fall back the whole plan if NOT meeting the configured threshold") { + withSQLConf((GlutenConfig.COLUMNAR_WHOLESTAGE_FALLBACK_THRESHOLD.key, "4")) { + CallerInfo.withLocalValue(isAqe = true, isCache = false) { + val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp())))) + val rule = newRuleApplier( + spark, + List( + _ => + _ => { + UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) + }, + c => InsertBackendTransitions(c.outputsColumnar))) + val outputPlan = rule.apply(originalPlan, false) + // Expect to get the plan with columnar rule applied. + assert(outputPlan != originalPlan) + } + } + } + + testGluten( + "Fall back the whole plan if meeting the configured threshold (leaf node is" + + " transformable)") { + withSQLConf((GlutenConfig.COLUMNAR_WHOLESTAGE_FALLBACK_THRESHOLD.key, "2")) { + CallerInfo.withLocalValue(isAqe = true, isCache = false) { + val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp())))) + val rule = newRuleApplier( + spark, + List( + _ => + _ => { + UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) + }, + c => InsertBackendTransitions(c.outputsColumnar))) + val outputPlan = rule.apply(originalPlan, false) + // Expect to fall back the entire plan. + assert(outputPlan == originalPlan) + } + } + } + + testGluten( + "Don't Fall back the whole plan if NOT meeting the configured threshold (" + + "leaf node is transformable)") { + withSQLConf((GlutenConfig.COLUMNAR_WHOLESTAGE_FALLBACK_THRESHOLD.key, "3")) { + CallerInfo.withLocalValue(isAqe = true, isCache = false) { + val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp())))) + val rule = newRuleApplier( + spark, + List( + _ => + _ => { + UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) + }, + c => InsertBackendTransitions(c.outputsColumnar))) + val outputPlan = rule.apply(originalPlan, false) + // Expect to get the plan with columnar rule applied. + assert(outputPlan != originalPlan) + } + } + } + + testGluten("Tag not transformable more than once") { + val originalPlan = UnaryOp1(LeafOp(supportsColumnar = true)) + FallbackTags.add(originalPlan, "fake reason") + val rule = FallbackEmptySchemaRelation() + val newPlan = rule.apply(originalPlan) + val reason = FallbackTags.get(newPlan).reason() + assert( + reason.contains("fake reason") && + reason.contains("at least one of its children has empty output")) + } + + testGluten("test enabling/disabling Gluten at thread level") { + spark.sql("create table fallback_by_thread_config (a int) using parquet") + spark.sql("insert overwrite fallback_by_thread_config select id as a from range(3)") + val sql = + """ + |select * + |from fallback_by_thread_config as t0 + |""".stripMargin + + val noFallbackPlan = spark.sql(sql).queryExecution.executedPlan + val noFallbackScanExec = noFallbackPlan.collect { case _: BasicScanExecTransformer => true } + assert(noFallbackScanExec.size == 1) + + val thread = new Thread( + () => { + spark.sparkContext + .setLocalProperty(GlutenSessionExtensions.GLUTEN_ENABLE_FOR_THREAD_KEY, "false") + val fallbackPlan = spark.sql(sql).queryExecution.executedPlan + val fallbackScanExec = fallbackPlan.collect { + case e: FileSourceScanExec if !e.isInstanceOf[BasicScanExecTransformer] => true + } + assert(fallbackScanExec.size == 1) + + spark.sparkContext + .setLocalProperty(GlutenSessionExtensions.GLUTEN_ENABLE_FOR_THREAD_KEY, null) + val noFallbackPlan = spark.sql(sql).queryExecution.executedPlan + val noFallbackScanExec = noFallbackPlan.collect { case _: BasicScanExecTransformer => true } + assert(noFallbackScanExec.size == 1) + }) + thread.start() + thread.join(10000) + } +} + +private object GlutenFallbackStrategiesSuite { + def newRuleApplier( + spark: SparkSession, + transformBuilders: Seq[ColumnarRuleCall => Rule[SparkPlan]]): HeuristicApplier = { + new HeuristicApplier( + spark, + transformBuilders, + List(c => p => ExpandFallbackPolicy(c.caller.isAqe(), p)), + List( + c => RemoveTopmostColumnarToRow(c.session, c.caller.isAqe()), + _ => ColumnarCollapseTransformStages(GlutenConfig.get) + ), + List(_ => RemoveFallbackTagRule()), + Nil + ) + } + + // TODO: Generalize the code among shim versions. + case class FallbackEmptySchemaRelation() extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = plan.transformDown { + case p => + if (p.children.exists(_.output.isEmpty)) { + // Some backends are not eligible to offload plan with zero-column input. + // If any child have empty output, mark the plan and that child as UNSUPPORTED. + FallbackTags.add(p, "at least one of its children has empty output") + p.children.foreach { + child => + if (child.output.isEmpty) { + FallbackTags.add(child, "at least one of its children has empty output") + } + } + } + p + } + } + + case class LeafOp(override val supportsColumnar: Boolean = false) extends LeafExecNode { + override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() + override def output: Seq[Attribute] = Seq.empty + } + + case class UnaryOp1(child: SparkPlan, override val supportsColumnar: Boolean = false) + extends UnaryExecNode { + override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() + override def output: Seq[Attribute] = child.output + override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1 = + copy(child = newChild) + } + + case class UnaryOp2(child: SparkPlan, override val supportsColumnar: Boolean = false) + extends UnaryExecNode { + override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() + override def output: Seq[Attribute] = child.output + override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp2 = + copy(child = newChild) + } + +// For replacing LeafOp. + case class LeafOpTransformer() extends LeafExecNode with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None + override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() + override def output: Seq[Attribute] = Seq.empty + } + +// For replacing UnaryOp1. + case class UnaryOp1Transformer(override val child: SparkPlan) + extends UnaryExecNode + with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None + override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() + override def output: Seq[Attribute] = child.output + override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer = + copy(child = newChild) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala new file mode 100644 index 000000000000..8edcef1c08c8 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala @@ -0,0 +1,235 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.gluten + +import org.apache.gluten.GlutenBuildInfo +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.events.GlutenPlanFallbackEvent +import org.apache.gluten.execution.FileSourceScanExecTransformer +import org.apache.gluten.utils.BackendTestUtils + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config.UI.UI_ENABLED +import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} +import org.apache.spark.sql.{GlutenSQLTestsTrait, Row} +import org.apache.spark.sql.execution.ProjectExec +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.ui.{GlutenSQLAppStatusStore, SparkListenerSQLExecutionStart} +import org.apache.spark.status.ElementTrackingStore + +import scala.collection.mutable.ArrayBuffer + +class GlutenFallbackSuite extends GlutenSQLTestsTrait with AdaptiveSparkPlanHelper { + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.RAS_ENABLED.key, "false") + .set(GlutenConfig.GLUTEN_UI_ENABLED.key, "true") + // The gluten ui event test suite expects the spark ui to be enable + .set(UI_ENABLED, true) + } + + testGluten("test fallback logging") { + val testAppender = new LogAppender("fallback reason") + withLogAppender(testAppender) { + withSQLConf( + GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key -> "false", + GlutenConfig.VALIDATION_LOG_LEVEL.key -> "error") { + withTable("t") { + spark.range(10).write.format("parquet").saveAsTable("t") + sql("SELECT * FROM t").collect() + } + } + val msgRegex = + """Validation failed for plan: Scan parquet spark_catalog.default\.t\[QueryId=[0-9]+\],""" + + """ due to: \[FallbackByUserOptions\] Validation failed on node Scan parquet""" + + """ spark_catalog\.default\.t""".stripMargin + assert(testAppender.loggingEvents.exists(_.getMessage.getFormattedMessage.matches(msgRegex))) + } + } + + testGluten("test fallback event") { + val kvStore = spark.sparkContext.statusStore.store.asInstanceOf[ElementTrackingStore] + val glutenStore = new GlutenSQLAppStatusStore(kvStore) + assert( + glutenStore + .buildInfo() + .info + .find(_._1 == "Gluten Version") + .exists(_._2 == GlutenBuildInfo.VERSION)) + + def runExecution(sqlString: String): Long = { + var id = 0L + val listener = new SparkListener { + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case e: SparkListenerSQLExecutionStart => id = e.executionId + case _ => + } + } + } + spark.sparkContext.addSparkListener(listener) + try { + sql(sqlString).collect() + spark.sparkContext.listenerBus.waitUntilEmpty() + } finally { + spark.sparkContext.removeSparkListener(listener) + } + id + } + + withTable("t") { + spark.range(10).write.format("parquet").saveAsTable("t") + val id = runExecution("SELECT * FROM t") + val execution = glutenStore.execution(id) + assert(execution.isDefined) + assert(execution.get.numGlutenNodes == 1) + assert(execution.get.numFallbackNodes == 0) + assert(execution.get.fallbackNodeToReason.isEmpty) + + withSQLConf(GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key -> "false") { + val id = runExecution("SELECT * FROM t") + val execution = glutenStore.execution(id) + assert(execution.isDefined) + assert(execution.get.numGlutenNodes == 0) + assert(execution.get.numFallbackNodes == 1) + val fallbackReason = execution.get.fallbackNodeToReason.head + assert(fallbackReason._1.contains("Scan parquet spark_catalog.default.t")) + assert(fallbackReason._2.contains( + "[FallbackByUserOptions] Validation failed on node Scan parquet spark_catalog.default.t")) + } + } + + withTable("t1", "t2") { + spark.range(10).write.format("parquet").saveAsTable("t1") + spark.range(10).write.format("parquet").saveAsTable("t2") + + val id = runExecution("SELECT * FROM t1 FULL OUTER JOIN t2") + val execution = glutenStore.execution(id) + if (BackendTestUtils.isVeloxBackendLoaded()) { + assert(execution.get.numFallbackNodes == 1) + assert( + execution.get.fallbackNodeToReason.head._2 + .contains("FullOuter join is not supported with BroadcastNestedLoopJoin")) + } else { + assert(execution.get.numFallbackNodes == 2) + } + } + } + + testGluten("Improve merge fallback reason") { + spark.sql("create table t using parquet as select 1 as c1, timestamp '2023-01-01' as c2") + withTable("t") { + val events = new ArrayBuffer[GlutenPlanFallbackEvent] + val listener = new SparkListener { + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case e: GlutenPlanFallbackEvent => events.append(e) + case _ => + } + } + } + spark.sparkContext.addSparkListener(listener) + withSQLConf(GlutenConfig.COLUMNAR_WHOLESTAGE_FALLBACK_THRESHOLD.key -> "1") { + try { + val df = + spark.sql("select c1, count(*) from t where c2 > timestamp '2022-01-01' group by c1") + checkAnswer(df, Row(1, 1)) + spark.sparkContext.listenerBus.waitUntilEmpty() + + // avoid failing when we support transform timestamp filter in future + val isFallback = find(df.queryExecution.executedPlan) { + _.isInstanceOf[FileSourceScanExecTransformer] + }.isEmpty + if (isFallback) { + events.exists( + _.fallbackNodeToReason.values.exists( + _.contains("Subfield filters creation not supported for input type 'TIMESTAMP'"))) + events.exists( + _.fallbackNodeToReason.values.exists( + _.contains("Timestamp is not fully supported in Filter"))) + } + } finally { + spark.sparkContext.removeSparkListener(listener) + } + } + } + } + + test("Add logical link to rewritten spark plan") { + val events = new ArrayBuffer[GlutenPlanFallbackEvent] + val listener = new SparkListener { + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case e: GlutenPlanFallbackEvent => events.append(e) + case _ => + } + } + } + spark.sparkContext.addSparkListener(listener) + withSQLConf(GlutenConfig.EXPRESSION_BLACK_LIST.key -> "add") { + try { + val df = spark.sql("select sum(id + 1) from range(10)") + df.collect() + spark.sparkContext.listenerBus.waitUntilEmpty() + val project = find(df.queryExecution.executedPlan) { + _.isInstanceOf[ProjectExec] + } + assert(project.isDefined) + assert( + events.exists(_.fallbackNodeToReason.values.toSet + .exists(_.contains("Not supported to map spark function name")))) + } finally { + spark.sparkContext.removeSparkListener(listener) + } + } + } + + test("ExpandFallbackPolicy should propagate fallback reason to vanilla SparkPlan") { + val events = new ArrayBuffer[GlutenPlanFallbackEvent] + val listener = new SparkListener { + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case e: GlutenPlanFallbackEvent => events.append(e) + case _ => + } + } + } + spark.sparkContext.addSparkListener(listener) + spark.range(10).selectExpr("id as c1", "id as c2").write.format("parquet").saveAsTable("t") + withTable("t") { + withSQLConf( + GlutenConfig.EXPRESSION_BLACK_LIST.key -> "max", + GlutenConfig.COLUMNAR_WHOLESTAGE_FALLBACK_THRESHOLD.key -> "1") { + try { + val df = spark.sql("select c2, max(c1) as id from t group by c2") + df.collect() + spark.sparkContext.listenerBus.waitUntilEmpty() + val agg = collect(df.queryExecution.executedPlan) { case a: HashAggregateExec => a } + assert(agg.size == 2) + assert( + events.count( + _.fallbackNodeToReason.values.toSet.exists(_.contains( + "Could not find a valid substrait mapping name for max" + ))) == 2) + } finally { + spark.sparkContext.removeSparkListener(listener) + } + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala new file mode 100644 index 000000000000..264a7e7836fd --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQueryCHSuite.scala @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.hive.execution + +import org.apache.gluten.config.GlutenConfig + +import org.apache.spark.{DebugFilesystem, SparkConf} +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.TableIdentifier + +class GlutenHiveSQLQueryCHSuite extends GlutenHiveSQLQuerySuiteBase { + + override def sparkConf: SparkConf = { + defaultSparkConf + .set("spark.plugins", "org.apache.gluten.GlutenPlugin") + .set(GlutenConfig.NATIVE_VALIDATION_ENABLED.key, "false") + .set(GlutenConfig.NATIVE_WRITER_ENABLED.key, "true") + .set("spark.sql.storeAssignmentPolicy", "legacy") + .set("spark.default.parallelism", "1") + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "1024MB") + .set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName) + } + + testGluten("5182: Fix failed to parse post join filters") { + withSQLConf( + "spark.sql.hive.convertMetastoreParquet" -> "false", + "spark.gluten.sql.complexType.scan.fallback.enabled" -> "false") { + sql("DROP TABLE IF EXISTS test_5182_0;") + sql("DROP TABLE IF EXISTS test_5182_1;") + sql( + "CREATE TABLE test_5182_0 (from_uid STRING, vgift_typeid int, vm_count int, " + + "status bigint, ts bigint, vm_typeid int) " + + "USING hive OPTIONS(fileFormat 'parquet') PARTITIONED BY (`day` STRING);") + sql( + "CREATE TABLE test_5182_1 (typeid int, groupid int, ss_id bigint, " + + "ss_start_time bigint, ss_end_time bigint) " + + "USING hive OPTIONS(fileFormat 'parquet');") + sql( + "INSERT INTO test_5182_0 partition(day='2024-03-31') " + + "VALUES('uid_1', 2, 10, 1, 11111111111, 2);") + sql("INSERT INTO test_5182_1 VALUES(2, 1, 1, 1000000000, 2111111111);") + val df = spark.sql( + "select ee.from_uid as uid,day, vgift_typeid, money from " + + "(select t_a.day, if(cast(substr(t_a.ts,1,10) as bigint) between " + + "t_b.ss_start_time and t_b.ss_end_time, t_b.ss_id, 0) ss_id, " + + "t_a.vgift_typeid, t_a.from_uid, vm_count money from " + + "(select from_uid,day,vgift_typeid,vm_count,ts from test_5182_0 " + + "where day between '2024-03-30' and '2024-03-31' and status=1 and vm_typeid=2) t_a " + + "left join test_5182_1 t_b on t_a.vgift_typeid=t_b.typeid " + + "where t_b.groupid in (1,2)) ee where ss_id=1;") + checkAnswer(df, Seq(Row("uid_1", "2024-03-31", 2, 10))) + } + spark.sessionState.catalog.dropTable( + TableIdentifier("test_5182_0"), + ignoreIfNotExists = true, + purge = false) + spark.sessionState.catalog.dropTable( + TableIdentifier("test_5182_1"), + ignoreIfNotExists = true, + purge = false) + } + + testGluten("5249: Reading csv may throw Unexpected empty column") { + withSQLConf( + "spark.gluten.sql.complexType.scan.fallback.enabled" -> "false" + ) { + sql("DROP TABLE IF EXISTS test_5249;") + sql( + "CREATE TABLE test_5249 (name STRING, uid STRING) " + + "ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' " + + "STORED AS INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' " + + "OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat';") + sql("INSERT INTO test_5249 VALUES('name_1', 'id_1');") + val df = spark.sql( + "SELECT name, uid, count(distinct uid) total_uid_num from test_5249 " + + "group by name, uid with cube;") + checkAnswer( + df, + Seq( + Row("name_1", "id_1", 1), + Row("name_1", null, 1), + Row(null, "id_1", 1), + Row(null, null, 1))) + } + spark.sessionState.catalog.dropTable( + TableIdentifier("test_5249"), + ignoreIfNotExists = true, + purge = false) + } + + testGluten("GLUTEN-7116: Support outer explode") { + sql("create table if not exists test_7116 (id int, name string)") + sql("insert into test_7116 values (1, 'a,b'), (2, null), (null, 'c,d'), (3, '')") + val query = + """ + |select id, col_name + |from test_7116 lateral view outer explode(split(name, ',')) as col_name + |""".stripMargin + val df = sql(query) + checkAnswer( + df, + Seq(Row(1, "a"), Row(1, "b"), Row(2, null), Row(null, "c"), Row(null, "d"), Row(3, ""))) + spark.sessionState.catalog.dropTable( + TableIdentifier("test_7116"), + ignoreIfNotExists = true, + purge = false) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuite.scala new file mode 100644 index 000000000000..f945b38edea1 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuite.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.hive.execution + +import org.apache.spark.SparkConf +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveTableScanExecTransformer} +import org.apache.spark.sql.hive.client.HiveClient + +class GlutenHiveSQLQuerySuite extends GlutenHiveSQLQuerySuiteBase { + + override def sparkConf: SparkConf = { + defaultSparkConf + .set("spark.plugins", "org.apache.gluten.GlutenPlugin") + .set("spark.default.parallelism", "1") + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "1024MB") + } + + testGluten("hive orc scan") { + withSQLConf("spark.sql.hive.convertMetastoreOrc" -> "false") { + sql("DROP TABLE IF EXISTS test_orc") + sql( + "CREATE TABLE test_orc (name STRING, favorite_color STRING)" + + " USING hive OPTIONS(fileFormat 'orc')") + sql("INSERT INTO test_orc VALUES('test_1', 'red')"); + val df = spark.sql("select * from test_orc") + checkAnswer(df, Seq(Row("test_1", "red"))) + checkOperatorMatch[HiveTableScanExecTransformer](df) + } + spark.sessionState.catalog.dropTable( + TableIdentifier("test_orc"), + ignoreIfNotExists = true, + purge = false) + } + + test("GLUTEN-11062: Supports mixed input format for partitioned Hive table") { + val hiveClient: HiveClient = + spark.sharedState.externalCatalog.unwrapped.asInstanceOf[HiveExternalCatalog].client + + withSQLConf("spark.sql.hive.convertMetastoreParquet" -> "false") { + withTempDir { + dir => + val parquetLoc = s"file:///$dir/test_parquet" + val orcLoc = s"file:///$dir/test_orc" + withTable("test_parquet", "test_orc") { + hiveClient.runSqlHive(s"""create table test_parquet(id int) + partitioned by(pid int) + stored as parquet location '$parquetLoc' + """.stripMargin) + hiveClient.runSqlHive("insert into test_parquet partition(pid=1) select 2") + hiveClient.runSqlHive(s"""create table test_orc(id int) + partitioned by(pid int) + stored as orc location '$orcLoc' + """.stripMargin) + hiveClient.runSqlHive("insert into test_orc partition(pid=2) select 2") + hiveClient.runSqlHive( + s"alter table test_parquet add partition (pid=2) location '$orcLoc/pid=2'") + hiveClient.runSqlHive("alter table test_parquet partition(pid=2) SET FILEFORMAT orc") + val df = sql("select pid, id from test_parquet order by pid") + checkAnswer(df, Seq(Row(1, 2), Row(2, 2))) + checkOperatorMatch[HiveTableScanExecTransformer](df) + } + } + } + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuiteBase.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuiteBase.scala new file mode 100644 index 000000000000..3382567ee286 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuiteBase.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.hive.execution + +import org.apache.gluten.execution.TransformSupport + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config +import org.apache.spark.internal.config.UI.UI_ENABLED +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsTrait} +import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode +import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation +import org.apache.spark.sql.classic.SparkSession +import org.apache.spark.sql.hive.HiveUtils +import org.apache.spark.sql.internal.SQLConf + +import scala.reflect.ClassTag + +abstract class GlutenHiveSQLQuerySuiteBase extends GlutenSQLTestsTrait { + private var _spark: SparkSession = null + + override def beforeAll(): Unit = { + prepareWorkDir() + if (_spark == null) { + _spark = SparkSession.builder().config(sparkConf).enableHiveSupport().getOrCreate() + } + + _spark.sparkContext.setLogLevel("warn") + } + + override protected def spark: SparkSession = _spark + + override def afterAll(): Unit = { + try { + super.afterAll() + if (_spark != null) { + try { + _spark.sessionState.catalog.reset() + } finally { + _spark.stop() + _spark = null + } + } + } finally { + SparkSession.clearActiveSession() + SparkSession.clearDefaultSession() + doThreadPostAudit() + } + } + + protected def defaultSparkConf: SparkConf = { + val conf = new SparkConf() + .set("spark.master", "local[1]") + .set("spark.sql.test", "") + .set("spark.sql.testkey", "true") + .set(SQLConf.CODEGEN_FALLBACK.key, "false") + .set(SQLConf.CODEGEN_FACTORY_MODE.key, CodegenObjectFactoryMode.CODEGEN_ONLY.toString) + .set( + HiveUtils.HIVE_METASTORE_BARRIER_PREFIXES.key, + "org.apache.spark.sql.hive.execution.PairSerDe") + // SPARK-8910 + .set(UI_ENABLED, false) + .set(config.UNSAFE_EXCEPTION_ON_MEMORY_LEAK, true) + // Hive changed the default of hive.metastore.disallow.incompatible.col.type.changes + // from false to true. For details, see the JIRA HIVE-12320 and HIVE-17764. + .set("spark.hadoop.hive.metastore.disallow.incompatible.col.type.changes", "false") + // Disable ConvertToLocalRelation for better test coverage. Test cases built on + // LocalRelation will exercise the optimization rules better by disabling it as + // this rule may potentially block testing of other optimization rules such as + // ConstantPropagation etc. + .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) + + conf.set( + "spark.sql.warehouse.dir", + getClass.getResource("/").getPath + "/tests-working-home/spark-warehouse") + val metastore = getClass.getResource("/").getPath + getClass.getCanonicalName + "/metastore_db" + conf.set("javax.jdo.option.ConnectionURL", s"jdbc:derby:;databaseName=$metastore;create=true") + } + + def checkOperatorMatch[T <: TransformSupport](df: DataFrame)(implicit tag: ClassTag[T]): Unit = { + val executedPlan = getExecutedPlan(df) + assert(executedPlan.exists(plan => plan.getClass == tag.runtimeClass)) + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenBucketedReadSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenBucketedReadSuite.scala new file mode 100644 index 000000000000..9a9f06e02c5d --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenBucketedReadSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.sources + +import org.apache.spark.sql._ + +class GlutenBucketedReadWithoutHiveSupportSuite + extends BucketedReadWithoutHiveSupportSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenBucketedWriteSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenBucketedWriteSuite.scala new file mode 100644 index 000000000000..e5dd2de8b8bd --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenBucketedWriteSuite.scala @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.sources +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenBucketedWriteWithoutHiveSupportSuite + extends BucketedWriteWithoutHiveSupportSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenCreateTableAsSelectSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenCreateTableAsSelectSuite.scala new file mode 100644 index 000000000000..7f31d62f74be --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenCreateTableAsSelectSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenCreateTableAsSelectSuite + extends CreateTableAsSelectSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenDDLSourceLoadSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenDDLSourceLoadSuite.scala new file mode 100644 index 000000000000..03775cab3914 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenDDLSourceLoadSuite.scala @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +// please note that the META-INF/services had to be modified for the test directory for this to work +class GlutenDDLSourceLoadSuite extends DDLSourceLoadSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenDisableUnnecessaryBucketedScanSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenDisableUnnecessaryBucketedScanSuite.scala new file mode 100644 index 000000000000..fd77663985bc --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenDisableUnnecessaryBucketedScanSuite.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuite + extends DisableUnnecessaryBucketedScanWithoutHiveSupportSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuiteAE + extends DisableUnnecessaryBucketedScanWithoutHiveSupportSuiteAE + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenExternalCommandRunnerSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenExternalCommandRunnerSuite.scala new file mode 100644 index 000000000000..84ba336099a1 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenExternalCommandRunnerSuite.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenExternalCommandRunnerSuite + extends ExternalCommandRunnerSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenFilteredScanSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenFilteredScanSuite.scala new file mode 100644 index 000000000000..d751f20ae3f6 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenFilteredScanSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.sources + +import org.apache.spark.sql._ + +class GlutenFilteredScanSuite extends FilteredScanSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenFiltersSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenFiltersSuite.scala new file mode 100644 index 000000000000..ad91b92aae20 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenFiltersSuite.scala @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +/** Unit test suites for data source filters. */ +class GlutenFiltersSuite extends FiltersSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala new file mode 100644 index 000000000000..0437e29a7d35 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala @@ -0,0 +1,610 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.sources + +import org.apache.gluten.GlutenColumnarWriteTestSupport +import org.apache.gluten.execution.GlutenPlan +import org.apache.gluten.execution.SortExecTransformer + +import org.apache.spark.SparkConf +import org.apache.spark.executor.OutputMetrics +import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.execution.{CommandResultExec, GlutenImplicits, QueryExecution, SparkPlan} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.command.DataWritingCommandExec +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.util.QueryExecutionListener + +import org.apache.hadoop.fs.{Path, RawLocalFileSystem} + +import java.io.{File, IOException} + +class GlutenInsertSuite + extends InsertSuite + with GlutenSQLTestsBaseTrait + with AdaptiveSparkPlanHelper + with GlutenColumnarWriteTestSupport { + + override def sparkConf: SparkConf = { + super.sparkConf.set("spark.sql.leafNodeDefaultParallelism", "1") + } + + override def beforeAll(): Unit = { + super.beforeAll() + spark.sql(""" + |CREATE TABLE source USING PARQUET AS + |SELECT cast(id as int) as c1, cast(id % 5 as string) c2 FROM range(100) + |""".stripMargin) + + spark.sql("INSERT INTO TABLE source SELECT 0, null") + spark.sql("INSERT INTO TABLE source SELECT 0, ''") + } + + override def afterAll(): Unit = { + spark.sql("DROP TABLE source") + super.afterAll() + } + + private def checkWriteFilesAndGetChild(df: DataFrame): (SparkPlan, SparkPlan) = { + val writeFiles = stripAQEPlan( + df.queryExecution.executedPlan + .asInstanceOf[CommandResultExec] + .commandPhysicalPlan).children.head + val child = checkWriteFilesAndGetChild(writeFiles) + (writeFiles, child) + } + + testGluten("insert partition table") { + withTable("pt", "pt2") { + spark.sql("CREATE TABLE pt (c1 int, c2 string) USING PARQUET PARTITIONED BY (pt string)") + spark.sql("CREATE TABLE pt2 (c1 int, c2 string) USING PARQUET PARTITIONED BY (pt string)") + + var taskMetrics: OutputMetrics = null + val taskListener = new SparkListener { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + taskMetrics = taskEnd.taskMetrics.outputMetrics + } + } + + var sqlMetrics: Map[String, SQLMetric] = null + val queryListener = new QueryExecutionListener { + override def onFailure(f: String, qe: QueryExecution, e: Exception): Unit = {} + override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { + qe.executedPlan match { + case dataWritingCommandExec: DataWritingCommandExec => + sqlMetrics = dataWritingCommandExec.cmd.metrics + case _ => + } + } + } + spark.sparkContext.addSparkListener(taskListener) + spark.listenerManager.register(queryListener) + try { + val df = + spark.sql("INSERT INTO TABLE pt partition(pt='a') SELECT * FROM VALUES(1, 'a'),(2, 'b')") + spark.sparkContext.listenerBus.waitUntilEmpty() + checkWriteFilesAndGetChild(df) + + assert(taskMetrics.bytesWritten > 0) + assert(taskMetrics.recordsWritten == 2) + assert(sqlMetrics("numParts").value == 1) + assert(sqlMetrics("numOutputRows").value == 2) + assert(sqlMetrics("numOutputBytes").value > 0) + assert(sqlMetrics("numFiles").value == 1) + + checkAnswer(spark.sql("SELECT * FROM pt"), Row(1, "a", "a") :: Row(2, "b", "a") :: Nil) + } finally { + spark.sparkContext.removeSparkListener(taskListener) + spark.listenerManager.unregister(queryListener) + } + + // check no fallback nodes + val df2 = spark.sql("INSERT INTO TABLE pt2 SELECT * FROM pt") + checkWriteFilesAndGetChild(df2) + val fallbackSummary = GlutenImplicits + .collectQueryExecutionFallbackSummary(spark, df2.queryExecution) + assert(fallbackSummary.numFallbackNodes == 0) + } + } + + ignoreGluten("Cleanup staging files if job failed") { + // Using a unique table name in this test. Sometimes, the table is not removed for some unknown + // reason, which can cause test failure (location already exists) if other following tests have + // the same table name. + withTable("tbl") { + spark.sql("CREATE TABLE tbl (c1 int, c2 string) USING PARQUET") + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("tbl")) + assert(new File(table.location).list().length == 0) + + intercept[Exception] { + spark.sql( + """ + |INSERT INTO TABLE tbl + |SELECT id, assert_true(SPARK_PARTITION_ID() = 1) FROM range(1, 3, 1, 2) + |""".stripMargin + ) + } + assert(new File(table.location).list().length == 0) + } + } + + private def validateDynamicPartitionWrite( + df: DataFrame, + expectedPartitionNames: Set[String]): Unit = { + val (writeFiles, writeChild) = checkWriteFilesAndGetChild(df) + assert( + writeFiles + .find(_.isInstanceOf[SortExecTransformer]) + .isEmpty) + // all operators should be transformed + assert(writeChild.find(!_.isInstanceOf[GlutenPlan]).isEmpty) + + val parts = spark.sessionState.catalog.listPartitionNames(TableIdentifier("pt")).toSet + assert(parts == expectedPartitionNames) + } + + testGluten("remove v1writes sort and project") { + // Only string type has empty2null expression + withTable("pt") { + spark.sql("CREATE TABLE pt (c1 int) USING PARQUET PARTITIONED BY(p string)") + + val df = spark.sql(s""" + |INSERT OVERWRITE TABLE pt PARTITION(p) + |SELECT c1, c2 as p FROM source + |""".stripMargin) + validateDynamicPartitionWrite( + df, + Set("p=0", "p=1", "p=2", "p=3", "p=4", "p=__HIVE_DEFAULT_PARTITION__")) + + // The partition column should never be empty + checkAnswer( + spark.sql("SELECT * FROM pt"), + spark.sql("SELECT c1, if(c2 = '', null, c2) FROM source")) + } + } + + testGluten("remove v1writes sort") { + // __HIVE_DEFAULT_PARTITION__ for other types are covered by other tests. + Seq( + ("p boolean", "coalesce(cast(c2 as boolean), false)", Set("p=false", "p=true")), + ("p short", "coalesce(cast(c2 as short), 0s)", Set("p=0", "p=1", "p=2", "p=3", "p=4")), + ("p int", "coalesce(cast(c2 as int), 0)", Set("p=0", "p=1", "p=2", "p=3", "p=4")), + ("p long", "coalesce(cast(c2 as long), 0l)", Set("p=0", "p=1", "p=2", "p=3", "p=4")), + ( + "p date", + "if(c2 < 3, date '2023-01-01', date '2024-01-01')", + Set("p=2023-01-01", "p=2024-01-01")), + ( + "p int, p2 string", + "if(cast(c2 as int) < 2, 0, 1), c2", + Set( + "p=0/p2=1", + "p=0/p2=0", + "p=1/p2=__HIVE_DEFAULT_PARTITION__", + "p=1/p2=2", + "p=1/p2=3", + "p=1/p2=4" + )) + ) + .foreach { + case (partitionType, partitionExpr, expectedPartitionNames) => + withTable("pt") { + spark.sql(s"CREATE TABLE pt (c1 int) USING PARQUET PARTITIONED BY($partitionType)") + + val df = spark.sql(s""" + |INSERT OVERWRITE TABLE pt + |SELECT c1, $partitionExpr FROM source + |""".stripMargin) + validateDynamicPartitionWrite(df, expectedPartitionNames) + } + } + } + + testGluten("do not remove non-v1writes sort and project") { + withTable("t") { + spark.sql("CREATE TABLE t (c1 int, c2 string) USING PARQUET") + + val df = spark.sql("INSERT OVERWRITE TABLE t SELECT c1, c2 FROM source SORT BY c1") + val (writeFiles, _) = checkWriteFilesAndGetChild(df) + assert(writeFiles.find(x => x.isInstanceOf[SortExecTransformer]).isDefined) + checkAnswer(spark.sql("SELECT * FROM t"), spark.sql("SELECT * FROM source SORT BY c1")) + } + } + + testGluten("SPARK-35106: Throw exception when rename custom partition paths returns false") { + withSQLConf( + "fs.file.impl" -> classOf[ + GlutenRenameFromSparkStagingToFinalDirAlwaysTurnsFalseFilesystem].getName, + "fs.file.impl.disable.cache" -> "true") { + withTempPath { + path => + withTable("t") { + sql(""" + |create table t(i int, part1 int, part2 int) using parquet + |partitioned by (part1, part2) + """.stripMargin) + + sql(s"alter table t add partition(part1=1, part2=1) location '${path.getAbsolutePath}'") + + val e = intercept[IOException] { + sql(s"insert into t partition(part1=1, part2=1) select 1") + } + assert(e.getMessage.contains("Failed to rename")) + } + } + } + } + + testGluten("Do not fallback write files if output columns contain Spark internal metadata") { + withTable("t1", "t2") { + spark.sql("CREATE TABLE t1 USING PARQUET AS SELECT id as c1, id % 3 as c2 FROM range(10)") + spark.sql("CREATE TABLE t2 (c1 long, c2 long) USING PARQUET") + val df = spark.sql("INSERT INTO TABLE t2 SELECT c2, count(*) FROM t1 GROUP BY c2") + checkWriteFilesAndGetChild(df) + } + } + + testGluten("Add metadata white list to allow native write files") { + withTable("t1", "t2") { + spark.sql(""" + |CREATE TABLE t1 (c1 long comment 'data column1', c2 long comment 'data column2') + |USING PARQUET + |""".stripMargin) + spark.sql("INSERT INTO TABLE t1 VALUES(1, 1),(2, 2)") + spark.sql("CREATE TABLE t2 (c1 long, c2 long) USING PARQUET") + val df = spark.sql("INSERT INTO TABLE t2 SELECT * FROM t1") + checkWriteFilesAndGetChild(df) + } + } + + testGluten("INSERT rows, ALTER TABLE ADD COLUMNS with DEFAULTs, then SELECT them") { + import testImplicits._ + case class Config(sqlConf: Option[(String, String)], useDataFrames: Boolean = false) + def runTest(dataSource: String, config: Config): Unit = { + def insertIntoT(): Unit = { + sql("insert into t(a, i) values('xyz', 42)") + } + def withTableT(f: => Unit): Unit = { + sql(s"create table t(a string, i int) using $dataSource") + withTable("t")(f) + } + // Positive tests: + // Adding a column with a valid default value into a table containing existing data + // returns null while it works successfully for newly added rows in Velox. + withTableT { + sql("alter table t add column (s string default concat('abc', 'def'))") + insertIntoT() + checkAnswer(spark.table("t"), Row("xyz", 42, "abcdef")) + checkAnswer(sql("select i, s from t"), Row(42, "abcdef")) + // Now alter the column to change the default value. + // This still returns the previous value, not the new value. + sql("alter table t alter column s set default concat('ghi', 'jkl')") + checkAnswer(sql("select i, s from t"), Row(42, "abcdef")) + } + // Adding a column with a default value and then inserting explicit NULL values works. + // Querying data back from the table differentiates between the explicit NULL values and + // default values. + withTableT { + sql("alter table t add column (s string default concat('abc', 'def'))") + insertIntoT() + if (config.useDataFrames) { + Seq((null, null, null)).toDF.write.insertInto("t") + } else { + sql("insert into t values(null, null, null)") + } + + checkAnswer(spark.table("t"), Seq(Row("xyz", 42, "abcdef"), Row(null, null, null))) + checkAnswer(sql("select i, s from t"), Seq(Row(42, "abcdef"), Row(null, null))) + } + // Adding two columns where only the first has a valid default value works successfully. + // Querying data from the altered table returns the default value as well as NULL for the + // second column.+ + withTableT { + sql("alter table t add column (s string default concat('abc', 'def'))") + insertIntoT() + sql("alter table t add column (x string)") + checkAnswer(spark.table("t"), Row("xyz", 42, "abcdef", null)) + checkAnswer(sql("select i, s, x from t"), Row(42, "abcdef", null)) + } + // Test other supported data types. + withTableT { + sql( + "alter table t add columns (" + + "s boolean default true, " + + "t byte default cast(null as byte), " + + "u short default cast(42 as short), " + + "v float default 0, " + + "w double default 0, " + + "x date default cast('2021-01-02' as date), " + + "y timestamp default cast('2021-01-02 01:01:01' as timestamp), " + + "z timestamp_ntz default cast('2021-01-02 01:01:01' as timestamp_ntz), " + + "a1 timestamp_ltz default cast('2021-01-02 01:01:01' as timestamp_ltz), " + + "a2 decimal(5, 2) default 123.45," + + "a3 bigint default 43," + + "a4 smallint default cast(5 as smallint)," + + "a5 tinyint default cast(6 as tinyint))") + insertIntoT() + // Manually inspect the result row values rather than using the 'checkAnswer' helper method + // in order to ensure the values' correctness while avoiding minor type incompatibilities. + val result: Array[Row] = + sql("select s, t, u, v, w, x, y, z, a1, a2, a3, a4, a5 from t").collect() + for (row <- result) { + assert(row.length == 13) + assert(row(0) == true) + assert(row(1) == null) + assert(row(2) == 42) + assert(row(3) == 0.0f) + assert(row(4) == 0.0d) + assert(row(5).toString == "2021-01-02") + assert(row(6).toString == "2021-01-02 01:01:01.0") + assert(row(7).toString.startsWith("2021-01-02")) + assert(row(8).toString == "2021-01-02 01:01:01.0") + assert(row(9).toString == "123.45") + assert(row(10) == 43L) + assert(row(11) == 5) + assert(row(12) == 6) + } + } + } + + // This represents one test configuration over a data source. + case class TestCase(dataSource: String, configs: Seq[Config]) + // Run the test several times using each configuration. + Seq( + TestCase( + dataSource = "csv", + Seq(Config(None), Config(Some(SQLConf.CSV_PARSER_COLUMN_PRUNING.key -> "false")))), + TestCase( + dataSource = "json", + Seq(Config(None), Config(Some(SQLConf.JSON_GENERATOR_IGNORE_NULL_FIELDS.key -> "false")))), + TestCase( + dataSource = "orc", + Seq(Config(None), Config(Some(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false")))), + TestCase( + dataSource = "parquet", + Seq(Config(None), Config(Some(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false")))) + ).foreach { + testCase: TestCase => + testCase.configs.foreach { + config: Config => + // Run the test twice, once using SQL for the INSERT operations + // and again using DataFrames. + for (useDataFrames <- Seq(false, true)) { + config.sqlConf + .map { + kv: (String, String) => + withSQLConf(kv) { + // Run the test with the pair of custom SQLConf values. + runTest(testCase.dataSource, config.copy(useDataFrames = useDataFrames)) + } + } + .getOrElse { + // Run the test with default settings. + runTest(testCase.dataSource, config.copy(useDataFrames = useDataFrames)) + } + } + } + } + } + + // TODO: fix in Spark-4.0 + ignoreGluten("SPARK-39557 INSERT INTO statements with tables with array defaults") { + withSQLConf("spark.gluten.sql.complexType.scan.fallback.enabled" -> "false") { + import testImplicits._ + // Positive tests: array types are supported as default values. + case class Config(dataSource: String, useDataFrames: Boolean = false) + Seq( + Config("parquet"), + Config("parquet", useDataFrames = true), + Config("orc"), + Config("orc", useDataFrames = true)).foreach { + config => + withTable("t") { + sql(s"create table t(i boolean) using ${config.dataSource}") + if (config.useDataFrames) { + Seq(false).toDF.write.insertInto("t") + } else { + sql("insert into t select false") + } + sql("alter table t add column s array default array(1, 2)") + checkAnswer(spark.table("t"), Row(false, null)) + sql("insert into t(i) values (true)") + checkAnswer(spark.table("t"), Seq(Row(false, null), Row(true, Seq(1, 2)))) + } + } + // Negative tests: provided array element types must match their corresponding DEFAULT + // declarations, if applicable. + val incompatibleDefault = + "Failed to execute ALTER TABLE ADD COLUMNS command because the destination " + + "table column `s` has a DEFAULT value" + Seq(Config("parquet"), Config("parquet", useDataFrames = true)).foreach { + config => + withTable("t") { + sql(s"create table t(i boolean) using ${config.dataSource}") + if (config.useDataFrames) { + Seq(false).toDF.write.insertInto("t") + } else { + sql("insert into t select false") + } + assert(intercept[AnalysisException] { + sql("alter table t add column s array default array('abc', 'def')") + }.getMessage.contains(incompatibleDefault)) + } + } + } + } + + // TODO: fix in Spark-4.0 + ignoreGluten("SPARK-39557 INSERT INTO statements with tables with struct defaults") { + withSQLConf("spark.gluten.sql.complexType.scan.fallback.enabled" -> "false") { + + import testImplicits._ + // Positive tests: struct types are supported as default values. + case class Config(dataSource: String, useDataFrames: Boolean = false) + Seq( + Config("parquet"), + Config("parquet", useDataFrames = true), + Config("orc"), + Config("orc", useDataFrames = true)).foreach { + config => + withTable("t") { + sql(s"create table t(i boolean) using ${config.dataSource}") + if (config.useDataFrames) { + Seq(false).toDF.write.insertInto("t") + } else { + sql("insert into t select false") + } + sql( + "alter table t add column s struct default struct(true, 'abc')") + checkAnswer(spark.table("t"), Row(false, null)) + sql("insert into t(i) values (true)") + checkAnswer(spark.table("t"), Seq(Row(false, null), Row(true, Row(true, "abc")))) + } + } + + // Negative tests: provided map element types must match their corresponding DEFAULT + // declarations, if applicable. + val incompatibleDefault = + "Failed to execute ALTER TABLE ADD COLUMNS command because the destination " + + "table column `s` has a DEFAULT value" + Seq(Config("parquet"), Config("parquet", useDataFrames = true)).foreach { + config => + withTable("t") { + sql(s"create table t(i boolean) using ${config.dataSource}") + if (config.useDataFrames) { + Seq(false).toDF.write.insertInto("t") + } else { + sql("insert into t select false") + } + assert(intercept[AnalysisException] { + sql("alter table t add column s struct default struct(42, 56)") + }.getMessage.contains(incompatibleDefault)) + } + } + } + } + + ignoreGluten("SPARK-39557 INSERT INTO statements with tables with map defaults") { + withSQLConf("spark.gluten.sql.complexType.scan.fallback.enabled" -> "false") { + + import testImplicits._ + // Positive tests: map types are supported as default values. + case class Config(dataSource: String, useDataFrames: Boolean = false) + Seq( + Config("parquet"), + Config("parquet", useDataFrames = true), + Config("orc"), + Config("orc", useDataFrames = true)).foreach { + config => + withTable("t") { + sql(s"create table t(i boolean) using ${config.dataSource}") + if (config.useDataFrames) { + Seq(false).toDF.write.insertInto("t") + } else { + sql("insert into t select false") + } + sql("alter table t add column s map default map(true, 'abc')") + checkAnswer(spark.table("t"), Row(false, null)) + sql("insert into t(i) select true") + checkAnswer(spark.table("t"), Seq(Row(false, null), Row(true, Map(true -> "abc")))) + } + withTable("t") { + sql(s""" + create table t( + i int, + s struct< + x array< + struct>, + y array< + map>> + default struct( + array( + struct(1, 2)), + array( + map(false, 'def', true, 'jkl')))) + using ${config.dataSource}""") + sql("insert into t select 1, default") + sql("alter table t alter column s drop default") + if (config.useDataFrames) { + Seq((2, null)).toDF.write.insertInto("t") + } else { + sql("insert into t select 2, default") + } + sql(""" + alter table t alter column s + set default struct( + array( + struct(3, 4)), + array( + map(false, 'mno', true, 'pqr')))""") + sql("insert into t select 3, default") + sql(""" + alter table t + add column t array< + map> + default array( + map(true, 'xyz'))""") + sql("insert into t(i, s) select 4, default") + checkAnswer( + spark.table("t"), + Seq( + Row(1, Row(Seq(Row(1, 2)), Seq(Map(false -> "def", true -> "jkl"))), null), + Row(2, null, null), + Row(3, Row(Seq(Row(3, 4)), Seq(Map(false -> "mno", true -> "pqr"))), null), + Row( + 4, + Row(Seq(Row(3, 4)), Seq(Map(false -> "mno", true -> "pqr"))), + Seq(Map(true -> "xyz"))) + ) + ) + } + } + // Negative tests: provided map element types must match their corresponding DEFAULT + // declarations, if applicable. + val incompatibleDefault = + "Failed to execute ALTER TABLE ADD COLUMNS command because the destination " + + "table column `s` has a DEFAULT value" + Seq(Config("parquet"), Config("parquet", useDataFrames = true)).foreach { + config => + withTable("t") { + sql(s"create table t(i boolean) using ${config.dataSource}") + if (config.useDataFrames) { + Seq(false).toDF.write.insertInto("t") + } else { + sql("insert into t select false") + } + assert(intercept[AnalysisException] { + sql("alter table t add column s map default map(42, 56)") + }.getMessage.contains(incompatibleDefault)) + } + } + } + } +} + +class GlutenRenameFromSparkStagingToFinalDirAlwaysTurnsFalseFilesystem extends RawLocalFileSystem { + override def rename(src: Path, dst: Path): Boolean = { + (!isSparkStagingDir(src) || isSparkStagingDir(dst)) && super.rename(src, dst) + } + + private def isSparkStagingDir(path: Path): Boolean = { + path.toString.contains("_temporary") + } +} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenPartitionedWriteSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenPartitionedWriteSuite.scala new file mode 100644 index 000000000000..26c847ff2323 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenPartitionedWriteSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenPartitionedWriteSuite extends PartitionedWriteSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenPathOptionSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenPathOptionSuite.scala new file mode 100644 index 000000000000..94171f44cecc --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenPathOptionSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenPathOptionSuite extends PathOptionSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenPrunedScanSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenPrunedScanSuite.scala new file mode 100644 index 000000000000..920d4f3af647 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenPrunedScanSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.sources + +import org.apache.spark.sql._ + +class GlutenPrunedScanSuite extends PrunedScanSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenResolvedDataSourceSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenResolvedDataSourceSuite.scala new file mode 100644 index 000000000000..ddd06bb3fd89 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenResolvedDataSourceSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenResolvedDataSourceSuite extends ResolvedDataSourceSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenSaveLoadSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenSaveLoadSuite.scala new file mode 100644 index 000000000000..5ae0204b835d --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenSaveLoadSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenSaveLoadSuite extends SaveLoadSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenTableScanSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenTableScanSuite.scala new file mode 100644 index 000000000000..ebd17781ff2b --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/sources/GlutenTableScanSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.sources + +import org.apache.spark.sql._ + +class GlutenTableScanSuite extends TableScanSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala new file mode 100644 index 000000000000..5bf53e66e642 --- /dev/null +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala @@ -0,0 +1,218 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.statistics + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.GlutenPlan +import org.apache.gluten.utils.BackendTestUtils + +import org.apache.spark.sql.{GlutenTestConstants, QueryTest, SparkSession} +import org.apache.spark.sql.catalyst.optimizer.{ConstantFolding, ConvertToLocalRelation, NullPropagation} +import org.apache.spark.sql.execution.{ProjectExec, SparkPlan} +import org.apache.spark.sql.internal.SQLConf + +import scala.util.control.Breaks.{break, breakable} + +/** + * TODO: There are some false positive & false negative cases for some functions. For such + * situation, we need to use a suitable test sql to do the check. + */ +class SparkFunctionStatistics extends QueryTest { + + var spark: SparkSession = null + + protected def initializeSession(): Unit = { + if (spark == null) { + val sparkBuilder = SparkSession + .builder() + .appName("Gluten-UT") + .master(s"local[2]") + // Avoid static evaluation for literal input by spark catalyst. + .config( + SQLConf.OPTIMIZER_EXCLUDED_RULES.key, + ConvertToLocalRelation.ruleName + + "," + ConstantFolding.ruleName + "," + NullPropagation.ruleName) + .config("spark.driver.memory", "1G") + .config("spark.sql.adaptive.enabled", "true") + .config("spark.sql.shuffle.partitions", "1") + .config("spark.sql.files.maxPartitionBytes", "134217728") + .config("spark.memory.offHeap.enabled", "true") + .config("spark.memory.offHeap.size", "1024MB") + .config("spark.plugins", "org.apache.gluten.GlutenPlugin") + .config("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + // Avoid the code size overflow error in Spark code generation. + .config("spark.sql.codegen.wholeStage", "false") + + spark = if (BackendTestUtils.isCHBackendLoaded()) { + sparkBuilder + .config("spark.io.compression.codec", "LZ4") + .config("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .config(GlutenConfig.NATIVE_VALIDATION_ENABLED.key, "false") + .config("spark.sql.files.openCostInBytes", "134217728") + .config("spark.unsafe.exceptionOnMemoryLeak", "true") + .getOrCreate() + } else { + sparkBuilder + .config("spark.unsafe.exceptionOnMemoryLeak", "true") + .getOrCreate() + } + } + } + + def extractQuery(examples: String): Seq[String] = { + examples + .split("\n") + .map(_.trim) + .filter(!_.isEmpty) + .filter(_.startsWith("> SELECT")) + .map(_.replace("> SELECT", "SELECT")) + } + + test(GlutenTestConstants.GLUTEN_TEST + "Run spark function statistics: ") { + initializeSession + val functionRegistry = spark.sessionState.functionRegistry + val sparkBuiltInFunctions = functionRegistry.listFunction() + // According to expressionsForTimestampNTZSupport in FunctionRegistry.scala, + // these functions are registered only for testing, not available for end users. + // Other functions like current_database is NOT necessarily offloaded to native. + val ignoreFunctions = Seq( + "get_fake_app_name", + "current_catalog", + "current_database", + "spark_partition_id", + "current_user", + "current_timezone") + val supportedFunctions = new java.util.ArrayList[String]() + val unsupportedFunctions = new java.util.ArrayList[String]() + val needInspectFunctions = new java.util.ArrayList[String]() + + for (func <- sparkBuiltInFunctions) { + val exprInfo = functionRegistry.lookupFunction(func).get + if (!ignoreFunctions.contains(exprInfo.getName)) { + val examples = extractQuery(exprInfo.getExamples) + if (examples.isEmpty) { + needInspectFunctions.add(exprInfo.getName) + // scalastyle:off println + println("## Not found examples for " + exprInfo.getName) + // scalastyle:on println + } + var isSupported: Boolean = true + breakable { + for (example <- examples) { + var executedPlan: SparkPlan = null + try { + executedPlan = spark.sql(example).queryExecution.executedPlan + } catch { + case t: Throwable => + needInspectFunctions.add(exprInfo.getName) + // scalastyle:off println + println("-- Need inspect " + exprInfo.getName) + println(exprInfo.getExamples) + // scalastyle:on println + break + } + val hasFallbackProject = executedPlan.find(_.isInstanceOf[ProjectExec]).isDefined + if (hasFallbackProject) { + isSupported = false + break + } + val hasGlutenPlan = executedPlan.find(_.isInstanceOf[GlutenPlan]).isDefined + if (!hasGlutenPlan) { + isSupported = false + break + } + break + } + } + if (isSupported && !needInspectFunctions.contains(exprInfo.getName)) { + supportedFunctions.add(exprInfo.getName) + } else if (!isSupported) { + unsupportedFunctions.add(exprInfo.getName) + } + } + } + // scalastyle:off println + println("Overall functions: " + (sparkBuiltInFunctions.size - ignoreFunctions.size)) + println("Supported functions: " + supportedFunctions.size()) + println("Unsupported functions: " + unsupportedFunctions.size()) + println("Need inspect functions: " + needInspectFunctions.size()) + // scalastyle:on println + // For correction. + val supportedCastAliasFunctions = Seq( + "boolean", + "tinyint", + "smallint", + "int", + "bigint", + "float", + "double", + "decimal", + "date", + "binary", + "string") + for (func <- supportedCastAliasFunctions) { + if (needInspectFunctions.contains(func)) { + needInspectFunctions.remove(func) + supportedFunctions.add(func) + } + } + + // For wrongly recognized unsupported case. + Seq("%", "ceil", "floor", "first", "first_value", "last", "last_value", "hash", "mod").foreach( + name => { + if (unsupportedFunctions.remove(name)) { + supportedFunctions.add(name) + } + }) + // For wrongly recognized supported case. + Seq( + "array_contains", + "map_keys", + "get_json_object", + "element_at", + "map_from_arrays", + "contains", + "startswith", + "endswith", + "map_contains_key", + "map_values", + "try_element_at", + "struct", + "array", + "ilike", + "sec", + "csc" + ).foreach( + name => { + if (supportedFunctions.remove(name)) { + unsupportedFunctions.add(name) + } + }) + // Functions in needInspectFunctions were checked. + unsupportedFunctions.addAll(needInspectFunctions) + // scalastyle:off println + println("---------------") + println("Overall functions: " + (sparkBuiltInFunctions.size - ignoreFunctions.size)) + println("Supported functions corrected: " + supportedFunctions.size()) + println("Unsupported functions corrected: " + unsupportedFunctions.size()) + println("Support list:") + println(supportedFunctions) + println("Not support list:") + println(unsupportedFunctions) + // scalastyle:on println + } +} diff --git a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala index 2ea814df27c9..53280a72cbda 100644 --- a/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala +++ b/tools/gluten-it/common/src/main/scala/org/apache/gluten/integration/Suite.scala @@ -69,6 +69,7 @@ abstract class Suite( sessionSwitcher.addDefaultConf("spark.sql.broadcastTimeout", "1800") sessionSwitcher.addDefaultConf("spark.network.io.preferDirectBufs", "false") sessionSwitcher.addDefaultConf("spark.unsafe.exceptionOnMemoryLeak", s"$errorOnMemLeak") + sessionSwitcher.addDefaultConf("spark.sql.unionOutputPartitioning", "false") if (dataSource() == "delta") { sessionSwitcher.addDefaultConf( diff --git a/tools/gluten-it/pom.xml b/tools/gluten-it/pom.xml index 27be4a1b5b8f..77e4eab9f6b9 100644 --- a/tools/gluten-it/pom.xml +++ b/tools/gluten-it/pom.xml @@ -327,6 +327,16 @@ 4.0.0 + + spark-4.1 + + 4.1.0 + 2.13.17 + 2.13 + delta-spark + 4.0.0 + + celeborn-0.5