diff --git a/R/README.md b/R/README.md index c808ca88f72dc..31174c73526f2 100644 --- a/R/README.md +++ b/R/README.md @@ -20,7 +20,7 @@ export R_HOME=/home/username/R Build Spark with [Maven](https://spark.apache.org/docs/latest/building-spark.html#buildmvn) and include the `-Psparkr` profile to build the R package. For example to use the default Hadoop versions you can run ```bash -build/mvn -DskipTests -Psparkr package +./build/mvn -DskipTests -Psparkr package ``` #### Running sparkR diff --git a/README.md b/README.md index 9759559e6cf6f..29777a5962bc2 100644 --- a/README.md +++ b/README.md @@ -25,7 +25,7 @@ This README file only contains basic setup instructions. Spark is built using [Apache Maven](https://maven.apache.org/). To build Spark and its example programs, run: - build/mvn -DskipTests clean package + ./build/mvn -DskipTests clean package (You do not need to do this if you downloaded a pre-built package.) diff --git a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala index f5beb403555e9..d0337b6e34962 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala @@ -182,19 +182,19 @@ private[spark] class ExecutorMonitor( if (updateExecutors) { val activeShuffleIds = shuffleStages.map(_._2).toSeq var needTimeoutUpdate = false - val activatedExecs = new mutable.ArrayBuffer[String]() + val activatedExecs = new ExecutorIdCollector() executors.asScala.foreach { case (id, exec) => if (!exec.hasActiveShuffle) { exec.updateActiveShuffles(activeShuffleIds) if (exec.hasActiveShuffle) { needTimeoutUpdate = true - activatedExecs += id + activatedExecs.add(id) } } } - logDebug(s"Activated executors ${activatedExecs.mkString(",")} due to shuffle data " + - s"needed by new job ${event.jobId}.") + logDebug(s"Activated executors $activatedExecs due to shuffle data needed by new job" + + s"${event.jobId}.") if (needTimeoutUpdate) { nextTimeout.set(Long.MinValue) @@ -233,18 +233,18 @@ private[spark] class ExecutorMonitor( } } - val deactivatedExecs = new mutable.ArrayBuffer[String]() + val deactivatedExecs = new ExecutorIdCollector() executors.asScala.foreach { case (id, exec) => if (exec.hasActiveShuffle) { exec.updateActiveShuffles(activeShuffles) if (!exec.hasActiveShuffle) { - deactivatedExecs += id + deactivatedExecs.add(id) } } } - logDebug(s"Executors ${deactivatedExecs.mkString(",")} do not have active shuffle data " + - s"after job ${event.jobId} finished.") + logDebug(s"Executors $deactivatedExecs do not have active shuffle data after job " + + s"${event.jobId} finished.") } jobToStageIDs.remove(event.jobId).foreach { stages => @@ -448,7 +448,8 @@ private[spark] class ExecutorMonitor( } else { idleTimeoutMs } - idleStart + timeout + val deadline = idleStart + timeout + if (deadline >= 0) deadline else Long.MaxValue } else { Long.MaxValue } @@ -491,4 +492,22 @@ private[spark] class ExecutorMonitor( private case class ShuffleCleanedEvent(id: Int) extends SparkListenerEvent { override protected[spark] def logEvent: Boolean = false } + + /** Used to collect executor IDs for debug messages (and avoid too long messages). */ + private class ExecutorIdCollector { + private val ids = if (log.isDebugEnabled) new mutable.ArrayBuffer[String]() else null + private var excess = 0 + + def add(id: String): Unit = if (log.isDebugEnabled) { + if (ids.size < 10) { + ids += id + } else { + excess += 1 + } + } + + override def toString(): String = { + ids.mkString(",") + (if (excess > 0) s" (and $excess more)" else "") + } + } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 838fc82d2ee37..54f0f8e226791 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -641,18 +641,22 @@ private[ui] class TaskPagedTable( {accumulatorsInfo(task)} }} {if (hasInput(stage)) { - metricInfo(task) { m => - val bytesRead = Utils.bytesToString(m.inputMetrics.bytesRead) - val records = m.inputMetrics.recordsRead - {bytesRead} / {records} - } + { + metricInfo(task) { m => + val bytesRead = Utils.bytesToString(m.inputMetrics.bytesRead) + val records = m.inputMetrics.recordsRead + Unparsed(s"$bytesRead / $records") + } + } }} {if (hasOutput(stage)) { - metricInfo(task) { m => - val bytesWritten = Utils.bytesToString(m.outputMetrics.bytesWritten) - val records = m.outputMetrics.recordsWritten - {bytesWritten} / {records} - } + { + metricInfo(task) { m => + val bytesWritten = Utils.bytesToString(m.outputMetrics.bytesWritten) + val records = m.outputMetrics.recordsWritten + Unparsed(s"$bytesWritten / $records") + } + } }} {if (hasShuffleRead(stage)) { diff --git a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala index e11ee97469b00..6a25754fcbe5a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala @@ -367,6 +367,26 @@ class ExecutorMonitorSuite extends SparkFunSuite { assert(monitor.timedOutExecutors(idleDeadline).toSet === Set("1", "2")) } + test("SPARK-28455: avoid overflow in timeout calculation") { + conf + .set(DYN_ALLOCATION_SHUFFLE_TIMEOUT, Long.MaxValue) + .set(DYN_ALLOCATION_SHUFFLE_TRACKING, true) + .set(SHUFFLE_SERVICE_ENABLED, false) + monitor = new ExecutorMonitor(conf, client, null, clock) + + // Generate events that will make executor 1 be idle, while still holding shuffle data. + // The executor should not be eligible for removal since the timeout is basically "infinite". + val stage = stageInfo(1, shuffleId = 0) + monitor.onJobStart(SparkListenerJobStart(1, clock.getTimeMillis(), Seq(stage))) + clock.advance(1000L) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onTaskStart(SparkListenerTaskStart(1, 0, taskInfo("1", 1))) + monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", Success, taskInfo("1", 1), null)) + monitor.onJobEnd(SparkListenerJobEnd(1, clock.getTimeMillis(), JobSucceeded)) + + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + } + private def idleDeadline: Long = clock.getTimeMillis() + idleTimeoutMs + 1 private def storageDeadline: Long = clock.getTimeMillis() + storageTimeoutMs + 1 private def shuffleDeadline: Long = clock.getTimeMillis() + shuffleTimeoutMs + 1 diff --git a/docs/README.md b/docs/README.md index 670e9e01130df..da531321aa5da 100644 --- a/docs/README.md +++ b/docs/README.md @@ -84,7 +84,7 @@ $ PRODUCTION=1 jekyll build ## API Docs (Scaladoc, Javadoc, Sphinx, roxygen2, MkDocs) -You can build just the Spark scaladoc and javadoc by running `build/sbt unidoc` from the `$SPARK_HOME` directory. +You can build just the Spark scaladoc and javadoc by running `./build/sbt unidoc` from the `$SPARK_HOME` directory. Similarly, you can build just the PySpark docs by running `make html` from the `$SPARK_HOME/python/docs` directory. Documentation is only generated for classes that are listed as @@ -94,7 +94,7 @@ after [building Spark](https://github.com/apache/spark#building-spark) first. When you run `jekyll build` in the `docs` directory, it will also copy over the scaladoc and javadoc for the various Spark subprojects into the `docs` directory (and then also into the `_site` directory). We use a -jekyll plugin to run `build/sbt unidoc` before building the site so if you haven't run it (recently) it +jekyll plugin to run `./build/sbt unidoc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc and javadoc using [Unidoc](https://github.com/sbt/sbt-unidoc). The jekyll plugin also generates the PySpark docs using [Sphinx](http://sphinx-doc.org/), SparkR docs using [roxygen2](https://cran.r-project.org/web/packages/roxygen2/index.html) and SQL docs diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index d4efb52e0fbba..769eed1e6f6b7 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -121,7 +121,7 @@ $ ./bin/docker-image-tool.sh -r -t my-tag -R ./kubernetes/dockerfiles/spa To launch Spark Pi in cluster mode, ```bash -$ bin/spark-submit \ +$ ./bin/spark-submit \ --master k8s://https://: \ --deploy-mode cluster \ --name spark-pi \ diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 907f414e5dc4c..cf51620a700bc 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -212,7 +212,7 @@ protected (port 7077 by default). By setting the Mesos proxy config property (requires mesos version >= 1.4), `--conf spark.mesos.proxy.baseURL=http://localhost:5050` when launching the dispatcher, the mesos sandbox URI for each driver is added to the mesos dispatcher UI. -If you like to run the `MesosClusterDispatcher` with Marathon, you need to run the `MesosClusterDispatcher` in the foreground (i.e: `bin/spark-class org.apache.spark.deploy.mesos.MesosClusterDispatcher`). Note that the `MesosClusterDispatcher` not yet supports multiple instances for HA. +If you like to run the `MesosClusterDispatcher` with Marathon, you need to run the `MesosClusterDispatcher` in the foreground (i.e: `./bin/spark-class org.apache.spark.deploy.mesos.MesosClusterDispatcher`). Note that the `MesosClusterDispatcher` not yet supports multiple instances for HA. The `MesosClusterDispatcher` also supports writing recovery state into Zookeeper. This will allow the `MesosClusterDispatcher` to be able to recover all submitted and running containers on relaunch. In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spark-env by configuring `spark.deploy.recoveryMode` and related spark.deploy.zookeeper.* configurations. For more information about these configurations please refer to the configurations [doc](configuration.html#deploy). @@ -362,7 +362,7 @@ The External Shuffle Service to use is the Mesos Shuffle Service. It provides sh on top of the Shuffle Service since Mesos doesn't yet support notifying another framework's termination. To launch it, run `$SPARK_HOME/sbin/start-mesos-shuffle-service.sh` on all slave nodes, with `spark.shuffle.service.enabled` set to `true`. -This can also be achieved through Marathon, using a unique host constraint, and the following command: `bin/spark-class org.apache.spark.deploy.mesos.MesosExternalShuffleService`. +This can also be achieved through Marathon, using a unique host constraint, and the following command: `./bin/spark-class org.apache.spark.deploy.mesos.MesosExternalShuffleService`. # Configuration diff --git a/docs/sql-data-sources-jdbc.md b/docs/sql-data-sources-jdbc.md index 3119ec004b2a1..c3502cbdea8e7 100644 --- a/docs/sql-data-sources-jdbc.md +++ b/docs/sql-data-sources-jdbc.md @@ -36,7 +36,7 @@ spark classpath. For example, to connect to postgres from the Spark Shell you wo following command: {% highlight bash %} -bin/spark-shell --driver-class-path postgresql-9.4.1207.jar --jars postgresql-9.4.1207.jar +./bin/spark-shell --driver-class-path postgresql-9.4.1207.jar --jars postgresql-9.4.1207.jar {% endhighlight %} Tables from the remote database can be loaded as a DataFrame or Spark SQL temporary view using diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index f13d298674b2d..e9d99b66353e2 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -151,7 +151,9 @@ license: | - Since Spark 3.0, substitution order of nested WITH clauses is changed and an inner CTE definition takes precedence over an outer. In version 2.4 and earlier, `WITH t AS (SELECT 1), t2 AS (WITH t AS (SELECT 2) SELECT * FROM t) SELECT * FROM t2` returns `1` while in version 3.0 it returns `2`. The previous behaviour can be restored by setting `spark.sql.legacy.ctePrecedence.enabled` to `true`. - - Since Spark 3.0, the `add_months` function adjusts the resulting date to a last day of month only if it is invalid. For example, `select add_months(DATE'2019-01-31', 1)` results `2019-02-28`. In Spark version 2.4 and earlier, the resulting date is adjusted when it is invalid, or the original date is a last day of months. For example, adding a month to `2019-02-28` resultes in `2019-03-31`. + - Since Spark 3.0, the `add_months` function does not adjust the resulting date to a last day of month if the original date is a last day of months. For example, `select add_months(DATE'2019-02-28', 1)` results `2019-03-28`. In Spark version 2.4 and earlier, the resulting date is adjusted when the original date is a last day of months. For example, adding a month to `2019-02-28` results in `2019-03-31`. + + - Since Spark 3.0, 0-argument Java UDF is executed in the executor side identically with other UDFs. In Spark version 2.4 and earlier, 0-argument Java UDF alone was executed in the driver side, and the result was propagated to executors, which might be more performant in some cases but caused inconsistency with a correctness issue in some cases. ## Upgrading from Spark SQL 2.4 to 2.4.1 diff --git a/docs/streaming-kinesis-integration.md b/docs/streaming-kinesis-integration.md index fd6d776045cd7..55acec53302e4 100644 --- a/docs/streaming-kinesis-integration.md +++ b/docs/streaming-kinesis-integration.md @@ -222,17 +222,17 @@ To run the example,
- bin/run-example --packages org.apache.spark:spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} streaming.KinesisWordCountASL [Kinesis app name] [Kinesis stream name] [endpoint URL] + ./bin/run-example --packages org.apache.spark:spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} streaming.KinesisWordCountASL [Kinesis app name] [Kinesis stream name] [endpoint URL]
- bin/run-example --packages org.apache.spark:spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} streaming.JavaKinesisWordCountASL [Kinesis app name] [Kinesis stream name] [endpoint URL] + ./bin/run-example --packages org.apache.spark:spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} streaming.JavaKinesisWordCountASL [Kinesis app name] [Kinesis stream name] [endpoint URL]
- bin/spark-submit --jars external/kinesis-asl/target/scala-*/\ + ./bin/spark-submit --jars external/kinesis-asl/target/scala-*/\ spark-streaming-kinesis-asl-assembly_*.jar \ external/kinesis-asl/src/main/python/examples/streaming/kinesis_wordcount_asl.py \ [Kinesis app name] [Kinesis stream name] [endpoint URL] [region name] @@ -244,7 +244,7 @@ To run the example, - To generate random string data to put onto the Kinesis stream, in another terminal, run the associated Kinesis data producer. - bin/run-example streaming.KinesisWordProducerASL [Kinesis stream name] [endpoint URL] 1000 10 + ./bin/run-example streaming.KinesisWordProducerASL [Kinesis stream name] [endpoint URL] 1000 10 This will push 1000 lines per second of 10 random numbers per line to the Kinesis stream. This data should then be received and processed by the running example. diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index fe3c60040d0a0..b0009e01703bf 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -388,6 +388,16 @@ The following configurations are optional: streaming and batch Rate limit on maximum number of offsets processed per trigger interval. The specified total number of offsets will be proportionally split across topicPartitions of different volume. + + minPartitions + int + none + streaming and batch + Minimum number of partitions to read from Kafka. + By default, Spark has a 1-1 mapping of topicPartitions to Spark partitions consuming from Kafka. + If you set this option to a value greater than your topicPartitions, Spark will divvy up large + Kafka partitions to smaller pieces. + groupIdPrefix string diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 925e2cfe717c0..821225753320d 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -511,7 +511,7 @@ returned by `SparkSession.readStream()`. In [R](api/R/read.stream.html), with th There are a few built-in sources. - **File source** - Reads files written in a directory as a stream of data. Files will be processed in the order of file modification time. If `latestFirst` is set, order will be reversed. Supported file formats are text, CSV, JSON, ORC, Parquet. See the docs of the DataStreamReader interface for a more up-to-date list, and supported options for each file format. Note that the files must be atomically placed in the given directory, which in most file systems, can be achieved by file move operations. - - **Kafka source** - Reads data from Kafka. It's compatible with Kafka broker versions 0.10.0 or higher. See the [Kafka Integration Guide](structured-streaming-kafka-0-10-integration.html) for more details. + - **Kafka source** - Reads data from Kafka. It's compatible with Kafka broker versions 0.10.0 or higher. See the [Kafka Integration Guide](structured-streaming-kafka-integration.html) for more details. - **Socket source (for testing)** - Reads UTF8 text data from a socket connection. The listening server socket is at the driver. Note that this should be used only for testing as this does not provide end-to-end fault-tolerance guarantees. @@ -582,7 +582,7 @@ Here are the details of all the sources in Spark. Kafka Source - See the Kafka Integration Guide. + See the Kafka Integration Guide. Yes @@ -1835,7 +1835,7 @@ Here are the details of all the sinks in Spark. Kafka Sink Append, Update, Complete - See the Kafka Integration Guide + See the Kafka Integration Guide Yes (at-least-once) More details in the Kafka Integration Guide diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index 924bf374c7370..a7c9e3fb7d329 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -1001,14 +1001,14 @@ abstract class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUti sql("select interval 1 days").write.format("avro").mode("overwrite").save(tempDir) }.getMessage assert(msg.contains("Cannot save interval data type into external storage.") || - msg.contains("AVRO data source does not support calendarinterval data type.")) + msg.contains("AVRO data source does not support interval data type.")) msg = intercept[AnalysisException] { spark.udf.register("testType", () => new IntervalData()) sql("select testType()").write.format("avro").mode("overwrite").save(tempDir) }.getMessage assert(msg.toLowerCase(Locale.ROOT) - .contains(s"avro data source does not support calendarinterval data type.")) + .contains(s"avro data source does not support interval data type.")) } } diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala index 462f88ff14a8d..89da9a1de6f74 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala @@ -206,4 +206,17 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { """.stripMargin.replaceAll("\n", " ")) assert(sql("select c1, c3 from queryOption").collect.toSet == expectedResult) } + + test("write byte as smallint") { + sqlContext.createDataFrame(Seq((1.toByte, 2.toShort))) + .write.jdbc(jdbcUrl, "byte_to_smallint_test", new Properties) + val df = sqlContext.read.jdbc(jdbcUrl, "byte_to_smallint_test", new Properties) + val schema = df.schema + assert(schema.head.dataType == ShortType) + assert(schema(1).dataType == ShortType) + val rows = df.collect() + assert(rows.length === 1) + assert(rows(0).getShort(0) === 1) + assert(rows(0).getShort(1) === 2) + } } diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala index 6e950f968a65d..6e43d60bd03a3 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala @@ -603,6 +603,15 @@ class SparseVector @Since("2.0.0") ( private[spark] override def asBreeze: BV[Double] = new BSV[Double](indices, values, size) + override def apply(i: Int): Double = { + if (i < 0 || i >= size) { + throw new IndexOutOfBoundsException(s"Index $i out of bounds [0, $size)") + } + + val j = util.Arrays.binarySearch(indices, i) + if (j < 0) 0.0 else values(j) + } + override def foreachActive(f: (Int, Double) => Unit): Unit = { var i = 0 val localValuesSize = values.length diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 9cdf1944329b8..b754fad0c1796 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -785,6 +785,15 @@ class SparseVector @Since("1.0.0") ( private[spark] override def asBreeze: BV[Double] = new BSV[Double](indices, values, size) + override def apply(i: Int): Double = { + if (i < 0 || i >= size) { + throw new IndexOutOfBoundsException(s"Index $i out of bounds [0, $size)") + } + + val j = util.Arrays.binarySearch(indices, i) + if (j < 0) 0.0 else values(j) + } + @Since("1.6.0") override def foreachActive(f: (Int, Double) => Unit): Unit = { var i = 0 diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala index a660492c7ae59..03afd29e47505 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.evaluation import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD class MultilabelMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -79,24 +80,24 @@ class MultilabelMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { val hammingLoss = (1.0 / (7 * 3)) * (2 + 2 + 1 + 0 + 0 + 1 + 1) val strictAccuracy = 2.0 / 7 val accuracy = 1.0 / 7 * (1.0 / 3 + 1.0 /3 + 0 + 1.0 / 1 + 2.0 / 2 + 2.0 / 3 + 1.0 / 2) - assert(math.abs(metrics.precision(0.0) - precision0) < delta) - assert(math.abs(metrics.precision(1.0) - precision1) < delta) - assert(math.abs(metrics.precision(2.0) - precision2) < delta) - assert(math.abs(metrics.recall(0.0) - recall0) < delta) - assert(math.abs(metrics.recall(1.0) - recall1) < delta) - assert(math.abs(metrics.recall(2.0) - recall2) < delta) - assert(math.abs(metrics.f1Measure(0.0) - f1measure0) < delta) - assert(math.abs(metrics.f1Measure(1.0) - f1measure1) < delta) - assert(math.abs(metrics.f1Measure(2.0) - f1measure2) < delta) - assert(math.abs(metrics.microPrecision - microPrecisionClass) < delta) - assert(math.abs(metrics.microRecall - microRecallClass) < delta) - assert(math.abs(metrics.microF1Measure - microF1MeasureClass) < delta) - assert(math.abs(metrics.precision - macroPrecisionDoc) < delta) - assert(math.abs(metrics.recall - macroRecallDoc) < delta) - assert(math.abs(metrics.f1Measure - macroF1MeasureDoc) < delta) - assert(math.abs(metrics.hammingLoss - hammingLoss) < delta) - assert(math.abs(metrics.subsetAccuracy - strictAccuracy) < delta) - assert(math.abs(metrics.accuracy - accuracy) < delta) + assert(metrics.precision(0.0) ~== precision0 absTol delta) + assert(metrics.precision(1.0) ~== precision1 absTol delta) + assert(metrics.precision(2.0) ~== precision2 absTol delta) + assert(metrics.recall(0.0) ~== recall0 absTol delta) + assert(metrics.recall(1.0) ~== recall1 absTol delta) + assert(metrics.recall(2.0) ~== recall2 absTol delta) + assert(metrics.f1Measure(0.0) ~== f1measure0 absTol delta) + assert(metrics.f1Measure(1.0) ~== f1measure1 absTol delta) + assert(metrics.f1Measure(2.0) ~== f1measure2 absTol delta) + assert(metrics.microPrecision ~== microPrecisionClass absTol delta) + assert(metrics.microRecall ~== microRecallClass absTol delta) + assert(metrics.microF1Measure ~== microF1MeasureClass absTol delta) + assert(metrics.precision ~== macroPrecisionDoc absTol delta) + assert(metrics.recall ~== macroRecallDoc absTol delta) + assert(metrics.f1Measure ~== macroF1MeasureDoc absTol delta) + assert(metrics.hammingLoss ~== hammingLoss absTol delta) + assert(metrics.subsetAccuracy ~== strictAccuracy absTol delta) + assert(metrics.accuracy ~== accuracy absTol delta) assert(metrics.labels.sameElements(Array(0.0, 1.0, 2.0))) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala index dcb1f398b04b8..26a75699248d0 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.fpm import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ class AssociationRulesSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -63,7 +64,7 @@ class AssociationRulesSuite extends SparkFunSuite with MLlibTestSparkContext { [1] 23 */ assert(results1.size === 23) - assert(results1.count(rule => math.abs(rule.confidence - 1.0D) < 1e-6) == 23) + assert(results1.count(rule => rule.confidence ~= 1.0D absTol 1e-6) == 23) val results2 = ar .setMinConfidence(0) @@ -84,7 +85,7 @@ class AssociationRulesSuite extends SparkFunSuite with MLlibTestSparkContext { [1] 23 */ assert(results2.size === 30) - assert(results2.count(rule => math.abs(rule.confidence - 1.0D) < 1e-6) == 23) + assert(results2.count(rule => rule.confidence ~= 1.0D absTol 1e-6) == 23) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala index 20bd2e5e0dc17..fa8f03be089ce 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.fpm import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.Utils class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -172,7 +173,7 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext { .collect() assert(rules.size === 23) - assert(rules.count(rule => math.abs(rule.confidence - 1.0D) < 1e-6) == 23) + assert(rules.count(rule => rule.confidence ~= 1.0D absTol 1e-6) == 23) } test("FP-Growth using Int type") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala index 566ce95be084a..cca4eb4e4260e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala @@ -22,6 +22,7 @@ import breeze.linalg.{diag => brzDiag, DenseMatrix => BDM, DenseVector => BDV} import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD class IndexedRowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -238,7 +239,7 @@ class IndexedRowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { for (i <- 0 until n; j <- i + 1 until n) { val trueResult = gram(i, j) / scala.math.sqrt(gram(i, i) * gram(j, j)) - assert(math.abs(G(i, j) - trueResult) < 1e-6) + assert(G(i, j) ~== trueResult absTol 1e-6) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala index e30ad159676ff..8011026e6fa65 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala @@ -20,9 +20,9 @@ package org.apache.spark.mllib.random import org.apache.commons.math3.special.Gamma import org.apache.spark.SparkFunSuite +import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.StatCounter -// TODO update tests to use TestingUtils for floating point comparison after PR 1367 is merged class RandomDataGeneratorSuite extends SparkFunSuite { def apiChecks(gen: RandomDataGenerator[Double]) { @@ -61,8 +61,8 @@ class RandomDataGeneratorSuite extends SparkFunSuite { gen.setSeed(seed.toLong) val sample = (0 until 100000).map { _ => gen.nextValue()} val stats = new StatCounter(sample) - assert(math.abs(stats.mean - mean) < epsilon) - assert(math.abs(stats.stdev - stddev) < epsilon) + assert(stats.mean ~== mean absTol epsilon) + assert(stats.stdev ~== stddev absTol epsilon) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala index f464d25c3fbda..9b4dc29d326a1 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala @@ -23,14 +23,13 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.rdd.{RandomRDD, RandomRDDPartition} import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD import org.apache.spark.util.StatCounter /* * Note: avoid including APIs that do not set the seed for the RNG in unit tests * in order to guarantee deterministic behavior. - * - * TODO update tests to use TestingUtils for floating point comparison after PR 1367 is merged */ class RandomRDDsSuite extends SparkFunSuite with MLlibTestSparkContext with Serializable { @@ -43,8 +42,8 @@ class RandomRDDsSuite extends SparkFunSuite with MLlibTestSparkContext with Seri val stats = rdd.stats() assert(expectedSize === stats.count) assert(expectedNumPartitions === rdd.partitions.size) - assert(math.abs(stats.mean - expectedMean) < epsilon) - assert(math.abs(stats.stdev - expectedStddev) < epsilon) + assert(stats.mean ~== expectedMean absTol epsilon) + assert(stats.stdev ~== expectedStddev absTol epsilon) } // assume test RDDs are small @@ -63,8 +62,8 @@ class RandomRDDsSuite extends SparkFunSuite with MLlibTestSparkContext with Seri }} assert(expectedRows === values.size / expectedColumns) val stats = new StatCounter(values) - assert(math.abs(stats.mean - expectedMean) < epsilon) - assert(math.abs(stats.stdev - expectedStddev) < epsilon) + assert(stats.mean ~== expectedMean absTol epsilon) + assert(stats.stdev ~== expectedStddev absTol epsilon) } test("RandomRDD sizes") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala index e32767edb17a8..4613f7fb6f400 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.mllib.random.RandomRDDs import org.apache.spark.mllib.stat.correlation.{Correlations, PearsonCorrelation, SpearmanCorrelation} import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext with Logging { @@ -57,15 +58,15 @@ class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext with Log val expected = 0.6546537 val default = Statistics.corr(x, y) val p1 = Statistics.corr(x, y, "pearson") - assert(approxEqual(expected, default)) - assert(approxEqual(expected, p1)) + assert(expected ~== default absTol 1e-6) + assert(expected ~== p1 absTol 1e-6) // numPartitions >= size for input RDDs for (numParts <- List(xData.size, xData.size * 2)) { val x1 = sc.parallelize(xData, numParts) val y1 = sc.parallelize(yData, numParts) val p2 = Statistics.corr(x1, y1) - assert(approxEqual(expected, p2)) + assert(expected ~== p2 absTol 1e-6) } // RDD of zero variance @@ -78,14 +79,14 @@ class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext with Log val y = sc.parallelize(yData) val expected = 0.5 val s1 = Statistics.corr(x, y, "spearman") - assert(approxEqual(expected, s1)) + assert(expected ~== s1 absTol 1e-6) // numPartitions >= size for input RDDs for (numParts <- List(xData.size, xData.size * 2)) { val x1 = sc.parallelize(xData, numParts) val y1 = sc.parallelize(yData, numParts) val s2 = Statistics.corr(x1, y1, "spearman") - assert(approxEqual(expected, s2)) + assert(expected ~== s2 absTol 1e-6) } // RDD of zero variance => zero variance in ranks @@ -141,14 +142,14 @@ class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext with Log val a = RandomRDDs.normalRDD(sc, 100000, 10).map(_ + 1000000000.0) val b = RandomRDDs.normalRDD(sc, 100000, 10).map(_ + 1000000000.0) val p = Statistics.corr(a, b, method = "pearson") - assert(approxEqual(p, 0.0, 0.01)) + assert(p ~== 0.0 absTol 0.01) } def approxEqual(v1: Double, v2: Double, threshold: Double = 1e-6): Boolean = { if (v1.isNaN) { v2.isNaN } else { - math.abs(v1 - v2) <= threshold + v1 ~== v2 absTol threshold } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala index 5feccdf33681a..9cbb3d0024daa 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala @@ -21,6 +21,7 @@ import org.apache.commons.math3.distribution.NormalDistribution import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ class KernelDensitySuite extends SparkFunSuite with MLlibTestSparkContext { test("kernel density single sample") { @@ -29,8 +30,8 @@ class KernelDensitySuite extends SparkFunSuite with MLlibTestSparkContext { val densities = new KernelDensity().setSample(rdd).setBandwidth(3.0).estimate(evaluationPoints) val normal = new NormalDistribution(5.0, 3.0) val acceptableErr = 1e-6 - assert(math.abs(densities(0) - normal.density(5.0)) < acceptableErr) - assert(math.abs(densities(1) - normal.density(6.0)) < acceptableErr) + assert(densities(0) ~== normal.density(5.0) absTol acceptableErr) + assert(densities(1) ~== normal.density(6.0) absTol acceptableErr) } test("kernel density multiple samples") { @@ -40,9 +41,9 @@ class KernelDensitySuite extends SparkFunSuite with MLlibTestSparkContext { val normal1 = new NormalDistribution(5.0, 3.0) val normal2 = new NormalDistribution(10.0, 3.0) val acceptableErr = 1e-6 - assert(math.abs( - densities(0) - (normal1.density(5.0) + normal2.density(5.0)) / 2) < acceptableErr) - assert(math.abs( - densities(1) - (normal1.density(6.0) + normal2.density(6.0)) / 2) < acceptableErr) + assert( + densities(0) ~== ((normal1.density(5.0) + normal2.density(5.0)) / 2) absTol acceptableErr) + assert( + densities(1) ~== ((normal1.density(6.0) + normal2.density(6.0)) / 2) absTol acceptableErr) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala index 1cc8f342021a0..d43e62bb65535 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala @@ -22,6 +22,7 @@ import scala.collection.mutable import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel +import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.StatCounter object EnsembleTestHelper { @@ -43,8 +44,8 @@ object EnsembleTestHelper { values ++= row } val stats = new StatCounter(values) - assert(math.abs(stats.mean - expectedMean) < epsilon) - assert(math.abs(stats.stdev - expectedStddev) < epsilon) + assert(stats.mean ~== expectedMean absTol epsilon) + assert(stats.stdev ~== expectedStddev absTol epsilon) } def validateClassifier( diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index c257ace02cfe9..ce6543952bf6d 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -874,13 +874,6 @@ class TreeClassifierParams(object): def __init__(self): super(TreeClassifierParams, self).__init__() - @since("1.6.0") - def setImpurity(self, value): - """ - Sets the value of :py:attr:`impurity`. - """ - return self._set(impurity=value) - @since("1.6.0") def getImpurity(self): """ @@ -1003,6 +996,49 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre def _create_model(self, java_model): return DecisionTreeClassificationModel(java_model) + def setMaxDepth(self, value): + """ + Sets the value of :py:attr:`maxDepth`. + """ + return self._set(maxDepth=value) + + def setMaxBins(self, value): + """ + Sets the value of :py:attr:`maxBins`. + """ + return self._set(maxBins=value) + + def setMinInstancesPerNode(self, value): + """ + Sets the value of :py:attr:`minInstancesPerNode`. + """ + return self._set(minInstancesPerNode=value) + + def setMinInfoGain(self, value): + """ + Sets the value of :py:attr:`minInfoGain`. + """ + return self._set(minInfoGain=value) + + def setMaxMemoryInMB(self, value): + """ + Sets the value of :py:attr:`maxMemoryInMB`. + """ + return self._set(maxMemoryInMB=value) + + def setCacheNodeIds(self, value): + """ + Sets the value of :py:attr:`cacheNodeIds`. + """ + return self._set(cacheNodeIds=value) + + @since("1.4.0") + def setImpurity(self, value): + """ + Sets the value of :py:attr:`impurity`. + """ + return self._set(impurity=value) + @inherit_doc class DecisionTreeClassificationModel(DecisionTreeModel, JavaClassificationModel, JavaMLWritable, @@ -1133,6 +1169,63 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre def _create_model(self, java_model): return RandomForestClassificationModel(java_model) + def setMaxDepth(self, value): + """ + Sets the value of :py:attr:`maxDepth`. + """ + return self._set(maxDepth=value) + + def setMaxBins(self, value): + """ + Sets the value of :py:attr:`maxBins`. + """ + return self._set(maxBins=value) + + def setMinInstancesPerNode(self, value): + """ + Sets the value of :py:attr:`minInstancesPerNode`. + """ + return self._set(minInstancesPerNode=value) + + def setMinInfoGain(self, value): + """ + Sets the value of :py:attr:`minInfoGain`. + """ + return self._set(minInfoGain=value) + + def setMaxMemoryInMB(self, value): + """ + Sets the value of :py:attr:`maxMemoryInMB`. + """ + return self._set(maxMemoryInMB=value) + + def setCacheNodeIds(self, value): + """ + Sets the value of :py:attr:`cacheNodeIds`. + """ + return self._set(cacheNodeIds=value) + + @since("1.4.0") + def setImpurity(self, value): + """ + Sets the value of :py:attr:`impurity`. + """ + return self._set(impurity=value) + + @since("1.4.0") + def setNumTrees(self, value): + """ + Sets the value of :py:attr:`numTrees`. + """ + return self._set(numTrees=value) + + @since("1.4.0") + def setSubsamplingRate(self, value): + """ + Sets the value of :py:attr:`subsamplingRate`. + """ + return self._set(subsamplingRate=value) + @since("2.4.0") def setFeatureSubsetStrategy(self, value): """ @@ -1317,6 +1410,49 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre def _create_model(self, java_model): return GBTClassificationModel(java_model) + def setMaxDepth(self, value): + """ + Sets the value of :py:attr:`maxDepth`. + """ + return self._set(maxDepth=value) + + def setMaxBins(self, value): + """ + Sets the value of :py:attr:`maxBins`. + """ + return self._set(maxBins=value) + + def setMinInstancesPerNode(self, value): + """ + Sets the value of :py:attr:`minInstancesPerNode`. + """ + return self._set(minInstancesPerNode=value) + + def setMinInfoGain(self, value): + """ + Sets the value of :py:attr:`minInfoGain`. + """ + return self._set(minInfoGain=value) + + def setMaxMemoryInMB(self, value): + """ + Sets the value of :py:attr:`maxMemoryInMB`. + """ + return self._set(maxMemoryInMB=value) + + def setCacheNodeIds(self, value): + """ + Sets the value of :py:attr:`cacheNodeIds`. + """ + return self._set(cacheNodeIds=value) + + @since("1.4.0") + def setImpurity(self, value): + """ + Sets the value of :py:attr:`impurity`. + """ + return self._set(impurity=value) + @since("1.4.0") def setLossType(self, value): """ @@ -1324,6 +1460,13 @@ def setLossType(self, value): """ return self._set(lossType=value) + @since("1.4.0") + def setSubsamplingRate(self, value): + """ + Sets the value of :py:attr:`subsamplingRate`. + """ + return self._set(subsamplingRate=value) + @since("2.4.0") def setFeatureSubsetStrategy(self, value): """ diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 6405b9fce7efb..56d6190723161 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -765,72 +765,36 @@ class DecisionTreeParams(Params): def __init__(self): super(DecisionTreeParams, self).__init__() - def setMaxDepth(self, value): - """ - Sets the value of :py:attr:`maxDepth`. - """ - return self._set(maxDepth=value) - def getMaxDepth(self): """ Gets the value of maxDepth or its default value. """ return self.getOrDefault(self.maxDepth) - def setMaxBins(self, value): - """ - Sets the value of :py:attr:`maxBins`. - """ - return self._set(maxBins=value) - def getMaxBins(self): """ Gets the value of maxBins or its default value. """ return self.getOrDefault(self.maxBins) - def setMinInstancesPerNode(self, value): - """ - Sets the value of :py:attr:`minInstancesPerNode`. - """ - return self._set(minInstancesPerNode=value) - def getMinInstancesPerNode(self): """ Gets the value of minInstancesPerNode or its default value. """ return self.getOrDefault(self.minInstancesPerNode) - def setMinInfoGain(self, value): - """ - Sets the value of :py:attr:`minInfoGain`. - """ - return self._set(minInfoGain=value) - def getMinInfoGain(self): """ Gets the value of minInfoGain or its default value. """ return self.getOrDefault(self.minInfoGain) - def setMaxMemoryInMB(self, value): - """ - Sets the value of :py:attr:`maxMemoryInMB`. - """ - return self._set(maxMemoryInMB=value) - def getMaxMemoryInMB(self): """ Gets the value of maxMemoryInMB or its default value. """ return self.getOrDefault(self.maxMemoryInMB) - def setCacheNodeIds(self, value): - """ - Sets the value of :py:attr:`cacheNodeIds`. - """ - return self._set(cacheNodeIds=value) - def getCacheNodeIds(self): """ Gets the value of cacheNodeIds or its default value. diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 927cc77e201a5..349130f22fade 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -609,13 +609,6 @@ class TreeEnsembleParams(DecisionTreeParams): def __init__(self): super(TreeEnsembleParams, self).__init__() - @since("1.4.0") - def setSubsamplingRate(self, value): - """ - Sets the value of :py:attr:`subsamplingRate`. - """ - return self._set(subsamplingRate=value) - @since("1.4.0") def getSubsamplingRate(self): """ @@ -623,15 +616,6 @@ def getSubsamplingRate(self): """ return self.getOrDefault(self.subsamplingRate) - @since("1.4.0") - def setFeatureSubsetStrategy(self, value): - """ - Sets the value of :py:attr:`featureSubsetStrategy`. - - .. note:: Deprecated in 2.4.0 and will be removed in 3.0.0. - """ - return self._set(featureSubsetStrategy=value) - @since("1.4.0") def getFeatureSubsetStrategy(self): """ @@ -655,13 +639,6 @@ class HasVarianceImpurity(Params): def __init__(self): super(HasVarianceImpurity, self).__init__() - @since("1.4.0") - def setImpurity(self, value): - """ - Sets the value of :py:attr:`impurity`. - """ - return self._set(impurity=value) - @since("1.4.0") def getImpurity(self): """ @@ -685,13 +662,6 @@ class RandomForestParams(TreeEnsembleParams): def __init__(self): super(RandomForestParams, self).__init__() - @since("1.4.0") - def setNumTrees(self, value): - """ - Sets the value of :py:attr:`numTrees`. - """ - return self._set(numTrees=value) - @since("1.4.0") def getNumTrees(self): """ @@ -843,6 +813,49 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre def _create_model(self, java_model): return DecisionTreeRegressionModel(java_model) + def setMaxDepth(self, value): + """ + Sets the value of :py:attr:`maxDepth`. + """ + return self._set(maxDepth=value) + + def setMaxBins(self, value): + """ + Sets the value of :py:attr:`maxBins`. + """ + return self._set(maxBins=value) + + def setMinInstancesPerNode(self, value): + """ + Sets the value of :py:attr:`minInstancesPerNode`. + """ + return self._set(minInstancesPerNode=value) + + def setMinInfoGain(self, value): + """ + Sets the value of :py:attr:`minInfoGain`. + """ + return self._set(minInfoGain=value) + + def setMaxMemoryInMB(self, value): + """ + Sets the value of :py:attr:`maxMemoryInMB`. + """ + return self._set(maxMemoryInMB=value) + + def setCacheNodeIds(self, value): + """ + Sets the value of :py:attr:`cacheNodeIds`. + """ + return self._set(cacheNodeIds=value) + + @since("1.4.0") + def setImpurity(self, value): + """ + Sets the value of :py:attr:`impurity`. + """ + return self._set(impurity=value) + @inherit_doc class DecisionTreeModel(JavaModel, JavaPredictionModel): @@ -1036,6 +1049,63 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre def _create_model(self, java_model): return RandomForestRegressionModel(java_model) + def setMaxDepth(self, value): + """ + Sets the value of :py:attr:`maxDepth`. + """ + return self._set(maxDepth=value) + + def setMaxBins(self, value): + """ + Sets the value of :py:attr:`maxBins`. + """ + return self._set(maxBins=value) + + def setMinInstancesPerNode(self, value): + """ + Sets the value of :py:attr:`minInstancesPerNode`. + """ + return self._set(minInstancesPerNode=value) + + def setMinInfoGain(self, value): + """ + Sets the value of :py:attr:`minInfoGain`. + """ + return self._set(minInfoGain=value) + + def setMaxMemoryInMB(self, value): + """ + Sets the value of :py:attr:`maxMemoryInMB`. + """ + return self._set(maxMemoryInMB=value) + + def setCacheNodeIds(self, value): + """ + Sets the value of :py:attr:`cacheNodeIds`. + """ + return self._set(cacheNodeIds=value) + + @since("1.4.0") + def setImpurity(self, value): + """ + Sets the value of :py:attr:`impurity`. + """ + return self._set(impurity=value) + + @since("1.4.0") + def setNumTrees(self, value): + """ + Sets the value of :py:attr:`numTrees`. + """ + return self._set(numTrees=value) + + @since("1.4.0") + def setSubsamplingRate(self, value): + """ + Sets the value of :py:attr:`subsamplingRate`. + """ + return self._set(subsamplingRate=value) + @since("2.4.0") def setFeatureSubsetStrategy(self, value): """ @@ -1180,6 +1250,49 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre def _create_model(self, java_model): return GBTRegressionModel(java_model) + def setMaxDepth(self, value): + """ + Sets the value of :py:attr:`maxDepth`. + """ + return self._set(maxDepth=value) + + def setMaxBins(self, value): + """ + Sets the value of :py:attr:`maxBins`. + """ + return self._set(maxBins=value) + + def setMinInstancesPerNode(self, value): + """ + Sets the value of :py:attr:`minInstancesPerNode`. + """ + return self._set(minInstancesPerNode=value) + + def setMinInfoGain(self, value): + """ + Sets the value of :py:attr:`minInfoGain`. + """ + return self._set(minInfoGain=value) + + def setMaxMemoryInMB(self, value): + """ + Sets the value of :py:attr:`maxMemoryInMB`. + """ + return self._set(maxMemoryInMB=value) + + def setCacheNodeIds(self, value): + """ + Sets the value of :py:attr:`cacheNodeIds`. + """ + return self._set(cacheNodeIds=value) + + @since("1.4.0") + def setImpurity(self, value): + """ + Sets the value of :py:attr:`impurity`. + """ + return self._set(impurity=value) + @since("1.4.0") def setLossType(self, value): """ @@ -1187,6 +1300,13 @@ def setLossType(self, value): """ return self._set(lossType=value) + @since("1.4.0") + def setSubsamplingRate(self, value): + """ + Sets the value of :py:attr:`subsamplingRate`. + """ + return self._set(subsamplingRate=value) + @since("2.4.0") def setFeatureSubsetStrategy(self, value): """ diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 6bb7da6b2edb2..e531000f3295c 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -36,6 +36,7 @@ from pyspark.sql.types import StringType, DataType # Keep UserDefinedFunction import for backwards compatible import; moved in SPARK-22409 from pyspark.sql.udf import UserDefinedFunction, _create_udf +from pyspark.sql.utils import to_str # Note to developers: all of PySpark functions here take string as column names whenever possible. # Namely, if columns are referred as arguments, they can be always both Column or string, @@ -114,6 +115,10 @@ def _(): _.__doc__ = 'Window function: ' + doc return _ + +def _options_to_str(options): + return {key: to_str(value) for (key, value) in options.items()} + _lit_doc = """ Creates a :class:`Column` of literal value. @@ -2343,7 +2348,7 @@ def from_json(col, schema, options={}): schema = schema.json() elif isinstance(schema, Column): schema = _to_java_column(schema) - jc = sc._jvm.functions.from_json(_to_java_column(col), schema, options) + jc = sc._jvm.functions.from_json(_to_java_column(col), schema, _options_to_str(options)) return Column(jc) @@ -2384,7 +2389,7 @@ def to_json(col, options={}): """ sc = SparkContext._active_spark_context - jc = sc._jvm.functions.to_json(_to_java_column(col), options) + jc = sc._jvm.functions.to_json(_to_java_column(col), _options_to_str(options)) return Column(jc) @@ -2415,7 +2420,7 @@ def schema_of_json(json, options={}): raise TypeError("schema argument should be a column or string") sc = SparkContext._active_spark_context - jc = sc._jvm.functions.schema_of_json(col, options) + jc = sc._jvm.functions.schema_of_json(col, _options_to_str(options)) return Column(jc) @@ -2442,7 +2447,7 @@ def schema_of_csv(csv, options={}): raise TypeError("schema argument should be a column or string") sc = SparkContext._active_spark_context - jc = sc._jvm.functions.schema_of_csv(col, options) + jc = sc._jvm.functions.schema_of_csv(col, _options_to_str(options)) return Column(jc) @@ -2464,7 +2469,7 @@ def to_csv(col, options={}): """ sc = SparkContext._active_spark_context - jc = sc._jvm.functions.to_csv(_to_java_column(col), options) + jc = sc._jvm.functions.to_csv(_to_java_column(col), _options_to_str(options)) return Column(jc) @@ -2693,7 +2698,10 @@ def array_repeat(col, count): [Row(r=[u'ab', u'ab', u'ab'])] """ sc = SparkContext._active_spark_context - return Column(sc._jvm.functions.array_repeat(_to_java_column(col), count)) + return Column(sc._jvm.functions.array_repeat( + _to_java_column(col), + _to_java_column(count) if isinstance(count, Column) else count + )) @since(2.4) @@ -2775,6 +2783,11 @@ def from_csv(col, schema, options={}): >>> value = data[0][0] >>> df.select(from_csv(df.value, schema_of_csv(value)).alias("csv")).collect() [Row(csv=Row(_c0=1, _c1=2, _c2=3))] + >>> data = [(" abc",)] + >>> df = spark.createDataFrame(data, ("value",)) + >>> options = {'ignoreLeadingWhiteSpace': True} + >>> df.select(from_csv(df.value, "s string", options).alias("csv")).collect() + [Row(csv=Row(s=u'abc'))] """ sc = SparkContext._active_spark_context @@ -2785,7 +2798,7 @@ def from_csv(col, schema, options={}): else: raise TypeError("schema argument should be a column or string") - jc = sc._jvm.functions.from_csv(_to_java_column(col), schema, options) + jc = sc._jvm.functions.from_csv(_to_java_column(col), schema, _options_to_str(options)) return Column(jc) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index aa5bf635d1874..f9bc2ff72a505 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -27,23 +27,11 @@ from pyspark.sql.column import _to_seq from pyspark.sql.types import * from pyspark.sql import utils +from pyspark.sql.utils import to_str __all__ = ["DataFrameReader", "DataFrameWriter"] -def to_str(value): - """ - A wrapper over str(), but converts bool values to lower case strings. - If None is given, just returns None, instead of converting it to string "None". - """ - if isinstance(value, bool): - return str(value).lower() - elif value is None: - return value - else: - return str(value) - - class OptionUtils(object): def _set_opts(self, schema=None, **options): @@ -757,7 +745,7 @@ def save(self, path=None, format=None, mode=None, partitionBy=None, **options): self._jwrite.save(path) @since(1.4) - def insertInto(self, tableName, overwrite=False): + def insertInto(self, tableName, overwrite=None): """Inserts the content of the :class:`DataFrame` to the specified table. It requires that the schema of the class:`DataFrame` is the same as the @@ -765,7 +753,9 @@ def insertInto(self, tableName, overwrite=False): Optionally overwriting any existing data. """ - self._jwrite.mode("overwrite" if overwrite else "append").insertInto(tableName) + if overwrite is not None: + self.mode("overwrite" if overwrite else "append") + self._jwrite.insertInto(tableName) @since(1.4) def saveAsTable(self, name, format=None, mode=None, partitionBy=None, **options): diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 7dfc757970091..64f2fd6a3919f 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -294,6 +294,16 @@ def test_input_file_name_reset_for_rdd(self): for result in results: self.assertEqual(result[0], '') + def test_array_repeat(self): + from pyspark.sql.functions import array_repeat, lit + + df = self.spark.range(1) + + self.assertEquals( + df.select(array_repeat("id", 3)).toDF("val").collect(), + df.select(array_repeat("id", lit(3))).toDF("val").collect(), + ) + if __name__ == "__main__": import unittest diff --git a/python/pyspark/sql/tests/test_readwriter.py b/python/pyspark/sql/tests/test_readwriter.py index a708072489601..2530cc2ebf224 100644 --- a/python/pyspark/sql/tests/test_readwriter.py +++ b/python/pyspark/sql/tests/test_readwriter.py @@ -141,6 +141,27 @@ def count_bucketed_cols(names, table="pyspark_bucket"): .mode("overwrite").saveAsTable("pyspark_bucket")) self.assertSetEqual(set(data), set(self.spark.table("pyspark_bucket").collect())) + def test_insert_into(self): + df = self.spark.createDataFrame([("a", 1), ("b", 2)], ["C1", "C2"]) + with self.table("test_table"): + df.write.saveAsTable("test_table") + self.assertEqual(2, self.spark.sql("select * from test_table").count()) + + df.write.insertInto("test_table") + self.assertEqual(4, self.spark.sql("select * from test_table").count()) + + df.write.mode("overwrite").insertInto("test_table") + self.assertEqual(2, self.spark.sql("select * from test_table").count()) + + df.write.insertInto("test_table", True) + self.assertEqual(2, self.spark.sql("select * from test_table").count()) + + df.write.insertInto("test_table", False) + self.assertEqual(4, self.spark.sql("select * from test_table").count()) + + df.write.mode("overwrite").insertInto("test_table", False) + self.assertEqual(6, self.spark.sql("select * from test_table").count()) + if __name__ == "__main__": import unittest diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index ca5e85bb3a9bb..c30cc1482750a 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -207,3 +207,16 @@ def call(self, jdf, batch_id): class Java: implements = ['org.apache.spark.sql.execution.streaming.sources.PythonForeachBatchFunction'] + + +def to_str(value): + """ + A wrapper over str(), but converts bool values to lower case strings. + If None is given, just returns None, instead of converting it to string "None". + """ + if isinstance(value, bool): + return str(value).lower() + elif value is None: + return value + else: + return str(value) diff --git a/resource-managers/kubernetes/integration-tests/README.md b/resource-managers/kubernetes/integration-tests/README.md index ea8286124a68c..d7ad35a175a61 100644 --- a/resource-managers/kubernetes/integration-tests/README.md +++ b/resource-managers/kubernetes/integration-tests/README.md @@ -11,7 +11,7 @@ is subject to change. Note that currently the integration tests only run with Ja The simplest way to run the integration tests is to install and run Minikube, then run the following from this directory: - dev/dev-run-integration-tests.sh + ./dev/dev-run-integration-tests.sh The minimum tested version of Minikube is 0.23.0. The kube-dns addon must be enabled. Minikube should run with a minimum of 4 CPUs and 6G of memory: @@ -62,11 +62,11 @@ By default, the test framework will build new Docker images on every test execut and it is written to file at `target/imageTag.txt`. To reuse the images built in a previous run, or to use a Docker image tag that you have built by other means already, pass the tag to the test script: - dev/dev-run-integration-tests.sh --image-tag + ./dev/dev-run-integration-tests.sh --image-tag where if you still want to use images that were built before by the test framework: - dev/dev-run-integration-tests.sh --image-tag $(cat target/imageTag.txt) + ./dev/dev-run-integration-tests.sh --image-tag $(cat target/imageTag.txt) ### Customising the Image Names @@ -74,11 +74,11 @@ If your image names do not follow the standard Spark naming convention - `spark` If you use the same basic pattern but a different prefix for the name e.g. `apache-spark` you can just set `--base-image-name ` e.g. - dev/dev-run-integration-tests.sh --base-image-name apache-spark + ./dev/dev-run-integration-tests.sh --base-image-name apache-spark Alternatively if you use completely custom names then you can set each individually via the `--jvm-image-name `, `--python-image-name ` and `--r-image-name ` arguments e.g. - dev/dev-run-integration-tests.sh --jvm-image-name jvm-spark --python-image-name pyspark --r-image-name sparkr + ./dev/dev-run-integration-tests.sh --jvm-image-name jvm-spark --python-image-name pyspark --r-image-name sparkr ## Spark Distribution Under Test diff --git a/sql/README.md b/sql/README.md index 70cc7c637b58d..f0ea848a41d09 100644 --- a/sql/README.md +++ b/sql/README.md @@ -9,4 +9,4 @@ Spark SQL is broken up into four subprojects: - Hive Support (sql/hive) - Includes an extension of SQLContext called HiveContext that allows users to write queries using a subset of HiveQL and access data from a Hive Metastore using Hive SerDes. There are also wrappers that allow users to run queries that include Hive UDFs, UDAFs, and UDTFs. - HiveServer and CLI support (sql/hive-thriftserver) - Includes support for the SQL CLI (bin/spark-sql) and a HiveServer2 (for JDBC/ODBC) compatible server. -Running `sql/create-docs.sh` generates SQL documentation for built-in functions under `sql/site`. +Running `./sql/create-docs.sh` generates SQL documentation for built-in functions under `sql/site`. diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index d991e7cf7e898..0a142c29a16f3 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -113,6 +113,14 @@ statement (AS? query)? #createHiveTable | CREATE TABLE (IF NOT EXISTS)? target=tableIdentifier LIKE source=tableIdentifier locationSpec? #createTableLike + | replaceTableHeader ('(' colTypeList ')')? tableProvider + ((OPTIONS options=tablePropertyList) | + (PARTITIONED BY partitioning=transformList) | + bucketSpec | + locationSpec | + (COMMENT comment=STRING) | + (TBLPROPERTIES tableProps=tablePropertyList))* + (AS? query)? #replaceTable | ANALYZE TABLE tableIdentifier partitionSpec? COMPUTE STATISTICS (identifier | FOR COLUMNS identifierSeq | FOR ALL COLUMNS)? #analyze | ALTER TABLE multipartIdentifier @@ -261,6 +269,10 @@ createTableHeader : CREATE TEMPORARY? EXTERNAL? TABLE (IF NOT EXISTS)? multipartIdentifier ; +replaceTableHeader + : (CREATE OR)? REPLACE TABLE multipartIdentifier + ; + bucketSpec : CLUSTERED BY identifierList (SORTED BY orderedIdentifierList)? diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java new file mode 100644 index 0000000000000..fc055e91a6acf --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/StagingTableCatalog.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalog.v2; + +import java.util.Map; + +import org.apache.spark.sql.catalog.v2.expressions.Transform; +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; +import org.apache.spark.sql.sources.v2.StagedTable; +import org.apache.spark.sql.sources.v2.SupportsWrite; +import org.apache.spark.sql.sources.v2.writer.BatchWrite; +import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +/** + * An optional mix-in for implementations of {@link TableCatalog} that support staging creation of + * the a table before committing the table's metadata along with its contents in CREATE TABLE AS + * SELECT or REPLACE TABLE AS SELECT operations. + *

+ * It is highly recommended to implement this trait whenever possible so that CREATE TABLE AS + * SELECT and REPLACE TABLE AS SELECT operations are atomic. For example, when one runs a REPLACE + * TABLE AS SELECT operation, if the catalog does not implement this trait, the planner will first + * drop the table via {@link TableCatalog#dropTable(Identifier)}, then create the table via + * {@link TableCatalog#createTable(Identifier, StructType, Transform[], Map)}, and then perform + * the write via {@link SupportsWrite#newWriteBuilder(CaseInsensitiveStringMap)}. However, if the + * write operation fails, the catalog will have already dropped the table, and the planner cannot + * roll back the dropping of the table. + *

+ * If the catalog implements this plugin, the catalog can implement the methods to "stage" the + * creation and the replacement of a table. After the table's + * {@link BatchWrite#commit(WriterCommitMessage[])} is called, + * {@link StagedTable#commitStagedChanges()} is called, at which point the staged table can + * complete both the data write and the metadata swap operation atomically. + */ +public interface StagingTableCatalog extends TableCatalog { + + /** + * Stage the creation of a table, preparing it to be committed into the metastore. + *

+ * When the table is committed, the contents of any writes performed by the Spark planner are + * committed along with the metadata about the table passed into this method's arguments. If the + * table exists when this method is called, the method should throw an exception accordingly. If + * another process concurrently creates the table before this table's staged changes are + * committed, an exception should be thrown by {@link StagedTable#commitStagedChanges()}. + * + * @param ident a table identifier + * @param schema the schema of the new table, as a struct type + * @param partitions transforms to use for partitioning data in the table + * @param properties a string map of table properties + * @return metadata for the new table + * @throws TableAlreadyExistsException If a table or view already exists for the identifier + * @throws UnsupportedOperationException If a requested partition transform is not supported + * @throws NoSuchNamespaceException If the identifier namespace does not exist (optional) + */ + StagedTable stageCreate( + Identifier ident, + StructType schema, + Transform[] partitions, + Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException; + + /** + * Stage the replacement of a table, preparing it to be committed into the metastore when the + * returned table's {@link StagedTable#commitStagedChanges()} is called. + *

+ * When the table is committed, the contents of any writes performed by the Spark planner are + * committed along with the metadata about the table passed into this method's arguments. If the + * table exists, the metadata and the contents of this table replace the metadata and contents of + * the existing table. If a concurrent process commits changes to the table's data or metadata + * while the write is being performed but before the staged changes are committed, the catalog + * can decide whether to move forward with the table replacement anyways or abort the commit + * operation. + *

+ * If the table does not exist, committing the staged changes should fail with + * {@link NoSuchTableException}. This differs from the semantics of + * {@link #stageCreateOrReplace(Identifier, StructType, Transform[], Map)}, which should create + * the table in the data source if the table does not exist at the time of committing the + * operation. + * + * @param ident a table identifier + * @param schema the schema of the new table, as a struct type + * @param partitions transforms to use for partitioning data in the table + * @param properties a string map of table properties + * @return metadata for the new table + * @throws UnsupportedOperationException If a requested partition transform is not supported + * @throws NoSuchNamespaceException If the identifier namespace does not exist (optional) + * @throws NoSuchTableException If the table does not exist + */ + StagedTable stageReplace( + Identifier ident, + StructType schema, + Transform[] partitions, + Map properties) throws NoSuchNamespaceException, NoSuchTableException; + + /** + * Stage the creation or replacement of a table, preparing it to be committed into the metastore + * when the returned table's {@link StagedTable#commitStagedChanges()} is called. + *

+ * When the table is committed, the contents of any writes performed by the Spark planner are + * committed along with the metadata about the table passed into this method's arguments. If the + * table exists, the metadata and the contents of this table replace the metadata and contents of + * the existing table. If a concurrent process commits changes to the table's data or metadata + * while the write is being performed but before the staged changes are committed, the catalog + * can decide whether to move forward with the table replacement anyways or abort the commit + * operation. + *

+ * If the table does not exist when the changes are committed, the table should be created in the + * backing data source. This differs from the expected semantics of + * {@link #stageReplace(Identifier, StructType, Transform[], Map)}, which should fail when + * the staged changes are committed but the table doesn't exist at commit time. + * + * @param ident a table identifier + * @param schema the schema of the new table, as a struct type + * @param partitions transforms to use for partitioning data in the table + * @param properties a string map of table properties + * @return metadata for the new table + * @throws UnsupportedOperationException If a requested partition transform is not supported + * @throws NoSuchNamespaceException If the identifier namespace does not exist (optional) + */ + StagedTable stageCreateOrReplace( + Identifier ident, + StructType schema, + Transform[] partitions, + Map properties) throws NoSuchNamespaceException; +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/StagedTable.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/StagedTable.java new file mode 100644 index 0000000000000..b2baa93b146a5 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/StagedTable.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2; + +import java.util.Map; +import org.apache.spark.sql.catalog.v2.Identifier; +import org.apache.spark.sql.catalog.v2.StagingTableCatalog; +import org.apache.spark.sql.catalog.v2.expressions.Transform; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +/** + * Represents a table which is staged for being committed to the metastore. + *

+ * This is used to implement atomic CREATE TABLE AS SELECT and REPLACE TABLE AS SELECT queries. The + * planner will create one of these via + * {@link StagingTableCatalog#stageCreate(Identifier, StructType, Transform[], Map)} or + * {@link StagingTableCatalog#stageReplace(Identifier, StructType, Transform[], Map)} to prepare the + * table for being written to. This table should usually implement {@link SupportsWrite}. A new + * writer will be constructed via {@link SupportsWrite#newWriteBuilder(CaseInsensitiveStringMap)}, + * and the write will be committed. The job concludes with a call to {@link #commitStagedChanges()}, + * at which point implementations are expected to commit the table's metadata into the metastore + * along with the data that was written by the writes from the write builder this table created. + */ +public interface StagedTable extends Table { + + /** + * Finalize the creation or replacement of this table. + */ + void commitStagedChanges(); + + /** + * Abort the changes that were staged, both in metadata and from temporary outputs of this + * table's writers. + */ + void abortStagedChanges(); +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala index 9853a4fcc2f9d..29d81c553ff61 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala @@ -80,4 +80,10 @@ trait Encoder[T] extends Serializable { * A ClassTag that can be used to construct an Array to contain a collection of `T`. */ def clsTag: ClassTag[T] + + /** + * Create a copied [[Encoder]]. The implementation may just copy internal reusable fields to speed + * up the [[Encoder]] creation. + */ + def makeCopy: Encoder[T] } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index 6020b068155fc..488252aa0c7b5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -343,6 +343,9 @@ object CatalystTypeConverters { private class DecimalConverter(dataType: DecimalType) extends CatalystTypeConverter[Any, JavaBigDecimal, Decimal] { + + private val nullOnOverflow = SQLConf.get.decimalOperationsNullOnOverflow + override def toCatalystImpl(scalaValue: Any): Decimal = { val decimal = scalaValue match { case d: BigDecimal => Decimal(d) @@ -353,7 +356,7 @@ object CatalystTypeConverters { s"The value (${other.toString}) of the type (${other.getClass.getCanonicalName}) " + s"cannot be converted to ${dataType.catalogString}") } - decimal.toPrecision(dataType.precision, dataType.scale) + decimal.toPrecision(dataType.precision, dataType.scale, Decimal.ROUND_HALF_UP, nullOnOverflow) } override def toScala(catalystValue: Decimal): JavaBigDecimal = { if (catalystValue == null) null diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.scala new file mode 100644 index 0000000000000..3036f7c21093f --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CannotReplaceMissingTableException.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.catalyst.analysis + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalog.v2.Identifier + +class CannotReplaceMissingTableException( + tableIdentifier: Identifier, + cause: Option[Throwable] = None) + extends AnalysisException( + s"Table $tableIdentifier cannot be replaced as it did not exist." + + s" Use CREATE OR REPLACE TABLE to create the table.", cause = cause) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index c72400a8b72c2..3408b496d9d2a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -414,6 +414,7 @@ object FunctionRegistry { expression[WeekOfYear]("weekofyear"), expression[Year]("year"), expression[TimeWindow]("window"), + expression[MakeDate]("make_date"), // collection functions expression[CreateArray]("array"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 2d646721f87a2..c6c1d3bfa6347 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -561,6 +561,8 @@ object CatalogTableType { val EXTERNAL = new CatalogTableType("EXTERNAL") val MANAGED = new CatalogTableType("MANAGED") val VIEW = new CatalogTableType("VIEW") + + val tableTypes = Seq(EXTERNAL, MANAGED, VIEW) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala index 1268fcffcfcd0..7d52847216cc9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala @@ -163,6 +163,11 @@ class CSVOptions( val inputBufferSize = 128 + /** + * The max error content length in CSV parser/writer exception message. + */ + val maxErrorContentLength = 1000 + val isCommentSet = this.comment != '\u0000' val samplingRatio = @@ -220,6 +225,7 @@ class CSVOptions( writerSettings.setSkipEmptyLines(true) writerSettings.setQuoteAllFields(quoteAll) writerSettings.setQuoteEscapingEnabled(escapeQuotes) + writerSettings.setErrorContentLength(maxErrorContentLength) writerSettings } @@ -246,6 +252,7 @@ class CSVOptions( lineSeparatorInRead.foreach { _ => settings.setNormalizeLineEndingsWithinQuotes(!multiLine) } + settings.setErrorContentLength(maxErrorContentLength) settings } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index c97303be1d27c..bd499671d6441 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -382,6 +382,8 @@ case class ExpressionEncoder[T]( .map { case(f, a) => s"${f.name}$a: ${f.dataType.simpleString}"}.mkString(", ") override def toString: String = s"class[$schemaString]" + + override def makeCopy: ExpressionEncoder[T] = copy() } // A dummy logical plan that can hold expressions and go through optimizer rules. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index f671ede21782a..5314821ea3a59 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -16,7 +16,8 @@ */ package org.apache.spark.sql.catalyst.expressions -import java.util.{Comparator, TimeZone} +import java.time.ZoneId +import java.util.Comparator import scala.collection.mutable import scala.reflect.ClassTag @@ -2459,10 +2460,10 @@ case class Sequence( new IntegralSequenceImpl(iType)(ct, iType.integral) case TimestampType => - new TemporalSequenceImpl[Long](LongType, 1, identity, timeZone) + new TemporalSequenceImpl[Long](LongType, 1, identity, zoneId) case DateType => - new TemporalSequenceImpl[Int](IntegerType, MICROS_PER_DAY, _.toInt, timeZone) + new TemporalSequenceImpl[Int](IntegerType, MICROS_PER_DAY, _.toInt, zoneId) } override def eval(input: InternalRow): Any = { @@ -2603,7 +2604,7 @@ object Sequence { } private class TemporalSequenceImpl[T: ClassTag] - (dt: IntegralType, scale: Long, fromLong: Long => T, timeZone: TimeZone) + (dt: IntegralType, scale: Long, fromLong: Long => T, zoneId: ZoneId) (implicit num: Integral[T]) extends SequenceImpl { override val defaultStep: DefaultStep = new DefaultStep( @@ -2642,7 +2643,7 @@ object Sequence { while (t < exclusiveItem ^ stepSign < 0) { arr(i) = fromLong(t / scale) i += 1 - t = timestampAddInterval(startMicros, i * stepMonths, i * stepMicros, timeZone) + t = timestampAddInterval(startMicros, i * stepMonths, i * stepMicros, zoneId) } // truncate array to the correct length @@ -2668,7 +2669,7 @@ object Sequence { val exclusiveItem = ctx.freshName("exclusiveItem") val t = ctx.freshName("t") val i = ctx.freshName("i") - val genTimeZone = ctx.addReferenceObj("timeZone", timeZone, classOf[TimeZone].getName) + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) val sequenceLengthCode = s""" @@ -2701,7 +2702,7 @@ object Sequence { | $arr[$i] = ($elemType) ($t / ${scale}L); | $i += 1; | $t = org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampAddInterval( - | $startMicros, $i * $stepMonths, $i * $stepMicros, $genTimeZone); + | $startMicros, $i * $stepMonths, $i * $stepMicros, $zid); | } | | if ($arr.length > $i) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index ccf6b36effa08..edb5382ae4437 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -996,14 +996,14 @@ case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[S override def nullSafeEval(start: Any, interval: Any): Any = { val itvl = interval.asInstanceOf[CalendarInterval] DateTimeUtils.timestampAddInterval( - start.asInstanceOf[Long], itvl.months, itvl.microseconds, timeZone) + start.asInstanceOf[Long], itvl.months, itvl.microseconds, zoneId) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val tz = ctx.addReferenceObj("timeZone", timeZone) + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (sd, i) => { - s"""$dtu.timestampAddInterval($sd, $i.months, $i.microseconds, $tz)""" + s"""$dtu.timestampAddInterval($sd, $i.months, $i.microseconds, $zid)""" }) } } @@ -1111,14 +1111,14 @@ case class TimeSub(start: Expression, interval: Expression, timeZoneId: Option[S override def nullSafeEval(start: Any, interval: Any): Any = { val itvl = interval.asInstanceOf[CalendarInterval] DateTimeUtils.timestampAddInterval( - start.asInstanceOf[Long], 0 - itvl.months, 0 - itvl.microseconds, timeZone) + start.asInstanceOf[Long], 0 - itvl.months, 0 - itvl.microseconds, zoneId) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val tz = ctx.addReferenceObj("timeZone", timeZone) + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (sd, i) => { - s"""$dtu.timestampAddInterval($sd, 0 - $i.months, 0 - $i.microseconds, $tz)""" + s"""$dtu.timestampAddInterval($sd, 0 - $i.months, 0 - $i.microseconds, $zid)""" }) } } @@ -1605,3 +1605,55 @@ private case class GetTimestamp( override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = copy(timeZoneId = Option(timeZoneId)) } + +@ExpressionDescription( + usage = "_FUNC_(year, month, day) - Create date from year, month and day fields.", + arguments = """ + Arguments: + * year - the year to represent, from 1 to 9999 + * month - the month-of-year to represent, from 1 (January) to 12 (December) + * day - the day-of-month to represent, from 1 to 31 + """, + examples = """ + Examples: + > SELECT _FUNC_(2013, 7, 15); + 2013-07-15 + > SELECT _FUNC_(2019, 13, 1); + NULL + > SELECT _FUNC_(2019, 7, NULL); + NULL + > SELECT _FUNC_(2019, 2, 30); + NULL + """, + since = "3.0.0") +case class MakeDate(year: Expression, month: Expression, day: Expression) + extends TernaryExpression with ImplicitCastInputTypes { + + override def children: Seq[Expression] = Seq(year, month, day) + override def inputTypes: Seq[AbstractDataType] = Seq(IntegerType, IntegerType, IntegerType) + override def dataType: DataType = DateType + override def nullable: Boolean = true + + override def nullSafeEval(year: Any, month: Any, day: Any): Any = { + try { + val ld = LocalDate.of(year.asInstanceOf[Int], month.asInstanceOf[Int], day.asInstanceOf[Int]) + localDateToDays(ld) + } catch { + case _: java.time.DateTimeException => null + } + } + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + nullSafeCodeGen(ctx, ev, (year, month, day) => { + s""" + try { + ${ev.value} = $dtu.localDateToDays(java.time.LocalDate.of($year, $month, $day)); + } catch (java.time.DateTimeException e) { + ${ev.isNull} = true; + }""" + }) + } + + override def prettyName: String = "make_date" +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala index e873f8ed1a21c..6dd2fa716e6bc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala @@ -1177,6 +1177,7 @@ abstract class RoundBase(child: Expression, scale: Expression, dataType match { case DecimalType.Fixed(_, s) => val decimal = input1.asInstanceOf[Decimal] + // Overflow cannot happen, so no need to control nullOnOverflow decimal.toPrecision(decimal.precision, s, mode) case ByteType => BigDecimal(input1.asInstanceOf[Byte]).setScale(_scale, mode).toByte diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala index 95aefb6422d67..43a6006f9b5c0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala @@ -54,7 +54,7 @@ object NestedColumnAliasing { /** * Return a replaced project list. */ - private def getNewProjectList( + def getNewProjectList( projectList: Seq[NamedExpression], nestedFieldToAlias: Map[ExtractValue, Alias]): Seq[NamedExpression] = { projectList.map(_.transform { @@ -66,7 +66,7 @@ object NestedColumnAliasing { /** * Return a plan with new children replaced with aliases. */ - private def replaceChildrenWithAliases( + def replaceChildrenWithAliases( plan: LogicalPlan, attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = { plan.withNewChildren(plan.children.map { plan => @@ -107,10 +107,10 @@ object NestedColumnAliasing { * 1. ExtractValue -> Alias: A new alias is created for each nested field. * 2. ExprId -> Seq[Alias]: A reference attribute has multiple aliases pointing it. */ - private def getAliasSubMap(projectList: Seq[NamedExpression]) + def getAliasSubMap(exprList: Seq[Expression]) : Option[(Map[ExtractValue, Alias], Map[ExprId, Seq[Alias]])] = { val (nestedFieldReferences, otherRootReferences) = - projectList.flatMap(collectRootReferenceAndExtractValue).partition { + exprList.flatMap(collectRootReferenceAndExtractValue).partition { case _: ExtractValue => true case _ => false } @@ -155,4 +155,15 @@ object NestedColumnAliasing { case MapType(keyType, valueType, _) => totalFieldNum(keyType) + totalFieldNum(valueType) case _ => 1 // UDT and others } + + /** + * This is a while-list for pruning nested fields at `Generator`. + */ + def canPruneGenerator(g: Generator): Boolean = g match { + case _: Explode => true + case _: Stack => true + case _: PosExplode => true + case _: Inline => true + case _ => false + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index c99d2c06fac63..206d09a6f79e9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -487,7 +487,7 @@ object LimitPushDown extends Rule[LogicalPlan] { * Union: * Right now, Union means UNION ALL, which does not de-duplicate rows. So, it is * safe to pushdown Filters and Projections through it. Filter pushdown is handled by another - * rule PushDownPredicate. Once we add UNION DISTINCT, we will not be able to pushdown Projections. + * rule PushDownPredicates. Once we add UNION DISTINCT, we will not be able to pushdown Projections. */ object PushProjectionThroughUnion extends Rule[LogicalPlan] with PredicateHelper { @@ -588,6 +588,24 @@ object ColumnPruning extends Rule[LogicalPlan] { .map(_._2) p.copy(child = g.copy(child = newChild, unrequiredChildIndex = unrequiredIndices)) + // prune unrequired nested fields + case p @ Project(projectList, g: Generate) if SQLConf.get.nestedPruningOnExpressions && + NestedColumnAliasing.canPruneGenerator(g.generator) => + NestedColumnAliasing.getAliasSubMap(projectList ++ g.generator.children).map { + case (nestedFieldToAlias, attrToAliases) => + val newGenerator = g.generator.transform { + case f: ExtractValue if nestedFieldToAlias.contains(f) => + nestedFieldToAlias(f).toAttribute + }.asInstanceOf[Generator] + + // Defer updating `Generate.unrequiredChildIndex` to next round of `ColumnPruning`. + val newGenerate = g.copy(generator = newGenerator) + + val newChild = NestedColumnAliasing.replaceChildrenWithAliases(newGenerate, attrToAliases) + + Project(NestedColumnAliasing.getNewProjectList(projectList, nestedFieldToAlias), newChild) + }.getOrElse(p) + // Eliminate unneeded attributes from right side of a Left Existence Join. case j @ Join(_, right, LeftExistence(_), _, _) => j.copy(right = prunedChild(right, j.references)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index d9f8b9a7203ff..a7a3b96ba726d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, QualifiedColType} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -1926,6 +1926,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case ("decimal", precision :: Nil) => DecimalType(precision.getText.toInt, 0) case ("decimal", precision :: scale :: Nil) => DecimalType(precision.getText.toInt, scale.getText.toInt) + case ("interval", Nil) => CalendarIntervalType case (dt, params) => val dtStr = if (params.nonEmpty) s"$dt(${params.mkString(",")})" else dt throw new ParseException(s"DataType $dtStr is not supported.", ctx) @@ -2127,6 +2128,15 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging (multipartIdentifier, temporary, ifNotExists, ctx.EXTERNAL != null) } + /** + * Validate a replace table statement and return the [[TableIdentifier]]. + */ + override def visitReplaceTableHeader( + ctx: ReplaceTableHeaderContext): TableHeader = withOrigin(ctx) { + val multipartIdentifier = ctx.multipartIdentifier.parts.asScala.map(_.getText) + (multipartIdentifier, false, false, false) + } + /** * Parse a qualified name to a multipart name. */ @@ -2294,6 +2304,69 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + /** + * Replace a table, returning a [[ReplaceTableStatement]] logical plan. + * + * Expected format: + * {{{ + * [CREATE OR] REPLACE TABLE [db_name.]table_name + * USING table_provider + * replace_table_clauses + * [[AS] select_statement]; + * + * replace_table_clauses (order insensitive): + * [OPTIONS table_property_list] + * [PARTITIONED BY (col_name, transform(col_name), transform(constant, col_name), ...)] + * [CLUSTERED BY (col_name, col_name, ...) + * [SORTED BY (col_name [ASC|DESC], ...)] + * INTO num_buckets BUCKETS + * ] + * [LOCATION path] + * [COMMENT table_comment] + * [TBLPROPERTIES (property_name=property_value, ...)] + * }}} + */ + override def visitReplaceTable(ctx: ReplaceTableContext): LogicalPlan = withOrigin(ctx) { + val (table, _, ifNotExists, external) = visitReplaceTableHeader(ctx.replaceTableHeader) + if (external) { + operationNotAllowed("REPLACE EXTERNAL TABLE ... USING", ctx) + } + + checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx) + checkDuplicateClauses(ctx.OPTIONS, "OPTIONS", ctx) + checkDuplicateClauses(ctx.PARTITIONED, "PARTITIONED BY", ctx) + checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx) + checkDuplicateClauses(ctx.bucketSpec(), "CLUSTERED BY", ctx) + checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx) + + val schema = Option(ctx.colTypeList()).map(createSchema) + val partitioning: Seq[Transform] = + Option(ctx.partitioning).map(visitTransformList).getOrElse(Nil) + val bucketSpec = ctx.bucketSpec().asScala.headOption.map(visitBucketSpec) + val properties = Option(ctx.tableProps).map(visitPropertyKeyValues).getOrElse(Map.empty) + val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) + + val provider = ctx.tableProvider.qualifiedName.getText + val location = ctx.locationSpec.asScala.headOption.map(visitLocationSpec) + val comment = Option(ctx.comment).map(string) + val orCreate = ctx.replaceTableHeader().CREATE() != null + + Option(ctx.query).map(plan) match { + case Some(_) if schema.isDefined => + operationNotAllowed( + "Schema may not be specified in a Replace Table As Select (RTAS) statement", + ctx) + + case Some(query) => + ReplaceTableAsSelectStatement(table, query, partitioning, bucketSpec, properties, + provider, options, location, comment, orCreate = orCreate) + + case _ => + ReplaceTableStatement(table, schema.getOrElse(new StructType), partitioning, + bucketSpec, properties, provider, options, location, comment, orCreate = orCreate) + } + } + /** * Create a [[DropTableStatement]] command. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 2cb04c9ec70c5..2698ba282f962 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -441,6 +441,47 @@ case class CreateTableAsSelect( } } +/** + * Replace a table with a v2 catalog. + * + * If the table does not exist, and orCreate is true, then it will be created. + * If the table does not exist, and orCreate is false, then an exception will be thrown. + * + * The persisted table will have no contents as a result of this operation. + */ +case class ReplaceTable( + catalog: TableCatalog, + tableName: Identifier, + tableSchema: StructType, + partitioning: Seq[Transform], + properties: Map[String, String], + orCreate: Boolean) extends Command + +/** + * Replaces a table from a select query with a v2 catalog. + * + * If the table does not exist, and orCreate is true, then it will be created. + * If the table does not exist, and orCreate is false, then an exception will be thrown. + */ +case class ReplaceTableAsSelect( + catalog: TableCatalog, + tableName: Identifier, + partitioning: Seq[Transform], + query: LogicalPlan, + properties: Map[String, String], + writeOptions: Map[String, String], + orCreate: Boolean) extends Command { + + override def children: Seq[LogicalPlan] = Seq(query) + + override lazy val resolved: Boolean = { + // the table schema is created from the query schema, so the only resolution needed is to check + // that the columns referenced by the table's partitioning exist in the query schema + val references = partitioning.flatMap(_.references).toSet + references.map(_.fieldNames).forall(query.schema.findNestedField(_).isDefined) + } +} + /** * Append data to an existing table. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala new file mode 100644 index 0000000000000..2808892b089b9 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ReplaceTableStatement.scala @@ -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. + */ + +package org.apache.spark.sql.catalyst.plans.logical.sql + +import org.apache.spark.sql.catalog.v2.expressions.Transform +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.types.StructType + +/** + * A REPLACE TABLE command, as parsed from SQL. + * + * If the table exists prior to running this command, executing this statement + * will replace the table's metadata and clear the underlying rows from the table. + */ +case class ReplaceTableStatement( + tableName: Seq[String], + tableSchema: StructType, + partitioning: Seq[Transform], + bucketSpec: Option[BucketSpec], + properties: Map[String, String], + provider: String, + options: Map[String, String], + location: Option[String], + comment: Option[String], + orCreate: Boolean) extends ParsedStatement + +/** + * A REPLACE TABLE AS SELECT command, as parsed from SQL. + */ +case class ReplaceTableAsSelectStatement( + tableName: Seq[String], + asSelect: LogicalPlan, + partitioning: Seq[Transform], + bucketSpec: Option[BucketSpec], + properties: Map[String, String], + provider: String, + options: Map[String, String], + location: Option[String], + comment: Option[String], + orCreate: Boolean) extends ParsedStatement { + + override def children: Seq[LogicalPlan] = Seq(asSelect) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 0596dc00985a1..e79000d583506 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -287,7 +287,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { mapChildren(_.transformDown(rule)) } else { // If the transform function replaces this node with a new one, carry over the tags. - afterRule.tags ++= this.tags + afterRule.copyTagsFrom(this) afterRule.mapChildren(_.transformDown(rule)) } } @@ -311,7 +311,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } } // If the transform function replaces this node with a new one, carry over the tags. - newNode.tags ++= this.tags + newNode.copyTagsFrom(this) newNode } @@ -429,8 +429,15 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { private def makeCopy( newArgs: Array[AnyRef], allowEmptyArgs: Boolean): BaseType = attachTree(this, "makeCopy") { + val allCtors = getClass.getConstructors + if (newArgs.isEmpty && allCtors.isEmpty) { + // This is a singleton object which doesn't have any constructor. Just return `this` as we + // can't copy it. + return this + } + // Skip no-arg constructors that are just there for kryo. - val ctors = getClass.getConstructors.filter(allowEmptyArgs || _.getParameterTypes.size != 0) + val ctors = allCtors.filter(allowEmptyArgs || _.getParameterTypes.size != 0) if (ctors.isEmpty) { sys.error(s"No valid constructor for $nodeName") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 1daf65a0c560c..10a7f9bd550e2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -19,8 +19,7 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} import java.time._ -import java.time.Year.isLeap -import java.time.temporal.IsoFields +import java.time.temporal.{ChronoUnit, IsoFields} import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit._ @@ -521,12 +520,12 @@ object DateTimeUtils { start: SQLTimestamp, months: Int, microseconds: Long, - timeZone: TimeZone): SQLTimestamp = { - val days = millisToDays(MICROSECONDS.toMillis(start), timeZone) - val newDays = dateAddMonths(days, months) - start + - MILLISECONDS.toMicros(daysToMillis(newDays, timeZone) - daysToMillis(days, timeZone)) + - microseconds + zoneId: ZoneId): SQLTimestamp = { + val resultTimestamp = microsToInstant(start) + .atZone(zoneId) + .plusMonths(months) + .plus(microseconds, ChronoUnit.MICROS) + instantToMicros(resultTimestamp.toInstant) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 57f5128fd4fbe..fbdb1c5f957d9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1656,6 +1656,16 @@ object SQLConf { .booleanConf .createWithDefault(false) + val NESTED_PRUNING_ON_EXPRESSIONS = + buildConf("spark.sql.optimizer.expression.nestedPruning.enabled") + .internal() + .doc("Prune nested fields from expressions in an operator which are unnecessary in " + + "satisfying a query. Note that this optimization doesn't prune nested fields from " + + "physical data source scanning. For pruning nested fields from scanning, please use " + + "`spark.sql.optimizer.nestedSchemaPruning.enabled` config.") + .booleanConf + .createWithDefault(false) + val TOP_K_SORT_FALLBACK_THRESHOLD = buildConf("spark.sql.execution.topKSortFallbackThreshold") .internal() @@ -2315,6 +2325,8 @@ class SQLConf extends Serializable with Logging { def serializerNestedSchemaPruningEnabled: Boolean = getConf(SERIALIZER_NESTED_SCHEMA_PRUNING_ENABLED) + def nestedPruningOnExpressions: Boolean = getConf(NESTED_PRUNING_ON_EXPRESSIONS) + def csvColumnPruning: Boolean = getConf(SQLConf.CSV_PARSER_COLUMN_PRUNING) def legacySizeOfNull: Boolean = getConf(SQLConf.LEGACY_SIZE_OF_NULL) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala index 8e297874a0d62..ea94cf626698a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala @@ -34,6 +34,8 @@ class CalendarIntervalType private() extends DataType { override def defaultSize: Int = 16 + override def simpleString: String = "interval" + private[spark] override def asNullable: CalendarIntervalType = this } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 1bf322af21799..a5d1a72d62d5c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -414,20 +414,12 @@ final class Decimal extends Ordered[Decimal] with Serializable { def floor: Decimal = if (scale == 0) this else { val newPrecision = DecimalType.bounded(precision - scale + 1, 0).precision - val res = toPrecision(newPrecision, 0, ROUND_FLOOR) - if (res == null) { - throw new AnalysisException(s"Overflow when setting precision to $newPrecision") - } - res + toPrecision(newPrecision, 0, ROUND_FLOOR, nullOnOverflow = false) } def ceil: Decimal = if (scale == 0) this else { val newPrecision = DecimalType.bounded(precision - scale + 1, 0).precision - val res = toPrecision(newPrecision, 0, ROUND_CEILING) - if (res == null) { - throw new AnalysisException(s"Overflow when setting precision to $newPrecision") - } - res + toPrecision(newPrecision, 0, ROUND_CEILING, nullOnOverflow = false) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala index 725764755c626..4440ac9e281c4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala @@ -77,9 +77,9 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { assertErrorForDifferingTypes(BitwiseOr('intField, 'booleanField)) assertErrorForDifferingTypes(BitwiseXor('intField, 'booleanField)) - assertError(Add('booleanField, 'booleanField), "requires (numeric or calendarinterval) type") + assertError(Add('booleanField, 'booleanField), "requires (numeric or interval) type") assertError(Subtract('booleanField, 'booleanField), - "requires (numeric or calendarinterval) type") + "requires (numeric or interval) type") assertError(Multiply('booleanField, 'booleanField), "requires numeric type") assertError(Divide('booleanField, 'booleanField), "requires (double or decimal) type") assertError(Remainder('booleanField, 'booleanField), "requires numeric type") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index f4feeca1d05ad..9380c7e3f5f72 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -427,6 +427,10 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes testOverflowingBigNumeric(BigInt("9" * 100), "scala very large big int") testOverflowingBigNumeric(new BigInteger("9" * 100), "java very big int") + encodeDecodeTest("foo" -> 1L, "makeCopy") { + Encoders.product[(String, Long)].makeCopy.asInstanceOf[ExpressionEncoder[(String, Long)]] + } + private def testOverflowingBigNumeric[T: TypeTag](bigNumeric: T, testName: String): Unit = { Seq(true, false).foreach { allowNullOnOverflow => testAndVerifyNotLeakingReflectionObjects( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 4e8322d3c55d7..b4110afd55057 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -918,4 +918,14 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } } + + test("creating values of DateType via make_date") { + checkEvaluation(MakeDate(Literal(2013), Literal(7), Literal(15)), Date.valueOf("2013-7-15")) + checkEvaluation(MakeDate(Literal.create(null, IntegerType), Literal(7), Literal(15)), null) + checkEvaluation(MakeDate(Literal(2019), Literal.create(null, IntegerType), Literal(19)), null) + checkEvaluation(MakeDate(Literal(2019), Literal(7), Literal.create(null, IntegerType)), null) + checkEvaluation(MakeDate(Literal(Int.MaxValue), Literal(13), Literal(19)), null) + checkEvaluation(MakeDate(Literal(2019), Literal(13), Literal(19)), null) + checkEvaluation(MakeDate(Literal(2019), Literal(7), Literal(32)), null) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala index b190d6f5caa1c..f8400a590606a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala @@ -696,7 +696,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val struct2 = Literal.create(null, schema2) StructsToJson(Map.empty, struct2, gmtId).checkInputDataTypes() match { case TypeCheckResult.TypeCheckFailure(msg) => - assert(msg.contains("Unable to convert column a of type calendarinterval to JSON")) + assert(msg.contains("Unable to convert column a of type interval to JSON")) case _ => fail("from_json should not work on interval map value type.") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala index df92fa3475bd9..981ef57c051fd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala @@ -21,7 +21,8 @@ import java.util.Locale import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext -import org.apache.spark.sql.types.{IntegerType, StringType} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{DecimalType, IntegerType, StringType} class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -54,4 +55,26 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil, false :: Nil).genCode(ctx) assert(ctx.inlinedMutableStates.isEmpty) } + + test("SPARK-28369: honor nullOnOverflow config for ScalaUDF") { + withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "false") { + val udf = ScalaUDF( + (a: java.math.BigDecimal) => a.multiply(new java.math.BigDecimal(100)), + DecimalType.SYSTEM_DEFAULT, + Literal(BigDecimal("12345678901234567890.123")) :: Nil, false :: Nil) + val e1 = intercept[ArithmeticException](udf.eval()) + assert(e1.getMessage.contains("cannot be represented as Decimal")) + val e2 = intercept[SparkException] { + checkEvaluationWithUnsafeProjection(udf, null) + } + assert(e2.getCause.isInstanceOf[ArithmeticException]) + } + withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "true") { + val udf = ScalaUDF( + (a: java.math.BigDecimal) => a.multiply(new java.math.BigDecimal(100)), + DecimalType.SYSTEM_DEFAULT, + Literal(BigDecimal("12345678901234567890.123")) :: Nil, false :: Nil) + checkEvaluation(udf, null) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala index 78ae131328644..75ff07637fccc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.optimizer import scala.reflect.runtime.universe.TypeTag +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder @@ -26,7 +27,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{StringType, StructType} class ColumnPruningSuite extends PlanTest { @@ -101,6 +103,81 @@ class ColumnPruningSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + test("Nested column pruning for Generate") { + def runTest( + origGenerator: Generator, + replacedGenerator: Seq[String] => Generator, + aliasedExprs: Seq[String] => Seq[Expression], + unrequiredChildIndex: Seq[Int], + generatorOutputNames: Seq[String]) { + withSQLConf(SQLConf.NESTED_PRUNING_ON_EXPRESSIONS.key -> "true") { + val structType = StructType.fromDDL("d double, e array, f double, g double, " + + "h array>") + val input = LocalRelation('a.int, 'b.int, 'c.struct(structType)) + val generatorOutputs = generatorOutputNames.map(UnresolvedAttribute(_)) + + val selectedExprs = Seq(UnresolvedAttribute("a"), 'c.getField("d")) ++ + generatorOutputs + + val query = + input + .generate(origGenerator, outputNames = generatorOutputNames) + .select(selectedExprs: _*) + .analyze + + val optimized = Optimize.execute(query) + + val aliases = NestedColumnAliasingSuite.collectGeneratedAliases(optimized) + + val selectedFields = UnresolvedAttribute("a") +: aliasedExprs(aliases) + val finalSelectedExprs = Seq(UnresolvedAttribute("a"), $"${aliases(0)}".as("c.d")) ++ + generatorOutputs + + val correctAnswer = + input + .select(selectedFields: _*) + .generate(replacedGenerator(aliases), + unrequiredChildIndex = unrequiredChildIndex, + outputNames = generatorOutputNames) + .select(finalSelectedExprs: _*) + .analyze + + comparePlans(optimized, correctAnswer) + } + } + + runTest( + Explode('c.getField("e")), + aliases => Explode($"${aliases(1)}".as("c.e")), + aliases => Seq('c.getField("d").as(aliases(0)), 'c.getField("e").as(aliases(1))), + Seq(2), + Seq("explode") + ) + runTest(Stack(2 :: 'c.getField("f") :: 'c.getField("g") :: Nil), + aliases => Stack(2 :: $"${aliases(1)}".as("c.f") :: $"${aliases(2)}".as("c.g") :: Nil), + aliases => Seq( + 'c.getField("d").as(aliases(0)), + 'c.getField("f").as(aliases(1)), + 'c.getField("g").as(aliases(2))), + Seq(2, 3), + Seq("stack") + ) + runTest( + PosExplode('c.getField("e")), + aliases => PosExplode($"${aliases(1)}".as("c.e")), + aliases => Seq('c.getField("d").as(aliases(0)), 'c.getField("e").as(aliases(1))), + Seq(2), + Seq("pos", "explode") + ) + runTest( + Inline('c.getField("h")), + aliases => Inline($"${aliases(1)}".as("c.h")), + aliases => Seq('c.getField("d").as(aliases(0)), 'c.getField("h").as(aliases(1))), + Seq(2), + Seq("h1", "h2") + ) + } + test("Column pruning for Project on Sort") { val input = LocalRelation('a.int, 'b.string, 'c.double) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala index ab2bd6dff1265..2351d8321c5f3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala @@ -29,6 +29,8 @@ import org.apache.spark.sql.types.{StringType, StructField, StructType} class NestedColumnAliasingSuite extends SchemaPruningTest { + import NestedColumnAliasingSuite._ + object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Nested column pruning", FixedPoint(100), ColumnPruning, @@ -264,9 +266,10 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { .analyze comparePlans(optimized, expected) } +} - - private def collectGeneratedAliases(query: LogicalPlan): ArrayBuffer[String] = { +object NestedColumnAliasingSuite { + def collectGeneratedAliases(query: LogicalPlan): ArrayBuffer[String] = { val aliases = ArrayBuffer[String]() query.transformAllExpressions { case a @ Alias(_, name) if name.startsWith("_gen_alias_") => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index d008b3c78fac3..dd84170e26200 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -19,11 +19,11 @@ package org.apache.spark.sql.catalyst.parser import java.util.Locale -import org.apache.spark.sql.catalog.v2.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, YearsTransform} +import org.apache.spark.sql.catalog.v2.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.catalyst.analysis.AnalysisTest import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, QualifiedColType} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String @@ -47,82 +47,71 @@ class DDLParserSuite extends AnalysisTest { comparePlans(parsePlan(sql), expected, checkAnalysis = false) } - test("create table using - schema") { - val sql = "CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) USING parquet" - - parsePlan(sql) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("my_tab")) - assert(create.tableSchema == new StructType() - .add("a", IntegerType, nullable = true, "test") - .add("b", StringType)) - assert(create.partitioning.isEmpty) - assert(create.bucketSpec.isEmpty) - assert(create.properties.isEmpty) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.isEmpty) - assert(create.comment.isEmpty) - assert(!create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") + test("create/replace table using - schema") { + val createSql = "CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) USING parquet" + val replaceSql = "REPLACE TABLE my_tab(a INT COMMENT 'test', b STRING) USING parquet" + val expectedTableSpec = TableSpec( + Seq("my_tab"), + Some(new StructType() + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType)), + Seq.empty[Transform], + None, + Map.empty[String, String], + "parquet", + Map.empty[String, String], + None, + None) + + Seq(createSql, replaceSql).foreach { sql => + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } intercept("CREATE TABLE my_tab(a: INT COMMENT 'test', b: STRING) USING parquet", "no viable alternative at input") } - test("create table - with IF NOT EXISTS") { + test("create/replace table - with IF NOT EXISTS") { val sql = "CREATE TABLE IF NOT EXISTS my_tab(a INT, b STRING) USING parquet" - - parsePlan(sql) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("my_tab")) - assert(create.tableSchema == new StructType().add("a", IntegerType).add("b", StringType)) - assert(create.partitioning.isEmpty) - assert(create.bucketSpec.isEmpty) - assert(create.properties.isEmpty) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.isEmpty) - assert(create.comment.isEmpty) - assert(create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") - } + testCreateOrReplaceDdl( + sql, + TableSpec( + Seq("my_tab"), + Some(new StructType().add("a", IntegerType).add("b", StringType)), + Seq.empty[Transform], + None, + Map.empty[String, String], + "parquet", + Map.empty[String, String], + None, + None), + expectedIfNotExists = true) } - test("create table - with partitioned by") { - val query = "CREATE TABLE my_tab(a INT comment 'test', b STRING) " + + test("create/replace table - with partitioned by") { + val createSql = "CREATE TABLE my_tab(a INT comment 'test', b STRING) " + "USING parquet PARTITIONED BY (a)" - - parsePlan(query) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("my_tab")) - assert(create.tableSchema == new StructType() - .add("a", IntegerType, nullable = true, "test") - .add("b", StringType)) - assert(create.partitioning == Seq(IdentityTransform(FieldReference("a")))) - assert(create.bucketSpec.isEmpty) - assert(create.properties.isEmpty) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.isEmpty) - assert(create.comment.isEmpty) - assert(!create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $query") + val replaceSql = "REPLACE TABLE my_tab(a INT comment 'test', b STRING) " + + "USING parquet PARTITIONED BY (a)" + val expectedTableSpec = TableSpec( + Seq("my_tab"), + Some(new StructType() + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType)), + Seq(IdentityTransform(FieldReference("a"))), + None, + Map.empty[String, String], + "parquet", + Map.empty[String, String], + None, + None) + Seq(createSql, replaceSql).foreach { sql => + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } } - test("create table - partitioned by transforms") { - val sql = + test("create/replace table - partitioned by transforms") { + val createSql = """ |CREATE TABLE my_tab (a INT, b STRING, ts TIMESTAMP) USING parquet |PARTITIONED BY ( @@ -135,154 +124,151 @@ class DDLParserSuite extends AnalysisTest { | foo(a, "bar", 34)) """.stripMargin - parsePlan(sql) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("my_tab")) - assert(create.tableSchema == new StructType() - .add("a", IntegerType) - .add("b", StringType) - .add("ts", TimestampType)) - assert(create.partitioning == Seq( - IdentityTransform(FieldReference("a")), - BucketTransform(LiteralValue(16, IntegerType), Seq(FieldReference("b"))), - YearsTransform(FieldReference("ts")), - MonthsTransform(FieldReference("ts")), - DaysTransform(FieldReference("ts")), - HoursTransform(FieldReference("ts")), - ApplyTransform("foo", Seq( - FieldReference("a"), - LiteralValue(UTF8String.fromString("bar"), StringType), - LiteralValue(34, IntegerType))))) - assert(create.bucketSpec.isEmpty) - assert(create.properties.isEmpty) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.isEmpty) - assert(create.comment.isEmpty) - assert(!create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") + val replaceSql = + """ + |REPLACE TABLE my_tab (a INT, b STRING, ts TIMESTAMP) USING parquet + |PARTITIONED BY ( + | a, + | bucket(16, b), + | years(ts), + | months(ts), + | days(ts), + | hours(ts), + | foo(a, "bar", 34)) + """.stripMargin + val expectedTableSpec = TableSpec( + Seq("my_tab"), + Some(new StructType() + .add("a", IntegerType) + .add("b", StringType) + .add("ts", TimestampType)), + Seq( + IdentityTransform(FieldReference("a")), + BucketTransform(LiteralValue(16, IntegerType), Seq(FieldReference("b"))), + YearsTransform(FieldReference("ts")), + MonthsTransform(FieldReference("ts")), + DaysTransform(FieldReference("ts")), + HoursTransform(FieldReference("ts")), + ApplyTransform("foo", Seq( + FieldReference("a"), + LiteralValue(UTF8String.fromString("bar"), StringType), + LiteralValue(34, IntegerType)))), + None, + Map.empty[String, String], + "parquet", + Map.empty[String, String], + None, + None) + Seq(createSql, replaceSql).foreach { sql => + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } } - test("create table - with bucket") { - val query = "CREATE TABLE my_tab(a INT, b STRING) USING parquet " + + test("create/replace table - with bucket") { + val createSql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet " + "CLUSTERED BY (a) SORTED BY (b) INTO 5 BUCKETS" - parsePlan(query) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("my_tab")) - assert(create.tableSchema == new StructType().add("a", IntegerType).add("b", StringType)) - assert(create.partitioning.isEmpty) - assert(create.bucketSpec.contains(BucketSpec(5, Seq("a"), Seq("b")))) - assert(create.properties.isEmpty) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.isEmpty) - assert(create.comment.isEmpty) - assert(!create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $query") + val replaceSql = "REPLACE TABLE my_tab(a INT, b STRING) USING parquet " + + "CLUSTERED BY (a) SORTED BY (b) INTO 5 BUCKETS" + + val expectedTableSpec = TableSpec( + Seq("my_tab"), + Some(new StructType().add("a", IntegerType).add("b", StringType)), + Seq.empty[Transform], + Some(BucketSpec(5, Seq("a"), Seq("b"))), + Map.empty[String, String], + "parquet", + Map.empty[String, String], + None, + None) + Seq(createSql, replaceSql).foreach { sql => + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } } - test("create table - with comment") { - val sql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet COMMENT 'abc'" - - parsePlan(sql) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("my_tab")) - assert(create.tableSchema == new StructType().add("a", IntegerType).add("b", StringType)) - assert(create.partitioning.isEmpty) - assert(create.bucketSpec.isEmpty) - assert(create.properties.isEmpty) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.isEmpty) - assert(create.comment.contains("abc")) - assert(!create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") + test("create/replace table - with comment") { + val createSql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet COMMENT 'abc'" + val replaceSql = "REPLACE TABLE my_tab(a INT, b STRING) USING parquet COMMENT 'abc'" + val expectedTableSpec = TableSpec( + Seq("my_tab"), + Some(new StructType().add("a", IntegerType).add("b", StringType)), + Seq.empty[Transform], + None, + Map.empty[String, String], + "parquet", + Map.empty[String, String], + None, + Some("abc")) + Seq(createSql, replaceSql).foreach{ sql => + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } } - test("create table - with table properties") { - val sql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet TBLPROPERTIES('test' = 'test')" - - parsePlan(sql) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("my_tab")) - assert(create.tableSchema == new StructType().add("a", IntegerType).add("b", StringType)) - assert(create.partitioning.isEmpty) - assert(create.bucketSpec.isEmpty) - assert(create.properties == Map("test" -> "test")) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.isEmpty) - assert(create.comment.isEmpty) - assert(!create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") + test("create/replace table - with table properties") { + val createSql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet" + + " TBLPROPERTIES('test' = 'test')" + val replaceSql = "REPLACE TABLE my_tab(a INT, b STRING) USING parquet" + + " TBLPROPERTIES('test' = 'test')" + val expectedTableSpec = TableSpec( + Seq("my_tab"), + Some(new StructType().add("a", IntegerType).add("b", StringType)), + Seq.empty[Transform], + None, + Map("test" -> "test"), + "parquet", + Map.empty[String, String], + None, + None) + Seq(createSql, replaceSql).foreach { sql => + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } } - test("create table - with location") { - val sql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet LOCATION '/tmp/file'" - - parsePlan(sql) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("my_tab")) - assert(create.tableSchema == new StructType().add("a", IntegerType).add("b", StringType)) - assert(create.partitioning.isEmpty) - assert(create.bucketSpec.isEmpty) - assert(create.properties.isEmpty) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.contains("/tmp/file")) - assert(create.comment.isEmpty) - assert(!create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") + test("create/replace table - with location") { + val createSql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet LOCATION '/tmp/file'" + val replaceSql = "REPLACE TABLE my_tab(a INT, b STRING) USING parquet LOCATION '/tmp/file'" + val expectedTableSpec = TableSpec( + Seq("my_tab"), + Some(new StructType().add("a", IntegerType).add("b", StringType)), + Seq.empty[Transform], + None, + Map.empty[String, String], + "parquet", + Map.empty[String, String], + Some("/tmp/file"), + None) + Seq(createSql, replaceSql).foreach { sql => + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } } - test("create table - byte length literal table name") { - val sql = "CREATE TABLE 1m.2g(a INT) USING parquet" - - parsePlan(sql) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("1m", "2g")) - assert(create.tableSchema == new StructType().add("a", IntegerType)) - assert(create.partitioning.isEmpty) - assert(create.bucketSpec.isEmpty) - assert(create.properties.isEmpty) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.isEmpty) - assert(create.comment.isEmpty) - assert(!create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") + test("create/replace table - byte length literal table name") { + val createSql = "CREATE TABLE 1m.2g(a INT) USING parquet" + val replaceSql = "REPLACE TABLE 1m.2g(a INT) USING parquet" + val expectedTableSpec = TableSpec( + Seq("1m", "2g"), + Some(new StructType().add("a", IntegerType)), + Seq.empty[Transform], + None, + Map.empty[String, String], + "parquet", + Map.empty[String, String], + None, + None) + Seq(createSql, replaceSql).foreach { sql => + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = false) } } - test("Duplicate clauses - create table") { + test("Duplicate clauses - create/replace table") { def createTableHeader(duplicateClause: String): String = { s"CREATE TABLE my_tab(a INT, b STRING) USING parquet $duplicateClause $duplicateClause" } + def replaceTableHeader(duplicateClause: String): String = { + s"CREATE TABLE my_tab(a INT, b STRING) USING parquet $duplicateClause $duplicateClause" + } + intercept(createTableHeader("TBLPROPERTIES('test' = 'test2')"), "Found duplicate clauses: TBLPROPERTIES") intercept(createTableHeader("LOCATION '/tmp/file'"), @@ -293,31 +279,44 @@ class DDLParserSuite extends AnalysisTest { "Found duplicate clauses: CLUSTERED BY") intercept(createTableHeader("PARTITIONED BY (b)"), "Found duplicate clauses: PARTITIONED BY") + + intercept(replaceTableHeader("TBLPROPERTIES('test' = 'test2')"), + "Found duplicate clauses: TBLPROPERTIES") + intercept(replaceTableHeader("LOCATION '/tmp/file'"), + "Found duplicate clauses: LOCATION") + intercept(replaceTableHeader("COMMENT 'a table'"), + "Found duplicate clauses: COMMENT") + intercept(replaceTableHeader("CLUSTERED BY(b) INTO 256 BUCKETS"), + "Found duplicate clauses: CLUSTERED BY") + intercept(replaceTableHeader("PARTITIONED BY (b)"), + "Found duplicate clauses: PARTITIONED BY") } test("support for other types in OPTIONS") { - val sql = + val createSql = """ |CREATE TABLE table_name USING json |OPTIONS (a 1, b 0.1, c TRUE) """.stripMargin - - parsePlan(sql) match { - case create: CreateTableStatement => - assert(create.tableName == Seq("table_name")) - assert(create.tableSchema == new StructType) - assert(create.partitioning.isEmpty) - assert(create.bucketSpec.isEmpty) - assert(create.properties.isEmpty) - assert(create.provider == "json") - assert(create.options == Map("a" -> "1", "b" -> "0.1", "c" -> "true")) - assert(create.location.isEmpty) - assert(create.comment.isEmpty) - assert(!create.ifNotExists) - - case other => - fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") + val replaceSql = + """ + |REPLACE TABLE table_name USING json + |OPTIONS (a 1, b 0.1, c TRUE) + """.stripMargin + Seq(createSql, replaceSql).foreach { sql => + testCreateOrReplaceDdl( + sql, + TableSpec( + Seq("table_name"), + Some(new StructType), + Seq.empty[Transform], + Option.empty[BucketSpec], + Map.empty[String, String], + "json", + Map("a" -> "1", "b" -> "0.1", "c" -> "true"), + None, + None), + expectedIfNotExists = false) } } @@ -352,27 +351,28 @@ class DDLParserSuite extends AnalysisTest { |AS SELECT * FROM src """.stripMargin - checkParsing(s1) - checkParsing(s2) - checkParsing(s3) - - def checkParsing(sql: String): Unit = { - parsePlan(sql) match { - case create: CreateTableAsSelectStatement => - assert(create.tableName == Seq("mydb", "page_view")) - assert(create.partitioning.isEmpty) - assert(create.bucketSpec.isEmpty) - assert(create.properties == Map("p1" -> "v1", "p2" -> "v2")) - assert(create.provider == "parquet") - assert(create.options.isEmpty) - assert(create.location.contains("/user/external/page_view")) - assert(create.comment.contains("This is the staging page view table")) - assert(create.ifNotExists) + val s4 = + """ + |REPLACE TABLE mydb.page_view + |USING parquet + |COMMENT 'This is the staging page view table' + |LOCATION '/user/external/page_view' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |AS SELECT * FROM src + """.stripMargin - case other => - fail(s"Expected to parse ${classOf[CreateTableAsSelectStatement].getClass.getName} " + - s"from query, got ${other.getClass.getName}: $sql") - } + val expectedTableSpec = TableSpec( + Seq("mydb", "page_view"), + None, + Seq.empty[Transform], + None, + Map("p1" -> "v1", "p2" -> "v2"), + "parquet", + Map.empty[String, String], + Some("/user/external/page_view"), + Some("This is the staging page view table")) + Seq(s1, s2, s3, s4).foreach { sql => + testCreateOrReplaceDdl(sql, expectedTableSpec, expectedIfNotExists = true) } } @@ -403,6 +403,28 @@ class DDLParserSuite extends AnalysisTest { parseCompare(s"DROP VIEW IF EXISTS view", DropViewStatement(Seq("view"), ifExists = true)) } + private def testCreateOrReplaceDdl( + sqlStatement: String, + tableSpec: TableSpec, + expectedIfNotExists: Boolean) { + val parsedPlan = parsePlan(sqlStatement) + val newTableToken = sqlStatement.split(" ")(0).trim.toUpperCase(Locale.ROOT) + parsedPlan match { + case create: CreateTableStatement if newTableToken == "CREATE" => + assert(create.ifNotExists == expectedIfNotExists) + case ctas: CreateTableAsSelectStatement if newTableToken == "CREATE" => + assert(ctas.ifNotExists == expectedIfNotExists) + case replace: ReplaceTableStatement if newTableToken == "REPLACE" => + case replace: ReplaceTableAsSelectStatement if newTableToken == "REPLACE" => + case other => + fail("First token in statement does not match the expected parsed plan; CREATE TABLE" + + " should create a CreateTableStatement, and REPLACE TABLE should create a" + + s" ReplaceTableStatement. Statement: $sqlStatement, plan type:" + + s" ${parsedPlan.getClass.getName}.") + } + assert(TableSpec(parsedPlan) === tableSpec) + } + // ALTER VIEW view_name SET TBLPROPERTIES ('comment' = new_comment); // ALTER VIEW view_name UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key'); test("alter view: alter view properties") { @@ -593,4 +615,69 @@ class DDLParserSuite extends AnalysisTest { Seq(Seq("x"), Seq("y"), Seq("a", "b", "c")))) } } + + private case class TableSpec( + name: Seq[String], + schema: Option[StructType], + partitioning: Seq[Transform], + bucketSpec: Option[BucketSpec], + properties: Map[String, String], + provider: String, + options: Map[String, String], + location: Option[String], + comment: Option[String]) + + private object TableSpec { + def apply(plan: LogicalPlan): TableSpec = { + plan match { + case create: CreateTableStatement => + TableSpec( + create.tableName, + Some(create.tableSchema), + create.partitioning, + create.bucketSpec, + create.properties, + create.provider, + create.options, + create.location, + create.comment) + case replace: ReplaceTableStatement => + TableSpec( + replace.tableName, + Some(replace.tableSchema), + replace.partitioning, + replace.bucketSpec, + replace.properties, + replace.provider, + replace.options, + replace.location, + replace.comment) + case ctas: CreateTableAsSelectStatement => + TableSpec( + ctas.tableName, + Some(ctas.asSelect).filter(_.resolved).map(_.schema), + ctas.partitioning, + ctas.bucketSpec, + ctas.properties, + ctas.provider, + ctas.options, + ctas.location, + ctas.comment) + case rtas: ReplaceTableAsSelectStatement => + TableSpec( + rtas.tableName, + Some(rtas.asSelect).filter(_.resolved).map(_.schema), + rtas.partitioning, + rtas.bucketSpec, + rtas.properties, + rtas.provider, + rtas.options, + rtas.location, + rtas.comment) + case other => + fail(s"Expected to parse Create, CTAS, Replace, or RTAS plan" + + s" from query, got ${other.getClass.getName}.") + } + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala index 2c491cd376edc..1a6286067a618 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DataTypeParserSuite.scala @@ -58,6 +58,7 @@ class DataTypeParserSuite extends SparkFunSuite { checkDataType("varchAr(20)", StringType) checkDataType("cHaR(27)", StringType) checkDataType("BINARY", BinaryType) + checkDataType("interval", CalendarIntervalType) checkDataType("array", ArrayType(DoubleType, true)) checkDataType("Array>", ArrayType(MapType(IntegerType, ByteType, true), true)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 5394732f41f2d..6e2a8427659c2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -84,6 +84,13 @@ trait PlanTestBase extends PredicateHelper with SQLHelper { self: Suite => } } + private def rewriteNameFromAttrNullability(plan: LogicalPlan): LogicalPlan = { + plan.transformAllExpressions { + case a @ AttributeReference(name, _, false, _) => + a.copy(name = s"*$name")(exprId = a.exprId, qualifier = a.qualifier) + } + } + /** * Normalizes plans: * - Filter the filter conditions that appear in a plan. For instance, @@ -138,7 +145,9 @@ trait PlanTestBase extends PredicateHelper with SQLHelper { self: Suite => fail( s""" |== FAIL: Plans do not match === - |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} + |${sideBySide( + rewriteNameFromAttrNullability(normalized1).treeString, + rewriteNameFromAttrNullability(normalized2).treeString).mkString("\n")} """.stripMargin) } } @@ -156,7 +165,9 @@ trait PlanTestBase extends PredicateHelper with SQLHelper { self: Suite => fail( s""" |== FAIL: Plans do not match === - |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} + |${sideBySide( + rewriteNameFromAttrNullability(normalized1).treeString, + rewriteNameFromAttrNullability(normalized2).treeString).mkString("\n")} """.stripMargin) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 4f8353922319f..8ff691fb17f27 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -31,7 +31,6 @@ import org.apache.spark.unsafe.types.UTF8String class DateTimeUtilsSuite extends SparkFunSuite { val TimeZonePST = TimeZone.getTimeZone("PST") - private def defaultTz = DateTimeUtils.defaultTimeZone() private def defaultZoneId = ZoneId.systemDefault() test("nanoseconds truncation") { @@ -366,13 +365,13 @@ class DateTimeUtilsSuite extends SparkFunSuite { test("timestamp add months") { val ts1 = date(1997, 2, 28, 10, 30, 0) val ts2 = date(2000, 2, 28, 10, 30, 0, 123000) - assert(timestampAddInterval(ts1, 36, 123000, defaultTz) === ts2) + assert(timestampAddInterval(ts1, 36, 123000, defaultZoneId) === ts2) val ts3 = date(1997, 2, 27, 16, 0, 0, 0, TimeZonePST) val ts4 = date(2000, 2, 27, 16, 0, 0, 123000, TimeZonePST) val ts5 = date(2000, 2, 28, 0, 0, 0, 123000, TimeZoneGMT) - assert(timestampAddInterval(ts3, 36, 123000, TimeZonePST) === ts4) - assert(timestampAddInterval(ts3, 36, 123000, TimeZoneGMT) === ts5) + assert(timestampAddInterval(ts3, 36, 123000, TimeZonePST.toZoneId) === ts4) + assert(timestampAddInterval(ts3, 36, 123000, TimeZoneGMT.toZoneId) === ts5) } test("monthsBetween") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala index c49308838a19e..e1fe983a42c2d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala @@ -82,4 +82,9 @@ class StructTypeSuite extends SparkFunSuite { assert(7 == schema.treeString(0).split("\n").length) assert(7 == schema.treeString(-1).split("\n").length) } + + test("interval keyword in schema string") { + val interval = "`a` INTERVAL" + assert(fromDDL(interval).toDDL === interval) + } } diff --git a/sql/core/benchmarks/MiscBenchmark-results.txt b/sql/core/benchmarks/MiscBenchmark-results.txt index 85acd57893655..c4ae052095656 100644 --- a/sql/core/benchmarks/MiscBenchmark-results.txt +++ b/sql/core/benchmarks/MiscBenchmark-results.txt @@ -105,6 +105,13 @@ generate big struct array: Best/Avg Time(ms) Rate(M/s) Per Ro generate big struct array wholestage off 708 / 776 0.1 11803.5 1.0X generate big struct array wholestage on 535 / 589 0.1 8913.9 1.3X +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +generate big nested struct array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +generate big nested struct array wholestage off 540 553 19 0.1 8997.4 1.0X +generate big nested struct array wholestage on 523 554 31 0.1 8725.0 1.0X + ================================================================================================ generate regular generator diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index c8531e9a046a4..1583b8d3a1f91 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.util.StringUtils.{PlanStringConcat, StringConcat} +import org.apache.spark.sql.catalyst.util.StringUtils.PlanStringConcat import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} @@ -60,36 +60,38 @@ class QueryExecution( lazy val analyzed: LogicalPlan = tracker.measurePhase(QueryPlanningTracker.ANALYSIS) { SparkSession.setActiveSession(sparkSession) + // We can't clone `logical` here, which will reset the `_analyzed` flag. sparkSession.sessionState.analyzer.executeAndCheck(logical, tracker) } lazy val withCachedData: LogicalPlan = { assertAnalyzed() assertSupported() - sparkSession.sharedState.cacheManager.useCachedData(analyzed) + // clone the plan to avoid sharing the plan instance between different stages like analyzing, + // optimizing and planning. + sparkSession.sharedState.cacheManager.useCachedData(analyzed.clone()) } lazy val optimizedPlan: LogicalPlan = tracker.measurePhase(QueryPlanningTracker.OPTIMIZATION) { - sparkSession.sessionState.optimizer.executeAndTrack(withCachedData, tracker) + // clone the plan to avoid sharing the plan instance between different stages like analyzing, + // optimizing and planning. + sparkSession.sessionState.optimizer.executeAndTrack(withCachedData.clone(), tracker) } lazy val sparkPlan: SparkPlan = tracker.measurePhase(QueryPlanningTracker.PLANNING) { SparkSession.setActiveSession(sparkSession) - // Runtime re-optimization requires a unique instance of every node in the logical plan. - val logicalPlan = if (sparkSession.sessionState.conf.adaptiveExecutionEnabled) { - optimizedPlan.clone() - } else { - optimizedPlan - } // TODO: We use next(), i.e. take the first plan returned by the planner, here for now, // but we will implement to choose the best plan. - planner.plan(ReturnAnswer(logicalPlan)).next() + // Clone the logical plan here, in case the planner rules change the states of the logical plan. + planner.plan(ReturnAnswer(optimizedPlan.clone())).next() } // executedPlan should not be used to initialize any SparkPlan. It should be // only used for execution. lazy val executedPlan: SparkPlan = tracker.measurePhase(QueryPlanningTracker.PLANNING) { - prepareForExecution(sparkPlan) + // clone the plan to avoid sharing the plan instance between different stages like analyzing, + // optimizing and planning. + prepareForExecution(sparkPlan.clone()) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 1de2b6e0a85de..b77f90d19b62d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -223,6 +223,13 @@ case class InMemoryRelation( statsOfPlanToCache).asInstanceOf[this.type] } + // override `clone` since the default implementation won't carry over mutable states. + override def clone(): LogicalPlan = { + val cloned = this.copy() + cloned.statsOfPlanToCache = this.statsOfPlanToCache + cloned + } + override def simpleString(maxFields: Int): String = s"InMemoryRelation [${truncatedString(output, ", ", maxFields)}], ${cacheBuilder.storageLevel}" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index 45c62b4676570..39b08e2894dcd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.internal.Logging import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.IgnoreCachedData +import org.apache.spark.sql.catalyst.plans.logical.{IgnoreCachedData, LogicalPlan} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.sql.types.{StringType, StructField, StructType} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala index 1b7bb169b36fd..8685d2f7a8566 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala @@ -27,8 +27,8 @@ import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, UnresolvedCatalogRelation} -import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan} -import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, QualifiedColType} +import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, ReplaceTable, ReplaceTableAsSelect} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, QualifiedColType, ReplaceTableAsSelectStatement, ReplaceTableStatement} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DropTableCommand} import org.apache.spark.sql.execution.datasources.v2.{CatalogTableAsV2, DataSourceV2Relation} @@ -94,6 +94,38 @@ case class DataSourceResolution( convertCTAS(v2SessionCatalog.asTableCatalog, identifier, create) } + case ReplaceTableStatement( + AsTableIdentifier(table), schema, partitionCols, bucketSpec, properties, + V1WriteProvider(provider), options, location, comment, orCreate) => + throw new AnalysisException( + s"Replacing tables is not supported using the legacy / v1 Spark external catalog" + + s" API. Write provider name: $provider, identifier: $table.") + + case ReplaceTableAsSelectStatement( + AsTableIdentifier(table), query, partitionCols, bucketSpec, properties, + V1WriteProvider(provider), options, location, comment, orCreate) => + throw new AnalysisException( + s"Replacing tables is not supported using the legacy / v1 Spark external catalog" + + s" API. Write provider name: $provider, identifier: $table.") + + case replace: ReplaceTableStatement => + // the provider was not a v1 source, convert to a v2 plan + val CatalogObjectIdentifier(maybeCatalog, identifier) = replace.tableName + val catalog = maybeCatalog.orElse(defaultCatalog) + .getOrElse(throw new AnalysisException( + s"No catalog specified for table ${identifier.quoted} and no default catalog is set")) + .asTableCatalog + convertReplaceTable(catalog, identifier, replace) + + case rtas: ReplaceTableAsSelectStatement => + // the provider was not a v1 source, convert to a v2 plan + val CatalogObjectIdentifier(maybeCatalog, identifier) = rtas.tableName + val catalog = maybeCatalog.orElse(defaultCatalog) + .getOrElse(throw new AnalysisException( + s"No catalog specified for table ${identifier.quoted} and no default catalog is set")) + .asTableCatalog + convertRTAS(catalog, identifier, rtas) + case DropTableStatement(CatalogObjectIdentifier(Some(catalog), ident), ifExists, _) => DropTable(catalog.asTableCatalog, ident, ifExists) @@ -226,6 +258,43 @@ case class DataSourceResolution( ignoreIfExists = create.ifNotExists) } + private def convertRTAS( + catalog: TableCatalog, + identifier: Identifier, + rtas: ReplaceTableAsSelectStatement): ReplaceTableAsSelect = { + // convert the bucket spec and add it as a transform + val partitioning = rtas.partitioning ++ rtas.bucketSpec.map(_.asTransform) + val properties = convertTableProperties( + rtas.properties, rtas.options, rtas.location, rtas.comment, rtas.provider) + + ReplaceTableAsSelect( + catalog, + identifier, + partitioning, + rtas.asSelect, + properties, + writeOptions = rtas.options.filterKeys(_ != "path"), + orCreate = rtas.orCreate) + } + + private def convertReplaceTable( + catalog: TableCatalog, + identifier: Identifier, + replace: ReplaceTableStatement): ReplaceTable = { + // convert the bucket spec and add it as a transform + val partitioning = replace.partitioning ++ replace.bucketSpec.map(_.asTransform) + val properties = convertTableProperties( + replace.properties, replace.options, replace.location, replace.comment, replace.provider) + + ReplaceTable( + catalog, + identifier, + replace.tableSchema, + partitioning, + properties, + orCreate = replace.orCreate) + } + private def convertTableProperties( properties: Map[String, String], options: Map[String, String], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala index f29e7869fb27c..a1de287b93f9d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala @@ -52,4 +52,10 @@ case class SaveIntoDataSourceCommand( val redacted = SQLConf.get.redactOptions(options) s"SaveIntoDataSourceCommand ${dataSource}, ${redacted}, ${mode}" } + + // Override `clone` since the default implementation will turn `CaseInsensitiveMap` to a normal + // map. + override def clone(): LogicalPlan = { + SaveIntoDataSourceCommand(query.clone(), dataSource, options, mode) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 4f8507da39240..52e2896536355 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -21,9 +21,10 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.sql.{AnalysisException, Strategy} +import org.apache.spark.sql.catalog.v2.StagingTableCatalog import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition, ReplaceTable, ReplaceTableAsSelect} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec} @@ -165,8 +166,45 @@ object DataSourceV2Strategy extends Strategy with PredicateHelper { case CreateTableAsSelect(catalog, ident, parts, query, props, options, ifNotExists) => val writeOptions = new CaseInsensitiveStringMap(options.asJava) - CreateTableAsSelectExec( - catalog, ident, parts, planLater(query), props, writeOptions, ifNotExists) :: Nil + catalog match { + case staging: StagingTableCatalog => + AtomicCreateTableAsSelectExec( + staging, ident, parts, planLater(query), props, writeOptions, ifNotExists) :: Nil + case _ => + CreateTableAsSelectExec( + catalog, ident, parts, planLater(query), props, writeOptions, ifNotExists) :: Nil + } + + case ReplaceTable(catalog, ident, schema, parts, props, orCreate) => + catalog match { + case staging: StagingTableCatalog => + AtomicReplaceTableExec(staging, ident, schema, parts, props, orCreate = orCreate) :: Nil + case _ => + ReplaceTableExec(catalog, ident, schema, parts, props, orCreate = orCreate) :: Nil + } + + case ReplaceTableAsSelect(catalog, ident, parts, query, props, options, orCreate) => + val writeOptions = new CaseInsensitiveStringMap(options.asJava) + catalog match { + case staging: StagingTableCatalog => + AtomicReplaceTableAsSelectExec( + staging, + ident, + parts, + planLater(query), + props, + writeOptions, + orCreate = orCreate) :: Nil + case _ => + ReplaceTableAsSelectExec( + catalog, + ident, + parts, + planLater(query), + props, + writeOptions, + orCreate = orCreate) :: Nil + } case AppendData(r: DataSourceV2Relation, query, _) => AppendDataExec(r.table.asWritable, r.options, planLater(query)) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala new file mode 100644 index 0000000000000..35d86ee2abbbb --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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 scala.collection.JavaConverters._ + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalog.v2.{Identifier, StagingTableCatalog, TableCatalog} +import org.apache.spark.sql.catalog.v2.expressions.Transform +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException} +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.sources.v2.StagedTable +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.Utils + +case class ReplaceTableExec( + catalog: TableCatalog, + ident: Identifier, + tableSchema: StructType, + partitioning: Seq[Transform], + tableProperties: Map[String, String], + orCreate: Boolean) extends LeafExecNode { + + override protected def doExecute(): RDD[InternalRow] = { + if (catalog.tableExists(ident)) { + catalog.dropTable(ident) + } else if (!orCreate) { + throw new CannotReplaceMissingTableException(ident) + } + catalog.createTable(ident, tableSchema, partitioning.toArray, tableProperties.asJava) + sqlContext.sparkContext.parallelize(Seq.empty, 1) + } + + override def output: Seq[Attribute] = Seq.empty +} + +case class AtomicReplaceTableExec( + catalog: StagingTableCatalog, + identifier: Identifier, + tableSchema: StructType, + partitioning: Seq[Transform], + tableProperties: Map[String, String], + orCreate: Boolean) extends LeafExecNode { + + override protected def doExecute(): RDD[InternalRow] = { + val staged = if (orCreate) { + catalog.stageCreateOrReplace( + identifier, tableSchema, partitioning.toArray, tableProperties.asJava) + } else if (catalog.tableExists(identifier)) { + try { + catalog.stageReplace( + identifier, tableSchema, partitioning.toArray, tableProperties.asJava) + } catch { + case e: NoSuchTableException => + throw new CannotReplaceMissingTableException(identifier, Some(e)) + } + } else { + throw new CannotReplaceMissingTableException(identifier) + } + commitOrAbortStagedChanges(staged) + + sqlContext.sparkContext.parallelize(Seq.empty, 1) + } + + override def output: Seq[Attribute] = Seq.empty + + private def commitOrAbortStagedChanges(staged: StagedTable): Unit = { + Utils.tryWithSafeFinallyAndFailureCallbacks({ + staged.commitStagedChanges() + })(catchBlock = { + staged.abortStagedChanges() + }) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index 6c771ea988324..9f644de1929a1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -26,15 +26,15 @@ import org.apache.spark.{SparkEnv, SparkException, TaskContext} import org.apache.spark.executor.CommitDeniedException import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} +import org.apache.spark.sql.catalog.v2.{Identifier, StagingTableCatalog, TableCatalog} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.sources.{AlwaysTrue, Filter} -import org.apache.spark.sql.sources.v2.SupportsWrite +import org.apache.spark.sql.sources.v2.{StagedTable, SupportsWrite} import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriterFactory, SupportsDynamicOverwrite, SupportsOverwrite, SupportsTruncate, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.{LongAccumulator, Utils} @@ -51,11 +51,13 @@ case class WriteToDataSourceV2(batchWrite: BatchWrite, query: LogicalPlan) } /** - * Physical plan node for v2 create table as select. + * Physical plan node for v2 create table as select when the catalog does not support staging + * the table creation. * * A new table will be created using the schema of the query, and rows from the query are appended. - * If either table creation or the append fails, the table will be deleted. This implementation does - * not provide an atomic CTAS. + * If either table creation or the append fails, the table will be deleted. This implementation is + * not atomic; for an atomic variant for catalogs that support the appropriate features, see + * CreateTableAsSelectStagingExec. */ case class CreateTableAsSelectExec( catalog: TableCatalog, @@ -78,7 +80,8 @@ case class CreateTableAsSelectExec( } Utils.tryWithSafeFinallyAndFailureCallbacks({ - catalog.createTable(ident, query.schema, partitioning.toArray, properties.asJava) match { + catalog.createTable( + ident, query.schema, partitioning.toArray, properties.asJava) match { case table: SupportsWrite => val batchWrite = table.newWriteBuilder(writeOptions) .withInputDataSchema(query.schema) @@ -89,15 +92,145 @@ case class CreateTableAsSelectExec( case _ => // table does not support writes - throw new SparkException(s"Table implementation does not support writes: ${ident.quoted}") + throw new SparkException( + s"Table implementation does not support writes: ${ident.quoted}") } + })(catchBlock = { + catalog.dropTable(ident) + }) + } +} +/** + * Physical plan node for v2 create table as select, when the catalog is determined to support + * staging table creation. + * + * A new table will be created using the schema of the query, and rows from the query are appended. + * The CTAS operation is atomic. The creation of the table is staged and the commit of the write + * should bundle the commitment of the metadata and the table contents in a single unit. If the + * write fails, the table is instructed to roll back all staged changes. + */ +case class AtomicCreateTableAsSelectExec( + catalog: StagingTableCatalog, + ident: Identifier, + partitioning: Seq[Transform], + query: SparkPlan, + properties: Map[String, String], + writeOptions: CaseInsensitiveStringMap, + ifNotExists: Boolean) extends AtomicTableWriteExec { + + override protected def doExecute(): RDD[InternalRow] = { + if (catalog.tableExists(ident)) { + if (ifNotExists) { + return sparkContext.parallelize(Seq.empty, 1) + } + + throw new TableAlreadyExistsException(ident) + } + val stagedTable = catalog.stageCreate( + ident, query.schema, partitioning.toArray, properties.asJava) + writeToStagedTable(stagedTable, writeOptions, ident) + } +} + +/** + * Physical plan node for v2 replace table as select when the catalog does not support staging + * table replacement. + * + * A new table will be created using the schema of the query, and rows from the query are appended. + * If the table exists, its contents and schema should be replaced with the schema and the contents + * of the query. This is a non-atomic implementation that drops the table and then runs non-atomic + * CTAS. For an atomic implementation for catalogs with the appropriate support, see + * ReplaceTableAsSelectStagingExec. + */ +case class ReplaceTableAsSelectExec( + catalog: TableCatalog, + ident: Identifier, + partitioning: Seq[Transform], + query: SparkPlan, + properties: Map[String, String], + writeOptions: CaseInsensitiveStringMap, + orCreate: Boolean) extends AtomicTableWriteExec { + + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.IdentifierHelper + + override protected def doExecute(): RDD[InternalRow] = { + // Note that this operation is potentially unsafe, but these are the strict semantics of + // RTAS if the catalog does not support atomic operations. + // + // There are numerous cases we concede to where the table will be dropped and irrecoverable: + // + // 1. Creating the new table fails, + // 2. Writing to the new table fails, + // 3. The table returned by catalog.createTable doesn't support writing. + if (catalog.tableExists(ident)) { + catalog.dropTable(ident) + } else if (!orCreate) { + throw new CannotReplaceMissingTableException(ident) + } + val createdTable = catalog.createTable( + ident, query.schema, partitioning.toArray, properties.asJava) + Utils.tryWithSafeFinallyAndFailureCallbacks({ + createdTable match { + case table: SupportsWrite => + val batchWrite = table.newWriteBuilder(writeOptions) + .withInputDataSchema(query.schema) + .withQueryId(UUID.randomUUID().toString) + .buildForBatch() + + doWrite(batchWrite) + + case _ => + // table does not support writes + throw new SparkException( + s"Table implementation does not support writes: ${ident.quoted}") + } })(catchBlock = { catalog.dropTable(ident) }) } } +/** + * + * Physical plan node for v2 replace table as select when the catalog supports staging + * table replacement. + * + * A new table will be created using the schema of the query, and rows from the query are appended. + * If the table exists, its contents and schema should be replaced with the schema and the contents + * of the query. This implementation is atomic. The table replacement is staged, and the commit + * operation at the end should perform tne replacement of the table's metadata and contents. If the + * write fails, the table is instructed to roll back staged changes and any previously written table + * is left untouched. + */ +case class AtomicReplaceTableAsSelectExec( + catalog: StagingTableCatalog, + ident: Identifier, + partitioning: Seq[Transform], + query: SparkPlan, + properties: Map[String, String], + writeOptions: CaseInsensitiveStringMap, + orCreate: Boolean) extends AtomicTableWriteExec { + + override protected def doExecute(): RDD[InternalRow] = { + val staged = if (orCreate) { + catalog.stageCreateOrReplace( + ident, query.schema, partitioning.toArray, properties.asJava) + } else if (catalog.tableExists(ident)) { + try { + catalog.stageReplace( + ident, query.schema, partitioning.toArray, properties.asJava) + } catch { + case e: NoSuchTableException => + throw new CannotReplaceMissingTableException(ident, Some(e)) + } + } else { + throw new CannotReplaceMissingTableException(ident) + } + writeToStagedTable(staged, writeOptions, ident) + } +} + /** * Physical plan node for append into a v2 table. * @@ -330,6 +463,36 @@ object DataWritingSparkTask extends Logging { } } +private[v2] trait AtomicTableWriteExec extends V2TableWriteExec { + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.IdentifierHelper + + protected def writeToStagedTable( + stagedTable: StagedTable, + writeOptions: CaseInsensitiveStringMap, + ident: Identifier): RDD[InternalRow] = { + Utils.tryWithSafeFinallyAndFailureCallbacks({ + stagedTable match { + case table: SupportsWrite => + val batchWrite = table.newWriteBuilder(writeOptions) + .withInputDataSchema(query.schema) + .withQueryId(UUID.randomUUID().toString) + .buildForBatch() + + val writtenRows = doWrite(batchWrite) + stagedTable.commitStagedChanges() + writtenRows + case _ => + // Table does not support writes - staged changes are also rolled back below. + throw new SparkException( + s"Table implementation does not support writes: ${ident.quoted}") + } + })(catchBlock = { + // Failure rolls back the staged writes and metadata changes. + stagedTable.abortStagedChanges() + }) + } +} + private[v2] case class DataWritingSparkTaskResult( numRows: Long, writerCommitMessage: WriterCommitMessage) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala index aede08820503b..2bdb3402c14b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala @@ -21,7 +21,6 @@ import java.util.concurrent.TimeUnit import scala.concurrent.duration.Duration -import org.apache.spark.annotation.{Evolving, Experimental} import org.apache.spark.sql.streaming.Trigger import org.apache.spark.unsafe.types.CalendarInterval @@ -47,15 +46,12 @@ private object Triggers { * A [[Trigger]] that processes only one batch of data in a streaming query then terminates * the query. */ -@Experimental -@Evolving private[sql] case object OneTimeTrigger extends Trigger /** * A [[Trigger]] that runs a query periodically based on the processing time. If `interval` is 0, * the query will run as fast as possible. */ -@Evolving private[sql] case class ProcessingTimeTrigger(intervalMs: Long) extends Trigger { Triggers.validate(intervalMs) } @@ -84,7 +80,6 @@ private[sql] object ProcessingTimeTrigger { * A [[Trigger]] that continuously processes streaming data, asynchronously checkpointing at * the specified interval. */ -@Evolving private[sql] case class ContinuousTrigger(intervalMs: Long) extends Trigger { Triggers.validate(intervalMs) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceRDD.scala index aec756c0eb2a4..14046f6a99c24 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceRDD.scala @@ -110,8 +110,9 @@ class ContinuousCoalesceRDD( context.getLocalProperty(ContinuousExecution.START_EPOCH_KEY).toLong) while (!context.isInterrupted() && !context.isCompleted()) { writer.write(prev.compute(prevSplit, context).asInstanceOf[Iterator[UnsafeRow]]) - // Note that current epoch is a non-inheritable thread local, so each writer thread - // can properly increment its own epoch without affecting the main task thread. + // Note that current epoch is a inheritable thread local but makes another instance, + // so each writer thread can properly increment its own epoch without affecting + // the main task thread. EpochTracker.incrementCurrentEpoch() } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochTracker.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochTracker.scala index bc0ae428d4521..631ae4806d2f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochTracker.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochTracker.scala @@ -26,8 +26,15 @@ import java.util.concurrent.atomic.AtomicLong object EpochTracker { // The current epoch. Note that this is a shared reference; ContinuousWriteRDD.compute() will // update the underlying AtomicLong as it finishes epochs. Other code should only read the value. - private val currentEpoch: ThreadLocal[AtomicLong] = new ThreadLocal[AtomicLong] { - override def initialValue() = new AtomicLong(-1) + private val currentEpoch: InheritableThreadLocal[AtomicLong] = { + new InheritableThreadLocal[AtomicLong] { + override protected def childValue(parent: AtomicLong): AtomicLong = { + // Note: make another instance so that changes in the parent epoch aren't reflected in + // those in the children threads. This is required at `ContinuousCoalesceRDD`. + new AtomicLong(parent.get) + } + override def initialValue() = new AtomicLong(-1) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala index 5be45c973a5f2..2645e4c9d528b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala @@ -73,14 +73,13 @@ private object PostgresDialect extends JdbcDialect { case BooleanType => Some(JdbcType("BOOLEAN", Types.BOOLEAN)) case FloatType => Some(JdbcType("FLOAT4", Types.FLOAT)) case DoubleType => Some(JdbcType("FLOAT8", Types.DOUBLE)) - case ShortType => Some(JdbcType("SMALLINT", Types.SMALLINT)) + case ShortType | ByteType => Some(JdbcType("SMALLINT", Types.SMALLINT)) case t: DecimalType => Some( JdbcType(s"NUMERIC(${t.precision},${t.scale})", java.sql.Types.NUMERIC)) case ArrayType(et, _) if et.isInstanceOf[AtomicType] => getJDBCType(et).map(_.databaseTypeDefinition) .orElse(JdbcUtils.getCommonJDBCType(et).map(_.databaseTypeDefinition)) .map(typeName => JdbcType(s"$typeName[]", java.sql.Types.ARRAY)) - case ByteType => throw new IllegalArgumentException(s"Unsupported type in postgresql: $dt"); case _ => None } diff --git a/sql/core/src/test/resources/sql-tests/inputs/cast.sql b/sql/core/src/test/resources/sql-tests/inputs/cast.sql index 7244cd31bba5e..8a035f594be54 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/cast.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/cast.sql @@ -56,3 +56,7 @@ 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 month 1 hour AS string); diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql index e6943803da399..6cd3856e8672e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql @@ -349,15 +349,15 @@ SELECT f1 - date '2000-01-01' AS `Days From 2K` FROM DATE_TBL; -- SELECT EXTRACT(MICROSEC FROM DATE 'infinity'); -- ERROR: timestamp units "microsec" not recognized -- SELECT EXTRACT(UNDEFINED FROM DATE 'infinity'); -- ERROR: timestamp units "undefined" not supported --- skip test constructors -- test constructors --- select make_date(2013, 7, 15); --- select make_date(-44, 3, 15); +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_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); diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/numeric.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/numeric.sql new file mode 100644 index 0000000000000..0fe47a4d583ef --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/numeric.sql @@ -0,0 +1,1096 @@ +-- +-- 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; +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'); +INSERT INTO num_exp_div VALUES (0,0,'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'); +INSERT INTO num_exp_div VALUES (0,1,'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'); +INSERT INTO num_exp_div VALUES (1,0,'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'); +INSERT INTO num_exp_div VALUES (1,1,'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'); +INSERT INTO num_exp_div VALUES (2,0,'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'); +INSERT INTO num_exp_div VALUES (2,1,'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'); +INSERT INTO num_exp_div VALUES (3,0,'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'); +INSERT INTO num_exp_div VALUES (3,1,'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'); +INSERT INTO num_exp_div VALUES (4,0,'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'); +INSERT INTO num_exp_div VALUES (4,1,'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'); +INSERT INTO num_exp_div VALUES (5,0,'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'); +INSERT INTO num_exp_div VALUES (5,1,'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'); +INSERT INTO num_exp_div VALUES (6,0,'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'); +INSERT INTO num_exp_div VALUES (6,1,'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'); +INSERT INTO num_exp_div VALUES (7,0,'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'); +INSERT INTO num_exp_div VALUES (7,1,'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'); +INSERT INTO num_exp_div VALUES (8,0,'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'); +INSERT INTO num_exp_div VALUES (8,1,'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'); +INSERT INTO num_exp_div VALUES (9,0,'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'); +INSERT INTO num_exp_div VALUES (9,1,'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; +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; +INSERT INTO num_exp_ln VALUES (0,'NaN'); +INSERT INTO num_exp_ln VALUES (1,'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; +INSERT INTO num_exp_log10 VALUES (0,'NaN'); +INSERT INTO num_exp_log10 VALUES (1,'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; +INSERT INTO num_exp_power_10_ln VALUES (0,'NaN'); +INSERT INTO num_exp_power_10_ln VALUES (1,'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'); +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; +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 +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 +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; +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); + +-- [SPARK-21117] Built-in SQL Function Support - WIDTH_BUCKET +-- 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(5.0::float8, 3.0::float8, 4.0::float8, 0); +-- SELECT width_bucket(5.0::float8, 3.0::float8, 4.0::float8, -5); +-- SELECT width_bucket(3.5::float8, 3.0::float8, 3.0::float8, 888); +-- SELECT width_bucket('NaN', 3.0, 4.0, 888); +-- SELECT width_bucket(0::float8, 'NaN', 4.0::float8, 888); + +-- normal operation +-- CREATE TABLE width_bucket_test (operand_num numeric, operand_f8 float8); + +-- 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; + +-- 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; + +-- for float8 only, check positive and negative infinity: we require +-- finite bucket bounds, but allow an infinite operand +-- SELECT width_bucket(0.0::float8, 'Infinity'::float8, 5, 10); -- error +-- SELECT width_bucket(0.0::float8, 5, '-Infinity'::float8, 20); -- error +-- SELECT width_bucket('Infinity'::float8, 1, 10, 10), +-- width_bucket('-Infinity'::float8, 1, 10, 10); + +-- DROP TABLE width_bucket_test; + +-- [SPARK-28137] Missing Data Type Formatting Functions: TO_CHAR +-- TO_CHAR() +-- +-- SELECT '' AS to_char_1, to_char(val, '9G999G999G999G999G999') +-- FROM num_data; + +-- SELECT '' AS to_char_2, to_char(val, '9G999G999G999G999G999D999G999G999G999G999') +-- FROM num_data; + +-- SELECT '' AS to_char_3, to_char(val, '9999999999999999.999999999999999PR') +-- FROM num_data; + +-- SELECT '' AS to_char_4, to_char(val, '9999999999999999.999999999999999S') +-- FROM num_data; + +-- SELECT '' AS to_char_5, to_char(val, 'MI9999999999999999.999999999999999') FROM num_data; +-- SELECT '' AS to_char_6, to_char(val, 'FMS9999999999999999.999999999999999') FROM num_data; +-- SELECT '' AS to_char_7, to_char(val, 'FM9999999999999999.999999999999999THPR') FROM num_data; +-- SELECT '' AS to_char_8, to_char(val, 'SG9999999999999999.999999999999999th') FROM num_data; +-- SELECT '' AS to_char_9, to_char(val, '0999999999999999.999999999999999') FROM num_data; +-- SELECT '' AS to_char_10, to_char(val, 'S0999999999999999.999999999999999') FROM num_data; +-- SELECT '' AS to_char_11, to_char(val, 'FM0999999999999999.999999999999999') FROM num_data; +-- SELECT '' AS to_char_12, to_char(val, 'FM9999999999999999.099999999999999') FROM num_data; +-- SELECT '' AS to_char_13, to_char(val, 'FM9999999999990999.990999999999999') FROM num_data; +-- SELECT '' AS to_char_14, to_char(val, 'FM0999999999999999.999909999999999') FROM num_data; +-- SELECT '' AS to_char_15, to_char(val, 'FM9999999990999999.099999999999999') FROM num_data; +-- SELECT '' AS to_char_16, to_char(val, 'L9999999999999999.099999999999999') FROM num_data; +-- SELECT '' AS to_char_17, to_char(val, 'FM9999999999999999.99999999999999') 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') 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') FROM num_data; +-- SELECT '' AS to_char_20, to_char(val, E'99999 "text" 9999 "9999" 999 "\\"text between quote marks\\"" 9999') FROM num_data; +-- SELECT '' AS to_char_21, to_char(val, '999999SG9999999999') FROM num_data; +-- SELECT '' AS to_char_22, to_char(val, 'FM9999999999999999.999999999999999') FROM num_data; +-- SELECT '' AS to_char_23, to_char(val, '9.999EEEE') 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() +-- +-- 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 +INSERT INTO num_input_test VALUES (trim(' 123')); +INSERT INTO num_input_test VALUES (trim(' 3245874 ')); +INSERT INTO num_input_test VALUES (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; +-- [SPARK-28322] DIV support decimal type +-- 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/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_implicit.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_implicit.sql new file mode 100644 index 0000000000000..54b3083a9f4a2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/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/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql new file mode 100644 index 0000000000000..02af15a879c84 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql @@ -0,0 +1,247 @@ +-- +-- 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; + +-- [SPARK-28141] Timestamp type can not accept special values +-- 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. + +-- INSERT INTO TIMESTAMP_TBL VALUES ('now'); +-- SELECT pg_sleep(0.1); + +-- BEGIN; + +-- INSERT INTO TIMESTAMP_TBL VALUES ('now'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('today'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('yesterday'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow'); +-- time zone should be ignored by this data type +-- INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow EST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('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'; +-- 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 TIMESTAMP_TBL; + +-- Special values +-- INSERT INTO TIMESTAMP_TBL VALUES ('-infinity'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('infinity'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('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 +INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02 03:04:05'); +INSERT INTO TIMESTAMP_TBL VALUES ('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 ('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'; + +-- [SPARK-28425] Add more Date/Time Operators +-- 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; + +-- [SPARK-28425] Add more Date/Time Operators +-- 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'; + +-- [SPARK-28420] Date/Time Functions: date_part +-- 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; + + +--[SPARK-28432] Missing Date/Time Functions: make_timestamp +-- timestamp numeric fields constructor +-- SELECT make_timestamp(2014,12,28,6,30,45.887); + +DROP TABLE TIMESTAMP_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_having.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_having.sql new file mode 100644 index 0000000000000..f52aa669e89db --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_having.sql @@ -0,0 +1,57 @@ +-- +-- 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/pgSQL/select_having.sql +-- TODO: We should add UDFs in GROUP BY clause when [SPARK-28445] 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)); + +-- 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/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_implicit.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_implicit.sql new file mode 100755 index 0000000000000..80edf12ef8ac5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-select_implicit.sql @@ -0,0 +1,167 @@ +-- +-- 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 pgSQL/select_implicit.sql +-- [SPARK-28445] Inconsistency between Scala and Python/Panda udfs when groupby with udf() is used +-- TODO: We should add UDFs in GROUP BY clause when [SPARK-28445] is resolved. + +-- 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 +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 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 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 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 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 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 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 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 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 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 a%2, udf(count(udf(b))) FROM test_missing_target +GROUP BY 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 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 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 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 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 (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 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 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 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/sql/core/src/test/resources/sql-tests/inputs/udf/udf-cross-join.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-cross-join.sql new file mode 100644 index 0000000000000..9aa6de7e92b8e --- /dev/null +++ b/sql/core/src/test/resources/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/sql/core/src/test/resources/sql-tests/inputs/udf/udf-except-all.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-except-all.sql new file mode 100644 index 0000000000000..71e05e67698c8 --- /dev/null +++ b/sql/core/src/test/resources/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/sql/core/src/test/resources/sql-tests/inputs/udf/udf-except.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-except.sql new file mode 100644 index 0000000000000..17e24880ea1bd --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-except.sql @@ -0,0 +1,59 @@ +-- 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 +--- [SPARK-28441] udf(max(udf(column))) throws java.lang.UnsupportedOperationException: Cannot evaluate expression: udf(null) +--- 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/sql/core/src/test/resources/sql-tests/inputs/udf/udf-group-analytics.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-group-analytics.sql new file mode 100644 index 0000000000000..618c8df977476 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-group-analytics.sql @@ -0,0 +1,64 @@ +-- This test file was converted from group-analytics.sql. +-- TODO: UDF should be inserted and tested at GROUP BY clause after SPARK-28445 +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, udf(SUM(a - b)) FROM testData GROUP BY a + b, b WITH CUBE; + +SELECT a, udf(b), SUM(b) FROM testData GROUP BY 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 a, b, udf(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 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, year; +SELECT course, udf(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, 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 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/sql/core/src/test/resources/sql-tests/inputs/udf/udf-group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-group-by.sql new file mode 100644 index 0000000000000..393de498edfa6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-group-by.sql @@ -0,0 +1,160 @@ +-- 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 1; + +-- Aggregate grouped by literals (sort aggregate). +SELECT 'foo', MAX(STRUCT(udf(a))) FROM testData WHERE a = 0 GROUP BY 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; + +-- [SPARK-28445] Inconsistency between Scala and Python/Panda udfs when groupby with udf() is used +-- The following query will make Scala UDF work, but Python and Pandas udfs will fail with an AnalysisException. +-- The query should be added after SPARK-28445. +-- SELECT udf(a + 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 a; + +-- Aliases in SELECT could be used in GROUP BY +SELECT 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 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 a, COUNT(udf(1)) FROM testData WHERE false GROUP BY 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 k, every(v), udf(some(v)), any(v) FROM test_agg GROUP BY k; + +-- having +SELECT udf(k), every(v) FROM test_agg GROUP BY k HAVING every(v) = false; +SELECT k, udf(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, + 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 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 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/sql/core/src/test/resources/sql-tests/inputs/udf/udf-inline-table.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-inline-table.sql new file mode 100644 index 0000000000000..179e528d74ca7 --- /dev/null +++ b/sql/core/src/test/resources/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/sql/core/src/test/resources/sql-tests/inputs/udf/udf-intersect-all.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-intersect-all.sql new file mode 100644 index 0000000000000..028d4c7695995 --- /dev/null +++ b/sql/core/src/test/resources/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/sql/core/src/test/resources/sql-tests/inputs/udf/udf-join-empty-relation.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-join-empty-relation.sql new file mode 100644 index 0000000000000..47fb70d02394b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-join-empty-relation.sql @@ -0,0 +1,35 @@ +-- List of configuration the test suite is run against: +--SET spark.sql.autoBroadcastJoinThreshold=10485760 +--SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +-- 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/sql/core/src/test/resources/sql-tests/inputs/udf/udf-outer-join.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-outer-join.sql new file mode 100644 index 0000000000000..4eb0805c9cc67 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-outer-join.sql @@ -0,0 +1,45 @@ +-- This test file was converted from outer-join.sql. +-- List of configuration the test suite is run against: +--SET spark.sql.autoBroadcastJoinThreshold=10485760 +--SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +-- 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/sql/core/src/test/resources/sql-tests/inputs/udf/udf-pivot.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-pivot.sql new file mode 100644 index 0000000000000..93937930de7f5 --- /dev/null +++ b/sql/core/src/test/resources/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/sql/core/src/test/resources/sql-tests/inputs/udf/udf-udaf.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-udaf.sql new file mode 100644 index 0000000000000..c22d552a0ebe6 --- /dev/null +++ b/sql/core/src/test/resources/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/sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql index 2a2b288f736ed..bcbf87f8a04c5 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql @@ -13,82 +13,82 @@ CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES 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 val ROWS CURRENT ROW) FROM testData -ORDER BY cate, val; -SELECT udf(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, udf(cate), sum(val_long) OVER(PARTITION BY cate ORDER BY val_long -ROWS BETWEEN CURRENT ROW AND CAST(2147483648 AS int) FOLLOWING) FROM testData ORDER BY cate, val_long; +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 cate ORDER BY val RANGE 1 PRECEDING) FROM testData -ORDER BY cate, val; -SELECT val, udf(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, udf(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, udf(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, udf(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, udf(cate), avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +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 cate, val_timestamp; +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 cate -ROWS BETWEEN UNBOUNDED FOLLOWING AND CAST(1 as int) FOLLOWING) FROM testData ORDER BY cate, val; -SELECT udf(val), cate, count(val) OVER(PARTITION BY cate +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 cate ORDER BY val, cate -RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; -SELECT udf(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 udf(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 udf(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; +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(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, -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, +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, +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(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 +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 cate ORDER BY val) -ORDER BY cate, val; +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; diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out index 1f8b5b6ebee79..13f72614f5778 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -25,7 +25,7 @@ select '1' year, 2 years -- !query 1 schema -struct +struct -- !query 1 output interval 1 seconds interval 2 seconds interval 1 minutes interval 2 minutes interval 1 hours interval 2 hours interval 1 days interval 2 days interval 1 months interval 2 months interval 1 years interval 2 years @@ -36,7 +36,7 @@ select interval '10' year, interval '11' month -- !query 2 schema -struct +struct -- !query 2 output interval 10 years 11 months interval 10 years interval 11 months @@ -47,7 +47,7 @@ select '10' year, '11' month -- !query 3 schema -struct +struct -- !query 3 output interval 10 years 11 months interval 10 years interval 11 months @@ -61,7 +61,7 @@ select interval '13' second, interval '13.123456789' second -- !query 4 schema -struct +struct -- !query 4 output interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 987 milliseconds 654 microseconds interval 1 weeks 3 days interval 11 hours interval 12 minutes interval 13 seconds interval 13 seconds 123 milliseconds 456 microseconds @@ -75,7 +75,7 @@ select '13' second, '13.123456789' second -- !query 5 schema -struct +struct -- !query 5 output interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 987 milliseconds 654 microseconds interval 1 weeks 3 days interval 11 hours interval 12 minutes interval 13 seconds interval 13 seconds 123 milliseconds 456 microseconds @@ -83,7 +83,7 @@ interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 987 milliseconds 654 microse -- !query 6 select map(1, interval 1 day, 2, interval 3 week) -- !query 6 schema -struct> +struct> -- !query 6 output {1:interval 1 days,2:interval 3 weeks} @@ -91,7 +91,7 @@ struct> -- !query 7 select map(1, 1 day, 2, 3 week) -- !query 7 schema -struct> +struct> -- !query 7 output {1:interval 1 days,2:interval 3 weeks} @@ -177,7 +177,7 @@ select interval '2-2' year to month - interval '3-3' year to month from interval_arithmetic -- !query 13 schema -struct<(interval 2 years 2 months + interval 3 years 3 months):calendarinterval,(interval 2 years 2 months - interval 3 years 3 months):calendarinterval> +struct<(interval 2 years 2 months + interval 3 years 3 months):interval,(interval 2 years 2 months - interval 3 years 3 months):interval> -- !query 13 output interval 5 years 5 months interval -1 years -1 months @@ -188,7 +188,7 @@ select '2-2' year to month - '3-3' year to month from interval_arithmetic -- !query 14 schema -struct<(interval 2 years 2 months + interval 3 years 3 months):calendarinterval,(interval 2 years 2 months - interval 3 years 3 months):calendarinterval> +struct<(interval 2 years 2 months + interval 3 years 3 months):interval,(interval 2 years 2 months - interval 3 years 3 months):interval> -- !query 14 output interval 5 years 5 months interval -1 years -1 months @@ -263,7 +263,7 @@ select interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second from interval_arithmetic -- !query 19 schema -struct<(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds + interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 milliseconds 456 microseconds):calendarinterval,(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds - interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 milliseconds 456 microseconds):calendarinterval> +struct<(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds + interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 milliseconds 456 microseconds):interval,(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds - interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 milliseconds 456 microseconds):interval> -- !query 19 output interval 15 weeks 4 days 20 hours 30 minutes 40 seconds 246 milliseconds 912 microseconds interval 12 weeks 5 days 2 hours 14 minutes 26 seconds @@ -274,7 +274,7 @@ select '99 11:22:33.123456789' day to second - '10 9:8:7.123456789' day to second from interval_arithmetic -- !query 20 schema -struct<(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds + interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 milliseconds 456 microseconds):calendarinterval,(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds - interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 milliseconds 456 microseconds):calendarinterval> +struct<(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds + interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 milliseconds 456 microseconds):interval,(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds - interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 milliseconds 456 microseconds):interval> -- !query 20 output interval 15 weeks 4 days 20 hours 30 minutes 40 seconds 246 milliseconds 912 microseconds interval 12 weeks 5 days 2 hours 14 minutes 26 seconds @@ -282,7 +282,7 @@ interval 15 weeks 4 days 20 hours 30 minutes 40 seconds 246 milliseconds 912 mic -- !query 21 select 30 day -- !query 21 schema -struct +struct -- !query 21 output interval 4 weeks 2 days diff --git a/sql/core/src/test/resources/sql-tests/results/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/cast.sql.out index 21b18e9b0f84f..adad21f049440 100644 --- a/sql/core/src/test/resources/sql-tests/results/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cast.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 33 +-- Number of queries: 35 -- !query 0 @@ -271,3 +271,19 @@ Extended Usage: Function: boolean Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`. + + +-- !query 33 +SELECT CAST('interval 3 month 1 hour' AS interval) +-- !query 33 schema +struct +-- !query 33 output +interval 3 months 1 hours + + +-- !query 34 +SELECT CAST(interval 3 month 1 hour AS string) +-- !query 34 schema +struct +-- !query 34 output +interval 3 months 1 hours diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index 9d3668d49d0a0..cacc44d2b1e38 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -323,7 +323,7 @@ select timestamp '2016-33-11 20:54:00.000' -- !query 34 select interval 13.123456789 seconds, interval -13.123456789 second -- !query 34 schema -struct +struct -- !query 34 output interval 13 seconds 123 milliseconds 456 microseconds interval -12 seconds -876 milliseconds -544 microseconds @@ -331,7 +331,7 @@ interval 13 seconds 123 milliseconds 456 microseconds interval -12 seconds -876 -- !query 35 select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond, 9 microsecond -- !query 35 schema -struct +struct -- !query 35 output interval 1 years 2 months 3 weeks 4 days 5 hours 6 minutes 7 seconds 8 milliseconds 9 @@ -419,6 +419,6 @@ struct<3.14:decimal(3,2),-3.14:decimal(3,2),3.14E+8:decimal(3,-6),3.14E-8:decima -- !query 43 select map(1, interval 1 day, 2, interval 3 week) -- !query 43 schema -struct> +struct> -- !query 43 output {1:interval 1 days,2:interval 3 weeks} diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out index 0d669ae7ce5b5..d4a0e07fdc72b 100644 --- a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 48 +-- Number of queries: 53 -- !query 0 @@ -508,8 +508,48 @@ struct -- !query 47 -DROP TABLE DATE_TBL +select make_date(2013, 7, 15) -- !query 47 schema -struct<> +struct -- !query 47 output +2013-07-15 + + +-- !query 48 +select make_date(-44, 3, 15) +-- !query 48 schema +struct +-- !query 48 output +0045-03-15 + + +-- !query 49 +select make_date(2013, 2, 30) +-- !query 49 schema +struct +-- !query 49 output +NULL + + +-- !query 50 +select make_date(2013, 13, 1) +-- !query 50 schema +struct +-- !query 50 output +NULL + + +-- !query 51 +select make_date(2013, 11, -1) +-- !query 51 schema +struct +-- !query 51 output +NULL + + +-- !query 52 +DROP TABLE DATE_TBL +-- !query 52 schema +struct<> +-- !query 52 output diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/numeric.sql.out new file mode 100644 index 0000000000000..04425fd4b5444 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/numeric.sql.out @@ -0,0 +1,4832 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 573 + + +-- !query 0 +CREATE TABLE num_data (id int, val decimal(38,10)) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TABLE num_exp_add (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TABLE num_exp_sub (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +CREATE TABLE num_exp_div (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +CREATE TABLE num_exp_mul (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +CREATE TABLE num_exp_sqrt (id int, expected decimal(38,10)) USING parquet +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +CREATE TABLE num_exp_ln (id int, expected decimal(38,10)) USING parquet +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +CREATE TABLE num_exp_log10 (id int, expected decimal(38,10)) USING parquet +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +CREATE TABLE num_exp_power_10_ln (id int, expected decimal(38,10)) USING parquet +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +CREATE TABLE num_result (id1 int, id2 int, result decimal(38,10)) USING parquet +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +INSERT INTO num_exp_add VALUES (0,0,'0') +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +INSERT INTO num_exp_sub VALUES (0,0,'0') +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +INSERT INTO num_exp_mul VALUES (0,0,'0') +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +INSERT INTO num_exp_div VALUES (0,0,'NaN') +-- !query 13 schema +struct<> +-- !query 13 output + + + +-- !query 14 +INSERT INTO num_exp_add VALUES (0,1,'0') +-- !query 14 schema +struct<> +-- !query 14 output + + + +-- !query 15 +INSERT INTO num_exp_sub VALUES (0,1,'0') +-- !query 15 schema +struct<> +-- !query 15 output + + + +-- !query 16 +INSERT INTO num_exp_mul VALUES (0,1,'0') +-- !query 16 schema +struct<> +-- !query 16 output + + + +-- !query 17 +INSERT INTO num_exp_div VALUES (0,1,'NaN') +-- !query 17 schema +struct<> +-- !query 17 output + + + +-- !query 18 +INSERT INTO num_exp_add VALUES (0,2,'-34338492.215397047') +-- !query 18 schema +struct<> +-- !query 18 output + + + +-- !query 19 +INSERT INTO num_exp_sub VALUES (0,2,'34338492.215397047') +-- !query 19 schema +struct<> +-- !query 19 output + + + +-- !query 20 +INSERT INTO num_exp_mul VALUES (0,2,'0') +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +INSERT INTO num_exp_div VALUES (0,2,'0') +-- !query 21 schema +struct<> +-- !query 21 output + + + +-- !query 22 +INSERT INTO num_exp_add VALUES (0,3,'4.31') +-- !query 22 schema +struct<> +-- !query 22 output + + + +-- !query 23 +INSERT INTO num_exp_sub VALUES (0,3,'-4.31') +-- !query 23 schema +struct<> +-- !query 23 output + + + +-- !query 24 +INSERT INTO num_exp_mul VALUES (0,3,'0') +-- !query 24 schema +struct<> +-- !query 24 output + + + +-- !query 25 +INSERT INTO num_exp_div VALUES (0,3,'0') +-- !query 25 schema +struct<> +-- !query 25 output + + + +-- !query 26 +INSERT INTO num_exp_add VALUES (0,4,'7799461.4119') +-- !query 26 schema +struct<> +-- !query 26 output + + + +-- !query 27 +INSERT INTO num_exp_sub VALUES (0,4,'-7799461.4119') +-- !query 27 schema +struct<> +-- !query 27 output + + + +-- !query 28 +INSERT INTO num_exp_mul VALUES (0,4,'0') +-- !query 28 schema +struct<> +-- !query 28 output + + + +-- !query 29 +INSERT INTO num_exp_div VALUES (0,4,'0') +-- !query 29 schema +struct<> +-- !query 29 output + + + +-- !query 30 +INSERT INTO num_exp_add VALUES (0,5,'16397.038491') +-- !query 30 schema +struct<> +-- !query 30 output + + + +-- !query 31 +INSERT INTO num_exp_sub VALUES (0,5,'-16397.038491') +-- !query 31 schema +struct<> +-- !query 31 output + + + +-- !query 32 +INSERT INTO num_exp_mul VALUES (0,5,'0') +-- !query 32 schema +struct<> +-- !query 32 output + + + +-- !query 33 +INSERT INTO num_exp_div VALUES (0,5,'0') +-- !query 33 schema +struct<> +-- !query 33 output + + + +-- !query 34 +INSERT INTO num_exp_add VALUES (0,6,'93901.57763026') +-- !query 34 schema +struct<> +-- !query 34 output + + + +-- !query 35 +INSERT INTO num_exp_sub VALUES (0,6,'-93901.57763026') +-- !query 35 schema +struct<> +-- !query 35 output + + + +-- !query 36 +INSERT INTO num_exp_mul VALUES (0,6,'0') +-- !query 36 schema +struct<> +-- !query 36 output + + + +-- !query 37 +INSERT INTO num_exp_div VALUES (0,6,'0') +-- !query 37 schema +struct<> +-- !query 37 output + + + +-- !query 38 +INSERT INTO num_exp_add VALUES (0,7,'-83028485') +-- !query 38 schema +struct<> +-- !query 38 output + + + +-- !query 39 +INSERT INTO num_exp_sub VALUES (0,7,'83028485') +-- !query 39 schema +struct<> +-- !query 39 output + + + +-- !query 40 +INSERT INTO num_exp_mul VALUES (0,7,'0') +-- !query 40 schema +struct<> +-- !query 40 output + + + +-- !query 41 +INSERT INTO num_exp_div VALUES (0,7,'0') +-- !query 41 schema +struct<> +-- !query 41 output + + + +-- !query 42 +INSERT INTO num_exp_add VALUES (0,8,'74881') +-- !query 42 schema +struct<> +-- !query 42 output + + + +-- !query 43 +INSERT INTO num_exp_sub VALUES (0,8,'-74881') +-- !query 43 schema +struct<> +-- !query 43 output + + + +-- !query 44 +INSERT INTO num_exp_mul VALUES (0,8,'0') +-- !query 44 schema +struct<> +-- !query 44 output + + + +-- !query 45 +INSERT INTO num_exp_div VALUES (0,8,'0') +-- !query 45 schema +struct<> +-- !query 45 output + + + +-- !query 46 +INSERT INTO num_exp_add VALUES (0,9,'-24926804.045047420') +-- !query 46 schema +struct<> +-- !query 46 output + + + +-- !query 47 +INSERT INTO num_exp_sub VALUES (0,9,'24926804.045047420') +-- !query 47 schema +struct<> +-- !query 47 output + + + +-- !query 48 +INSERT INTO num_exp_mul VALUES (0,9,'0') +-- !query 48 schema +struct<> +-- !query 48 output + + + +-- !query 49 +INSERT INTO num_exp_div VALUES (0,9,'0') +-- !query 49 schema +struct<> +-- !query 49 output + + + +-- !query 50 +INSERT INTO num_exp_add VALUES (1,0,'0') +-- !query 50 schema +struct<> +-- !query 50 output + + + +-- !query 51 +INSERT INTO num_exp_sub VALUES (1,0,'0') +-- !query 51 schema +struct<> +-- !query 51 output + + + +-- !query 52 +INSERT INTO num_exp_mul VALUES (1,0,'0') +-- !query 52 schema +struct<> +-- !query 52 output + + + +-- !query 53 +INSERT INTO num_exp_div VALUES (1,0,'NaN') +-- !query 53 schema +struct<> +-- !query 53 output + + + +-- !query 54 +INSERT INTO num_exp_add VALUES (1,1,'0') +-- !query 54 schema +struct<> +-- !query 54 output + + + +-- !query 55 +INSERT INTO num_exp_sub VALUES (1,1,'0') +-- !query 55 schema +struct<> +-- !query 55 output + + + +-- !query 56 +INSERT INTO num_exp_mul VALUES (1,1,'0') +-- !query 56 schema +struct<> +-- !query 56 output + + + +-- !query 57 +INSERT INTO num_exp_div VALUES (1,1,'NaN') +-- !query 57 schema +struct<> +-- !query 57 output + + + +-- !query 58 +INSERT INTO num_exp_add VALUES (1,2,'-34338492.215397047') +-- !query 58 schema +struct<> +-- !query 58 output + + + +-- !query 59 +INSERT INTO num_exp_sub VALUES (1,2,'34338492.215397047') +-- !query 59 schema +struct<> +-- !query 59 output + + + +-- !query 60 +INSERT INTO num_exp_mul VALUES (1,2,'0') +-- !query 60 schema +struct<> +-- !query 60 output + + + +-- !query 61 +INSERT INTO num_exp_div VALUES (1,2,'0') +-- !query 61 schema +struct<> +-- !query 61 output + + + +-- !query 62 +INSERT INTO num_exp_add VALUES (1,3,'4.31') +-- !query 62 schema +struct<> +-- !query 62 output + + + +-- !query 63 +INSERT INTO num_exp_sub VALUES (1,3,'-4.31') +-- !query 63 schema +struct<> +-- !query 63 output + + + +-- !query 64 +INSERT INTO num_exp_mul VALUES (1,3,'0') +-- !query 64 schema +struct<> +-- !query 64 output + + + +-- !query 65 +INSERT INTO num_exp_div VALUES (1,3,'0') +-- !query 65 schema +struct<> +-- !query 65 output + + + +-- !query 66 +INSERT INTO num_exp_add VALUES (1,4,'7799461.4119') +-- !query 66 schema +struct<> +-- !query 66 output + + + +-- !query 67 +INSERT INTO num_exp_sub VALUES (1,4,'-7799461.4119') +-- !query 67 schema +struct<> +-- !query 67 output + + + +-- !query 68 +INSERT INTO num_exp_mul VALUES (1,4,'0') +-- !query 68 schema +struct<> +-- !query 68 output + + + +-- !query 69 +INSERT INTO num_exp_div VALUES (1,4,'0') +-- !query 69 schema +struct<> +-- !query 69 output + + + +-- !query 70 +INSERT INTO num_exp_add VALUES (1,5,'16397.038491') +-- !query 70 schema +struct<> +-- !query 70 output + + + +-- !query 71 +INSERT INTO num_exp_sub VALUES (1,5,'-16397.038491') +-- !query 71 schema +struct<> +-- !query 71 output + + + +-- !query 72 +INSERT INTO num_exp_mul VALUES (1,5,'0') +-- !query 72 schema +struct<> +-- !query 72 output + + + +-- !query 73 +INSERT INTO num_exp_div VALUES (1,5,'0') +-- !query 73 schema +struct<> +-- !query 73 output + + + +-- !query 74 +INSERT INTO num_exp_add VALUES (1,6,'93901.57763026') +-- !query 74 schema +struct<> +-- !query 74 output + + + +-- !query 75 +INSERT INTO num_exp_sub VALUES (1,6,'-93901.57763026') +-- !query 75 schema +struct<> +-- !query 75 output + + + +-- !query 76 +INSERT INTO num_exp_mul VALUES (1,6,'0') +-- !query 76 schema +struct<> +-- !query 76 output + + + +-- !query 77 +INSERT INTO num_exp_div VALUES (1,6,'0') +-- !query 77 schema +struct<> +-- !query 77 output + + + +-- !query 78 +INSERT INTO num_exp_add VALUES (1,7,'-83028485') +-- !query 78 schema +struct<> +-- !query 78 output + + + +-- !query 79 +INSERT INTO num_exp_sub VALUES (1,7,'83028485') +-- !query 79 schema +struct<> +-- !query 79 output + + + +-- !query 80 +INSERT INTO num_exp_mul VALUES (1,7,'0') +-- !query 80 schema +struct<> +-- !query 80 output + + + +-- !query 81 +INSERT INTO num_exp_div VALUES (1,7,'0') +-- !query 81 schema +struct<> +-- !query 81 output + + + +-- !query 82 +INSERT INTO num_exp_add VALUES (1,8,'74881') +-- !query 82 schema +struct<> +-- !query 82 output + + + +-- !query 83 +INSERT INTO num_exp_sub VALUES (1,8,'-74881') +-- !query 83 schema +struct<> +-- !query 83 output + + + +-- !query 84 +INSERT INTO num_exp_mul VALUES (1,8,'0') +-- !query 84 schema +struct<> +-- !query 84 output + + + +-- !query 85 +INSERT INTO num_exp_div VALUES (1,8,'0') +-- !query 85 schema +struct<> +-- !query 85 output + + + +-- !query 86 +INSERT INTO num_exp_add VALUES (1,9,'-24926804.045047420') +-- !query 86 schema +struct<> +-- !query 86 output + + + +-- !query 87 +INSERT INTO num_exp_sub VALUES (1,9,'24926804.045047420') +-- !query 87 schema +struct<> +-- !query 87 output + + + +-- !query 88 +INSERT INTO num_exp_mul VALUES (1,9,'0') +-- !query 88 schema +struct<> +-- !query 88 output + + + +-- !query 89 +INSERT INTO num_exp_div VALUES (1,9,'0') +-- !query 89 schema +struct<> +-- !query 89 output + + + +-- !query 90 +INSERT INTO num_exp_add VALUES (2,0,'-34338492.215397047') +-- !query 90 schema +struct<> +-- !query 90 output + + + +-- !query 91 +INSERT INTO num_exp_sub VALUES (2,0,'-34338492.215397047') +-- !query 91 schema +struct<> +-- !query 91 output + + + +-- !query 92 +INSERT INTO num_exp_mul VALUES (2,0,'0') +-- !query 92 schema +struct<> +-- !query 92 output + + + +-- !query 93 +INSERT INTO num_exp_div VALUES (2,0,'NaN') +-- !query 93 schema +struct<> +-- !query 93 output + + + +-- !query 94 +INSERT INTO num_exp_add VALUES (2,1,'-34338492.215397047') +-- !query 94 schema +struct<> +-- !query 94 output + + + +-- !query 95 +INSERT INTO num_exp_sub VALUES (2,1,'-34338492.215397047') +-- !query 95 schema +struct<> +-- !query 95 output + + + +-- !query 96 +INSERT INTO num_exp_mul VALUES (2,1,'0') +-- !query 96 schema +struct<> +-- !query 96 output + + + +-- !query 97 +INSERT INTO num_exp_div VALUES (2,1,'NaN') +-- !query 97 schema +struct<> +-- !query 97 output + + + +-- !query 98 +INSERT INTO num_exp_add VALUES (2,2,'-68676984.430794094') +-- !query 98 schema +struct<> +-- !query 98 output + + + +-- !query 99 +INSERT INTO num_exp_sub VALUES (2,2,'0') +-- !query 99 schema +struct<> +-- !query 99 output + + + +-- !query 100 +INSERT INTO num_exp_mul VALUES (2,2,'1179132047626883.596862135856320209') +-- !query 100 schema +struct<> +-- !query 100 output + + + +-- !query 101 +INSERT INTO num_exp_div VALUES (2,2,'1.00000000000000000000') +-- !query 101 schema +struct<> +-- !query 101 output + + + +-- !query 102 +INSERT INTO num_exp_add VALUES (2,3,'-34338487.905397047') +-- !query 102 schema +struct<> +-- !query 102 output + + + +-- !query 103 +INSERT INTO num_exp_sub VALUES (2,3,'-34338496.525397047') +-- !query 103 schema +struct<> +-- !query 103 output + + + +-- !query 104 +INSERT INTO num_exp_mul VALUES (2,3,'-147998901.44836127257') +-- !query 104 schema +struct<> +-- !query 104 output + + + +-- !query 105 +INSERT INTO num_exp_div VALUES (2,3,'-7967167.56737750510440835266') +-- !query 105 schema +struct<> +-- !query 105 output + + + +-- !query 106 +INSERT INTO num_exp_add VALUES (2,4,'-26539030.803497047') +-- !query 106 schema +struct<> +-- !query 106 output + + + +-- !query 107 +INSERT INTO num_exp_sub VALUES (2,4,'-42137953.627297047') +-- !query 107 schema +struct<> +-- !query 107 output + + + +-- !query 108 +INSERT INTO num_exp_mul VALUES (2,4,'-267821744976817.8111137106593') +-- !query 108 schema +struct<> +-- !query 108 output + + + +-- !query 109 +INSERT INTO num_exp_div VALUES (2,4,'-4.40267480046830116685') +-- !query 109 schema +struct<> +-- !query 109 output + + + +-- !query 110 +INSERT INTO num_exp_add VALUES (2,5,'-34322095.176906047') +-- !query 110 schema +struct<> +-- !query 110 output + + + +-- !query 111 +INSERT INTO num_exp_sub VALUES (2,5,'-34354889.253888047') +-- !query 111 schema +struct<> +-- !query 111 output + + + +-- !query 112 +INSERT INTO num_exp_mul VALUES (2,5,'-563049578578.769242506736077') +-- !query 112 schema +struct<> +-- !query 112 output + + + +-- !query 113 +INSERT INTO num_exp_div VALUES (2,5,'-2094.18866914563535496429') +-- !query 113 schema +struct<> +-- !query 113 output + + + +-- !query 114 +INSERT INTO num_exp_add VALUES (2,6,'-34244590.637766787') +-- !query 114 schema +struct<> +-- !query 114 output + + + +-- !query 115 +INSERT INTO num_exp_sub VALUES (2,6,'-34432393.793027307') +-- !query 115 schema +struct<> +-- !query 115 output + + + +-- !query 116 +INSERT INTO num_exp_mul VALUES (2,6,'-3224438592470.18449811926184222') +-- !query 116 schema +struct<> +-- !query 116 output + + + +-- !query 117 +INSERT INTO num_exp_div VALUES (2,6,'-365.68599891479766440940') +-- !query 117 schema +struct<> +-- !query 117 output + + + +-- !query 118 +INSERT INTO num_exp_add VALUES (2,7,'-117366977.215397047') +-- !query 118 schema +struct<> +-- !query 118 output + + + +-- !query 119 +INSERT INTO num_exp_sub VALUES (2,7,'48689992.784602953') +-- !query 119 schema +struct<> +-- !query 119 output + + + +-- !query 120 +INSERT INTO num_exp_mul VALUES (2,7,'2851072985828710.485883795') +-- !query 120 schema +struct<> +-- !query 120 output + + + +-- !query 121 +INSERT INTO num_exp_div VALUES (2,7,'.41357483778485235518') +-- !query 121 schema +struct<> +-- !query 121 output + + + +-- !query 122 +INSERT INTO num_exp_add VALUES (2,8,'-34263611.215397047') +-- !query 122 schema +struct<> +-- !query 122 output + + + +-- !query 123 +INSERT INTO num_exp_sub VALUES (2,8,'-34413373.215397047') +-- !query 123 schema +struct<> +-- !query 123 output + + + +-- !query 124 +INSERT INTO num_exp_mul VALUES (2,8,'-2571300635581.146276407') +-- !query 124 schema +struct<> +-- !query 124 output + + + +-- !query 125 +INSERT INTO num_exp_div VALUES (2,8,'-458.57416721727870888476') +-- !query 125 schema +struct<> +-- !query 125 output + + + +-- !query 126 +INSERT INTO num_exp_add VALUES (2,9,'-59265296.260444467') +-- !query 126 schema +struct<> +-- !query 126 output + + + +-- !query 127 +INSERT INTO num_exp_sub VALUES (2,9,'-9411688.170349627') +-- !query 127 schema +struct<> +-- !query 127 output + + + +-- !query 128 +INSERT INTO num_exp_mul VALUES (2,9,'855948866655588.453741509242968740') +-- !query 128 schema +struct<> +-- !query 128 output + + + +-- !query 129 +INSERT INTO num_exp_div VALUES (2,9,'1.37757299946438931811') +-- !query 129 schema +struct<> +-- !query 129 output + + + +-- !query 130 +INSERT INTO num_exp_add VALUES (3,0,'4.31') +-- !query 130 schema +struct<> +-- !query 130 output + + + +-- !query 131 +INSERT INTO num_exp_sub VALUES (3,0,'4.31') +-- !query 131 schema +struct<> +-- !query 131 output + + + +-- !query 132 +INSERT INTO num_exp_mul VALUES (3,0,'0') +-- !query 132 schema +struct<> +-- !query 132 output + + + +-- !query 133 +INSERT INTO num_exp_div VALUES (3,0,'NaN') +-- !query 133 schema +struct<> +-- !query 133 output + + + +-- !query 134 +INSERT INTO num_exp_add VALUES (3,1,'4.31') +-- !query 134 schema +struct<> +-- !query 134 output + + + +-- !query 135 +INSERT INTO num_exp_sub VALUES (3,1,'4.31') +-- !query 135 schema +struct<> +-- !query 135 output + + + +-- !query 136 +INSERT INTO num_exp_mul VALUES (3,1,'0') +-- !query 136 schema +struct<> +-- !query 136 output + + + +-- !query 137 +INSERT INTO num_exp_div VALUES (3,1,'NaN') +-- !query 137 schema +struct<> +-- !query 137 output + + + +-- !query 138 +INSERT INTO num_exp_add VALUES (3,2,'-34338487.905397047') +-- !query 138 schema +struct<> +-- !query 138 output + + + +-- !query 139 +INSERT INTO num_exp_sub VALUES (3,2,'34338496.525397047') +-- !query 139 schema +struct<> +-- !query 139 output + + + +-- !query 140 +INSERT INTO num_exp_mul VALUES (3,2,'-147998901.44836127257') +-- !query 140 schema +struct<> +-- !query 140 output + + + +-- !query 141 +INSERT INTO num_exp_div VALUES (3,2,'-.00000012551512084352') +-- !query 141 schema +struct<> +-- !query 141 output + + + +-- !query 142 +INSERT INTO num_exp_add VALUES (3,3,'8.62') +-- !query 142 schema +struct<> +-- !query 142 output + + + +-- !query 143 +INSERT INTO num_exp_sub VALUES (3,3,'0') +-- !query 143 schema +struct<> +-- !query 143 output + + + +-- !query 144 +INSERT INTO num_exp_mul VALUES (3,3,'18.5761') +-- !query 144 schema +struct<> +-- !query 144 output + + + +-- !query 145 +INSERT INTO num_exp_div VALUES (3,3,'1.00000000000000000000') +-- !query 145 schema +struct<> +-- !query 145 output + + + +-- !query 146 +INSERT INTO num_exp_add VALUES (3,4,'7799465.7219') +-- !query 146 schema +struct<> +-- !query 146 output + + + +-- !query 147 +INSERT INTO num_exp_sub VALUES (3,4,'-7799457.1019') +-- !query 147 schema +struct<> +-- !query 147 output + + + +-- !query 148 +INSERT INTO num_exp_mul VALUES (3,4,'33615678.685289') +-- !query 148 schema +struct<> +-- !query 148 output + + + +-- !query 149 +INSERT INTO num_exp_div VALUES (3,4,'.00000055260225961552') +-- !query 149 schema +struct<> +-- !query 149 output + + + +-- !query 150 +INSERT INTO num_exp_add VALUES (3,5,'16401.348491') +-- !query 150 schema +struct<> +-- !query 150 output + + + +-- !query 151 +INSERT INTO num_exp_sub VALUES (3,5,'-16392.728491') +-- !query 151 schema +struct<> +-- !query 151 output + + + +-- !query 152 +INSERT INTO num_exp_mul VALUES (3,5,'70671.23589621') +-- !query 152 schema +struct<> +-- !query 152 output + + + +-- !query 153 +INSERT INTO num_exp_div VALUES (3,5,'.00026285234387695504') +-- !query 153 schema +struct<> +-- !query 153 output + + + +-- !query 154 +INSERT INTO num_exp_add VALUES (3,6,'93905.88763026') +-- !query 154 schema +struct<> +-- !query 154 output + + + +-- !query 155 +INSERT INTO num_exp_sub VALUES (3,6,'-93897.26763026') +-- !query 155 schema +struct<> +-- !query 155 output + + + +-- !query 156 +INSERT INTO num_exp_mul VALUES (3,6,'404715.7995864206') +-- !query 156 schema +struct<> +-- !query 156 output + + + +-- !query 157 +INSERT INTO num_exp_div VALUES (3,6,'.00004589912234457595') +-- !query 157 schema +struct<> +-- !query 157 output + + + +-- !query 158 +INSERT INTO num_exp_add VALUES (3,7,'-83028480.69') +-- !query 158 schema +struct<> +-- !query 158 output + + + +-- !query 159 +INSERT INTO num_exp_sub VALUES (3,7,'83028489.31') +-- !query 159 schema +struct<> +-- !query 159 output + + + +-- !query 160 +INSERT INTO num_exp_mul VALUES (3,7,'-357852770.35') +-- !query 160 schema +struct<> +-- !query 160 output + + + +-- !query 161 +INSERT INTO num_exp_div VALUES (3,7,'-.00000005190989574240') +-- !query 161 schema +struct<> +-- !query 161 output + + + +-- !query 162 +INSERT INTO num_exp_add VALUES (3,8,'74885.31') +-- !query 162 schema +struct<> +-- !query 162 output + + + +-- !query 163 +INSERT INTO num_exp_sub VALUES (3,8,'-74876.69') +-- !query 163 schema +struct<> +-- !query 163 output + + + +-- !query 164 +INSERT INTO num_exp_mul VALUES (3,8,'322737.11') +-- !query 164 schema +struct<> +-- !query 164 output + + + +-- !query 165 +INSERT INTO num_exp_div VALUES (3,8,'.00005755799201399553') +-- !query 165 schema +struct<> +-- !query 165 output + + + +-- !query 166 +INSERT INTO num_exp_add VALUES (3,9,'-24926799.735047420') +-- !query 166 schema +struct<> +-- !query 166 output + + + +-- !query 167 +INSERT INTO num_exp_sub VALUES (3,9,'24926808.355047420') +-- !query 167 schema +struct<> +-- !query 167 output + + + +-- !query 168 +INSERT INTO num_exp_mul VALUES (3,9,'-107434525.43415438020') +-- !query 168 schema +struct<> +-- !query 168 output + + + +-- !query 169 +INSERT INTO num_exp_div VALUES (3,9,'-.00000017290624149854') +-- !query 169 schema +struct<> +-- !query 169 output + + + +-- !query 170 +INSERT INTO num_exp_add VALUES (4,0,'7799461.4119') +-- !query 170 schema +struct<> +-- !query 170 output + + + +-- !query 171 +INSERT INTO num_exp_sub VALUES (4,0,'7799461.4119') +-- !query 171 schema +struct<> +-- !query 171 output + + + +-- !query 172 +INSERT INTO num_exp_mul VALUES (4,0,'0') +-- !query 172 schema +struct<> +-- !query 172 output + + + +-- !query 173 +INSERT INTO num_exp_div VALUES (4,0,'NaN') +-- !query 173 schema +struct<> +-- !query 173 output + + + +-- !query 174 +INSERT INTO num_exp_add VALUES (4,1,'7799461.4119') +-- !query 174 schema +struct<> +-- !query 174 output + + + +-- !query 175 +INSERT INTO num_exp_sub VALUES (4,1,'7799461.4119') +-- !query 175 schema +struct<> +-- !query 175 output + + + +-- !query 176 +INSERT INTO num_exp_mul VALUES (4,1,'0') +-- !query 176 schema +struct<> +-- !query 176 output + + + +-- !query 177 +INSERT INTO num_exp_div VALUES (4,1,'NaN') +-- !query 177 schema +struct<> +-- !query 177 output + + + +-- !query 178 +INSERT INTO num_exp_add VALUES (4,2,'-26539030.803497047') +-- !query 178 schema +struct<> +-- !query 178 output + + + +-- !query 179 +INSERT INTO num_exp_sub VALUES (4,2,'42137953.627297047') +-- !query 179 schema +struct<> +-- !query 179 output + + + +-- !query 180 +INSERT INTO num_exp_mul VALUES (4,2,'-267821744976817.8111137106593') +-- !query 180 schema +struct<> +-- !query 180 output + + + +-- !query 181 +INSERT INTO num_exp_div VALUES (4,2,'-.22713465002993920385') +-- !query 181 schema +struct<> +-- !query 181 output + + + +-- !query 182 +INSERT INTO num_exp_add VALUES (4,3,'7799465.7219') +-- !query 182 schema +struct<> +-- !query 182 output + + + +-- !query 183 +INSERT INTO num_exp_sub VALUES (4,3,'7799457.1019') +-- !query 183 schema +struct<> +-- !query 183 output + + + +-- !query 184 +INSERT INTO num_exp_mul VALUES (4,3,'33615678.685289') +-- !query 184 schema +struct<> +-- !query 184 output + + + +-- !query 185 +INSERT INTO num_exp_div VALUES (4,3,'1809619.81714617169373549883') +-- !query 185 schema +struct<> +-- !query 185 output + + + +-- !query 186 +INSERT INTO num_exp_add VALUES (4,4,'15598922.8238') +-- !query 186 schema +struct<> +-- !query 186 output + + + +-- !query 187 +INSERT INTO num_exp_sub VALUES (4,4,'0') +-- !query 187 schema +struct<> +-- !query 187 output + + + +-- !query 188 +INSERT INTO num_exp_mul VALUES (4,4,'60831598315717.14146161') +-- !query 188 schema +struct<> +-- !query 188 output + + + +-- !query 189 +INSERT INTO num_exp_div VALUES (4,4,'1.00000000000000000000') +-- !query 189 schema +struct<> +-- !query 189 output + + + +-- !query 190 +INSERT INTO num_exp_add VALUES (4,5,'7815858.450391') +-- !query 190 schema +struct<> +-- !query 190 output + + + +-- !query 191 +INSERT INTO num_exp_sub VALUES (4,5,'7783064.373409') +-- !query 191 schema +struct<> +-- !query 191 output + + + +-- !query 192 +INSERT INTO num_exp_mul VALUES (4,5,'127888068979.9935054429') +-- !query 192 schema +struct<> +-- !query 192 output + + + +-- !query 193 +INSERT INTO num_exp_div VALUES (4,5,'475.66281046305802686061') +-- !query 193 schema +struct<> +-- !query 193 output + + + +-- !query 194 +INSERT INTO num_exp_add VALUES (4,6,'7893362.98953026') +-- !query 194 schema +struct<> +-- !query 194 output + + + +-- !query 195 +INSERT INTO num_exp_sub VALUES (4,6,'7705559.83426974') +-- !query 195 schema +struct<> +-- !query 195 output + + + +-- !query 196 +INSERT INTO num_exp_mul VALUES (4,6,'732381731243.745115764094') +-- !query 196 schema +struct<> +-- !query 196 output + + + +-- !query 197 +INSERT INTO num_exp_div VALUES (4,6,'83.05996138436129499606') +-- !query 197 schema +struct<> +-- !query 197 output + + + +-- !query 198 +INSERT INTO num_exp_add VALUES (4,7,'-75229023.5881') +-- !query 198 schema +struct<> +-- !query 198 output + + + +-- !query 199 +INSERT INTO num_exp_sub VALUES (4,7,'90827946.4119') +-- !query 199 schema +struct<> +-- !query 199 output + + + +-- !query 200 +INSERT INTO num_exp_mul VALUES (4,7,'-647577464846017.9715') +-- !query 200 schema +struct<> +-- !query 200 output + + + +-- !query 201 +INSERT INTO num_exp_div VALUES (4,7,'-.09393717604145131637') +-- !query 201 schema +struct<> +-- !query 201 output + + + +-- !query 202 +INSERT INTO num_exp_add VALUES (4,8,'7874342.4119') +-- !query 202 schema +struct<> +-- !query 202 output + + + +-- !query 203 +INSERT INTO num_exp_sub VALUES (4,8,'7724580.4119') +-- !query 203 schema +struct<> +-- !query 203 output + + + +-- !query 204 +INSERT INTO num_exp_mul VALUES (4,8,'584031469984.4839') +-- !query 204 schema +struct<> +-- !query 204 output + + + +-- !query 205 +INSERT INTO num_exp_div VALUES (4,8,'104.15808298366741897143') +-- !query 205 schema +struct<> +-- !query 205 output + + + +-- !query 206 +INSERT INTO num_exp_add VALUES (4,9,'-17127342.633147420') +-- !query 206 schema +struct<> +-- !query 206 output + + + +-- !query 207 +INSERT INTO num_exp_sub VALUES (4,9,'32726265.456947420') +-- !query 207 schema +struct<> +-- !query 207 output + + + +-- !query 208 +INSERT INTO num_exp_mul VALUES (4,9,'-194415646271340.1815956522980') +-- !query 208 schema +struct<> +-- !query 208 output + + + +-- !query 209 +INSERT INTO num_exp_div VALUES (4,9,'-.31289456112403769409') +-- !query 209 schema +struct<> +-- !query 209 output + + + +-- !query 210 +INSERT INTO num_exp_add VALUES (5,0,'16397.038491') +-- !query 210 schema +struct<> +-- !query 210 output + + + +-- !query 211 +INSERT INTO num_exp_sub VALUES (5,0,'16397.038491') +-- !query 211 schema +struct<> +-- !query 211 output + + + +-- !query 212 +INSERT INTO num_exp_mul VALUES (5,0,'0') +-- !query 212 schema +struct<> +-- !query 212 output + + + +-- !query 213 +INSERT INTO num_exp_div VALUES (5,0,'NaN') +-- !query 213 schema +struct<> +-- !query 213 output + + + +-- !query 214 +INSERT INTO num_exp_add VALUES (5,1,'16397.038491') +-- !query 214 schema +struct<> +-- !query 214 output + + + +-- !query 215 +INSERT INTO num_exp_sub VALUES (5,1,'16397.038491') +-- !query 215 schema +struct<> +-- !query 215 output + + + +-- !query 216 +INSERT INTO num_exp_mul VALUES (5,1,'0') +-- !query 216 schema +struct<> +-- !query 216 output + + + +-- !query 217 +INSERT INTO num_exp_div VALUES (5,1,'NaN') +-- !query 217 schema +struct<> +-- !query 217 output + + + +-- !query 218 +INSERT INTO num_exp_add VALUES (5,2,'-34322095.176906047') +-- !query 218 schema +struct<> +-- !query 218 output + + + +-- !query 219 +INSERT INTO num_exp_sub VALUES (5,2,'34354889.253888047') +-- !query 219 schema +struct<> +-- !query 219 output + + + +-- !query 220 +INSERT INTO num_exp_mul VALUES (5,2,'-563049578578.769242506736077') +-- !query 220 schema +struct<> +-- !query 220 output + + + +-- !query 221 +INSERT INTO num_exp_div VALUES (5,2,'-.00047751189505192446') +-- !query 221 schema +struct<> +-- !query 221 output + + + +-- !query 222 +INSERT INTO num_exp_add VALUES (5,3,'16401.348491') +-- !query 222 schema +struct<> +-- !query 222 output + + + +-- !query 223 +INSERT INTO num_exp_sub VALUES (5,3,'16392.728491') +-- !query 223 schema +struct<> +-- !query 223 output + + + +-- !query 224 +INSERT INTO num_exp_mul VALUES (5,3,'70671.23589621') +-- !query 224 schema +struct<> +-- !query 224 output + + + +-- !query 225 +INSERT INTO num_exp_div VALUES (5,3,'3804.41728329466357308584') +-- !query 225 schema +struct<> +-- !query 225 output + + + +-- !query 226 +INSERT INTO num_exp_add VALUES (5,4,'7815858.450391') +-- !query 226 schema +struct<> +-- !query 226 output + + + +-- !query 227 +INSERT INTO num_exp_sub VALUES (5,4,'-7783064.373409') +-- !query 227 schema +struct<> +-- !query 227 output + + + +-- !query 228 +INSERT INTO num_exp_mul VALUES (5,4,'127888068979.9935054429') +-- !query 228 schema +struct<> +-- !query 228 output + + + +-- !query 229 +INSERT INTO num_exp_div VALUES (5,4,'.00210232958726897192') +-- !query 229 schema +struct<> +-- !query 229 output + + + +-- !query 230 +INSERT INTO num_exp_add VALUES (5,5,'32794.076982') +-- !query 230 schema +struct<> +-- !query 230 output + + + +-- !query 231 +INSERT INTO num_exp_sub VALUES (5,5,'0') +-- !query 231 schema +struct<> +-- !query 231 output + + + +-- !query 232 +INSERT INTO num_exp_mul VALUES (5,5,'268862871.275335557081') +-- !query 232 schema +struct<> +-- !query 232 output + + + +-- !query 233 +INSERT INTO num_exp_div VALUES (5,5,'1.00000000000000000000') +-- !query 233 schema +struct<> +-- !query 233 output + + + +-- !query 234 +INSERT INTO num_exp_add VALUES (5,6,'110298.61612126') +-- !query 234 schema +struct<> +-- !query 234 output + + + +-- !query 235 +INSERT INTO num_exp_sub VALUES (5,6,'-77504.53913926') +-- !query 235 schema +struct<> +-- !query 235 output + + + +-- !query 236 +INSERT INTO num_exp_mul VALUES (5,6,'1539707782.76899778633766') +-- !query 236 schema +struct<> +-- !query 236 output + + + +-- !query 237 +INSERT INTO num_exp_div VALUES (5,6,'.17461941433576102689') +-- !query 237 schema +struct<> +-- !query 237 output + + + +-- !query 238 +INSERT INTO num_exp_add VALUES (5,7,'-83012087.961509') +-- !query 238 schema +struct<> +-- !query 238 output + + + +-- !query 239 +INSERT INTO num_exp_sub VALUES (5,7,'83044882.038491') +-- !query 239 schema +struct<> +-- !query 239 output + + + +-- !query 240 +INSERT INTO num_exp_mul VALUES (5,7,'-1361421264394.416135') +-- !query 240 schema +struct<> +-- !query 240 output + + + +-- !query 241 +INSERT INTO num_exp_div VALUES (5,7,'-.00019748690453643710') +-- !query 241 schema +struct<> +-- !query 241 output + + + +-- !query 242 +INSERT INTO num_exp_add VALUES (5,8,'91278.038491') +-- !query 242 schema +struct<> +-- !query 242 output + + + +-- !query 243 +INSERT INTO num_exp_sub VALUES (5,8,'-58483.961509') +-- !query 243 schema +struct<> +-- !query 243 output + + + +-- !query 244 +INSERT INTO num_exp_mul VALUES (5,8,'1227826639.244571') +-- !query 244 schema +struct<> +-- !query 244 output + + + +-- !query 245 +INSERT INTO num_exp_div VALUES (5,8,'.21897461960978085228') +-- !query 245 schema +struct<> +-- !query 245 output + + + +-- !query 246 +INSERT INTO num_exp_add VALUES (5,9,'-24910407.006556420') +-- !query 246 schema +struct<> +-- !query 246 output + + + +-- !query 247 +INSERT INTO num_exp_sub VALUES (5,9,'24943201.083538420') +-- !query 247 schema +struct<> +-- !query 247 output + + + +-- !query 248 +INSERT INTO num_exp_mul VALUES (5,9,'-408725765384.257043660243220') +-- !query 248 schema +struct<> +-- !query 248 output + + + +-- !query 249 +INSERT INTO num_exp_div VALUES (5,9,'-.00065780749354660427') +-- !query 249 schema +struct<> +-- !query 249 output + + + +-- !query 250 +INSERT INTO num_exp_add VALUES (6,0,'93901.57763026') +-- !query 250 schema +struct<> +-- !query 250 output + + + +-- !query 251 +INSERT INTO num_exp_sub VALUES (6,0,'93901.57763026') +-- !query 251 schema +struct<> +-- !query 251 output + + + +-- !query 252 +INSERT INTO num_exp_mul VALUES (6,0,'0') +-- !query 252 schema +struct<> +-- !query 252 output + + + +-- !query 253 +INSERT INTO num_exp_div VALUES (6,0,'NaN') +-- !query 253 schema +struct<> +-- !query 253 output + + + +-- !query 254 +INSERT INTO num_exp_add VALUES (6,1,'93901.57763026') +-- !query 254 schema +struct<> +-- !query 254 output + + + +-- !query 255 +INSERT INTO num_exp_sub VALUES (6,1,'93901.57763026') +-- !query 255 schema +struct<> +-- !query 255 output + + + +-- !query 256 +INSERT INTO num_exp_mul VALUES (6,1,'0') +-- !query 256 schema +struct<> +-- !query 256 output + + + +-- !query 257 +INSERT INTO num_exp_div VALUES (6,1,'NaN') +-- !query 257 schema +struct<> +-- !query 257 output + + + +-- !query 258 +INSERT INTO num_exp_add VALUES (6,2,'-34244590.637766787') +-- !query 258 schema +struct<> +-- !query 258 output + + + +-- !query 259 +INSERT INTO num_exp_sub VALUES (6,2,'34432393.793027307') +-- !query 259 schema +struct<> +-- !query 259 output + + + +-- !query 260 +INSERT INTO num_exp_mul VALUES (6,2,'-3224438592470.18449811926184222') +-- !query 260 schema +struct<> +-- !query 260 output + + + +-- !query 261 +INSERT INTO num_exp_div VALUES (6,2,'-.00273458651128995823') +-- !query 261 schema +struct<> +-- !query 261 output + + + +-- !query 262 +INSERT INTO num_exp_add VALUES (6,3,'93905.88763026') +-- !query 262 schema +struct<> +-- !query 262 output + + + +-- !query 263 +INSERT INTO num_exp_sub VALUES (6,3,'93897.26763026') +-- !query 263 schema +struct<> +-- !query 263 output + + + +-- !query 264 +INSERT INTO num_exp_mul VALUES (6,3,'404715.7995864206') +-- !query 264 schema +struct<> +-- !query 264 output + + + +-- !query 265 +INSERT INTO num_exp_div VALUES (6,3,'21786.90896293735498839907') +-- !query 265 schema +struct<> +-- !query 265 output + + + +-- !query 266 +INSERT INTO num_exp_add VALUES (6,4,'7893362.98953026') +-- !query 266 schema +struct<> +-- !query 266 output + + + +-- !query 267 +INSERT INTO num_exp_sub VALUES (6,4,'-7705559.83426974') +-- !query 267 schema +struct<> +-- !query 267 output + + + +-- !query 268 +INSERT INTO num_exp_mul VALUES (6,4,'732381731243.745115764094') +-- !query 268 schema +struct<> +-- !query 268 output + + + +-- !query 269 +INSERT INTO num_exp_div VALUES (6,4,'.01203949512295682469') +-- !query 269 schema +struct<> +-- !query 269 output + + + +-- !query 270 +INSERT INTO num_exp_add VALUES (6,5,'110298.61612126') +-- !query 270 schema +struct<> +-- !query 270 output + + + +-- !query 271 +INSERT INTO num_exp_sub VALUES (6,5,'77504.53913926') +-- !query 271 schema +struct<> +-- !query 271 output + + + +-- !query 272 +INSERT INTO num_exp_mul VALUES (6,5,'1539707782.76899778633766') +-- !query 272 schema +struct<> +-- !query 272 output + + + +-- !query 273 +INSERT INTO num_exp_div VALUES (6,5,'5.72674008674192359679') +-- !query 273 schema +struct<> +-- !query 273 output + + + +-- !query 274 +INSERT INTO num_exp_add VALUES (6,6,'187803.15526052') +-- !query 274 schema +struct<> +-- !query 274 output + + + +-- !query 275 +INSERT INTO num_exp_sub VALUES (6,6,'0') +-- !query 275 schema +struct<> +-- !query 275 output + + + +-- !query 276 +INSERT INTO num_exp_mul VALUES (6,6,'8817506281.4517452372676676') +-- !query 276 schema +struct<> +-- !query 276 output + + + +-- !query 277 +INSERT INTO num_exp_div VALUES (6,6,'1.00000000000000000000') +-- !query 277 schema +struct<> +-- !query 277 output + + + +-- !query 278 +INSERT INTO num_exp_add VALUES (6,7,'-82934583.42236974') +-- !query 278 schema +struct<> +-- !query 278 output + + + +-- !query 279 +INSERT INTO num_exp_sub VALUES (6,7,'83122386.57763026') +-- !query 279 schema +struct<> +-- !query 279 output + + + +-- !query 280 +INSERT INTO num_exp_mul VALUES (6,7,'-7796505729750.37795610') +-- !query 280 schema +struct<> +-- !query 280 output + + + +-- !query 281 +INSERT INTO num_exp_div VALUES (6,7,'-.00113095617281538980') +-- !query 281 schema +struct<> +-- !query 281 output + + + +-- !query 282 +INSERT INTO num_exp_add VALUES (6,8,'168782.57763026') +-- !query 282 schema +struct<> +-- !query 282 output + + + +-- !query 283 +INSERT INTO num_exp_sub VALUES (6,8,'19020.57763026') +-- !query 283 schema +struct<> +-- !query 283 output + + + +-- !query 284 +INSERT INTO num_exp_mul VALUES (6,8,'7031444034.53149906') +-- !query 284 schema +struct<> +-- !query 284 output + + + +-- !query 285 +INSERT INTO num_exp_div VALUES (6,8,'1.25401073209839612184') +-- !query 285 schema +struct<> +-- !query 285 output + + + +-- !query 286 +INSERT INTO num_exp_add VALUES (6,9,'-24832902.467417160') +-- !query 286 schema +struct<> +-- !query 286 output + + + +-- !query 287 +INSERT INTO num_exp_sub VALUES (6,9,'25020705.622677680') +-- !query 287 schema +struct<> +-- !query 287 output + + + +-- !query 288 +INSERT INTO num_exp_mul VALUES (6,9,'-2340666225110.29929521292692920') +-- !query 288 schema +struct<> +-- !query 288 output + + + +-- !query 289 +INSERT INTO num_exp_div VALUES (6,9,'-.00376709254265256789') +-- !query 289 schema +struct<> +-- !query 289 output + + + +-- !query 290 +INSERT INTO num_exp_add VALUES (7,0,'-83028485') +-- !query 290 schema +struct<> +-- !query 290 output + + + +-- !query 291 +INSERT INTO num_exp_sub VALUES (7,0,'-83028485') +-- !query 291 schema +struct<> +-- !query 291 output + + + +-- !query 292 +INSERT INTO num_exp_mul VALUES (7,0,'0') +-- !query 292 schema +struct<> +-- !query 292 output + + + +-- !query 293 +INSERT INTO num_exp_div VALUES (7,0,'NaN') +-- !query 293 schema +struct<> +-- !query 293 output + + + +-- !query 294 +INSERT INTO num_exp_add VALUES (7,1,'-83028485') +-- !query 294 schema +struct<> +-- !query 294 output + + + +-- !query 295 +INSERT INTO num_exp_sub VALUES (7,1,'-83028485') +-- !query 295 schema +struct<> +-- !query 295 output + + + +-- !query 296 +INSERT INTO num_exp_mul VALUES (7,1,'0') +-- !query 296 schema +struct<> +-- !query 296 output + + + +-- !query 297 +INSERT INTO num_exp_div VALUES (7,1,'NaN') +-- !query 297 schema +struct<> +-- !query 297 output + + + +-- !query 298 +INSERT INTO num_exp_add VALUES (7,2,'-117366977.215397047') +-- !query 298 schema +struct<> +-- !query 298 output + + + +-- !query 299 +INSERT INTO num_exp_sub VALUES (7,2,'-48689992.784602953') +-- !query 299 schema +struct<> +-- !query 299 output + + + +-- !query 300 +INSERT INTO num_exp_mul VALUES (7,2,'2851072985828710.485883795') +-- !query 300 schema +struct<> +-- !query 300 output + + + +-- !query 301 +INSERT INTO num_exp_div VALUES (7,2,'2.41794207151503385700') +-- !query 301 schema +struct<> +-- !query 301 output + + + +-- !query 302 +INSERT INTO num_exp_add VALUES (7,3,'-83028480.69') +-- !query 302 schema +struct<> +-- !query 302 output + + + +-- !query 303 +INSERT INTO num_exp_sub VALUES (7,3,'-83028489.31') +-- !query 303 schema +struct<> +-- !query 303 output + + + +-- !query 304 +INSERT INTO num_exp_mul VALUES (7,3,'-357852770.35') +-- !query 304 schema +struct<> +-- !query 304 output + + + +-- !query 305 +INSERT INTO num_exp_div VALUES (7,3,'-19264149.65197215777262180974') +-- !query 305 schema +struct<> +-- !query 305 output + + + +-- !query 306 +INSERT INTO num_exp_add VALUES (7,4,'-75229023.5881') +-- !query 306 schema +struct<> +-- !query 306 output + + + +-- !query 307 +INSERT INTO num_exp_sub VALUES (7,4,'-90827946.4119') +-- !query 307 schema +struct<> +-- !query 307 output + + + +-- !query 308 +INSERT INTO num_exp_mul VALUES (7,4,'-647577464846017.9715') +-- !query 308 schema +struct<> +-- !query 308 output + + + +-- !query 309 +INSERT INTO num_exp_div VALUES (7,4,'-10.64541262725136247686') +-- !query 309 schema +struct<> +-- !query 309 output + + + +-- !query 310 +INSERT INTO num_exp_add VALUES (7,5,'-83012087.961509') +-- !query 310 schema +struct<> +-- !query 310 output + + + +-- !query 311 +INSERT INTO num_exp_sub VALUES (7,5,'-83044882.038491') +-- !query 311 schema +struct<> +-- !query 311 output + + + +-- !query 312 +INSERT INTO num_exp_mul VALUES (7,5,'-1361421264394.416135') +-- !query 312 schema +struct<> +-- !query 312 output + + + +-- !query 313 +INSERT INTO num_exp_div VALUES (7,5,'-5063.62688881730941836574') +-- !query 313 schema +struct<> +-- !query 313 output + + + +-- !query 314 +INSERT INTO num_exp_add VALUES (7,6,'-82934583.42236974') +-- !query 314 schema +struct<> +-- !query 314 output + + + +-- !query 315 +INSERT INTO num_exp_sub VALUES (7,6,'-83122386.57763026') +-- !query 315 schema +struct<> +-- !query 315 output + + + +-- !query 316 +INSERT INTO num_exp_mul VALUES (7,6,'-7796505729750.37795610') +-- !query 316 schema +struct<> +-- !query 316 output + + + +-- !query 317 +INSERT INTO num_exp_div VALUES (7,6,'-884.20756174009028770294') +-- !query 317 schema +struct<> +-- !query 317 output + + + +-- !query 318 +INSERT INTO num_exp_add VALUES (7,7,'-166056970') +-- !query 318 schema +struct<> +-- !query 318 output + + + +-- !query 319 +INSERT INTO num_exp_sub VALUES (7,7,'0') +-- !query 319 schema +struct<> +-- !query 319 output + + + +-- !query 320 +INSERT INTO num_exp_mul VALUES (7,7,'6893729321395225') +-- !query 320 schema +struct<> +-- !query 320 output + + + +-- !query 321 +INSERT INTO num_exp_div VALUES (7,7,'1.00000000000000000000') +-- !query 321 schema +struct<> +-- !query 321 output + + + +-- !query 322 +INSERT INTO num_exp_add VALUES (7,8,'-82953604') +-- !query 322 schema +struct<> +-- !query 322 output + + + +-- !query 323 +INSERT INTO num_exp_sub VALUES (7,8,'-83103366') +-- !query 323 schema +struct<> +-- !query 323 output + + + +-- !query 324 +INSERT INTO num_exp_mul VALUES (7,8,'-6217255985285') +-- !query 324 schema +struct<> +-- !query 324 output + + + +-- !query 325 +INSERT INTO num_exp_div VALUES (7,8,'-1108.80577182462841041118') +-- !query 325 schema +struct<> +-- !query 325 output + + + +-- !query 326 +INSERT INTO num_exp_add VALUES (7,9,'-107955289.045047420') +-- !query 326 schema +struct<> +-- !query 326 output + + + +-- !query 327 +INSERT INTO num_exp_sub VALUES (7,9,'-58101680.954952580') +-- !query 327 schema +struct<> +-- !query 327 output + + + +-- !query 328 +INSERT INTO num_exp_mul VALUES (7,9,'2069634775752159.035758700') +-- !query 328 schema +struct<> +-- !query 328 output + + + +-- !query 329 +INSERT INTO num_exp_div VALUES (7,9,'3.33089171198810413382') +-- !query 329 schema +struct<> +-- !query 329 output + + + +-- !query 330 +INSERT INTO num_exp_add VALUES (8,0,'74881') +-- !query 330 schema +struct<> +-- !query 330 output + + + +-- !query 331 +INSERT INTO num_exp_sub VALUES (8,0,'74881') +-- !query 331 schema +struct<> +-- !query 331 output + + + +-- !query 332 +INSERT INTO num_exp_mul VALUES (8,0,'0') +-- !query 332 schema +struct<> +-- !query 332 output + + + +-- !query 333 +INSERT INTO num_exp_div VALUES (8,0,'NaN') +-- !query 333 schema +struct<> +-- !query 333 output + + + +-- !query 334 +INSERT INTO num_exp_add VALUES (8,1,'74881') +-- !query 334 schema +struct<> +-- !query 334 output + + + +-- !query 335 +INSERT INTO num_exp_sub VALUES (8,1,'74881') +-- !query 335 schema +struct<> +-- !query 335 output + + + +-- !query 336 +INSERT INTO num_exp_mul VALUES (8,1,'0') +-- !query 336 schema +struct<> +-- !query 336 output + + + +-- !query 337 +INSERT INTO num_exp_div VALUES (8,1,'NaN') +-- !query 337 schema +struct<> +-- !query 337 output + + + +-- !query 338 +INSERT INTO num_exp_add VALUES (8,2,'-34263611.215397047') +-- !query 338 schema +struct<> +-- !query 338 output + + + +-- !query 339 +INSERT INTO num_exp_sub VALUES (8,2,'34413373.215397047') +-- !query 339 schema +struct<> +-- !query 339 output + + + +-- !query 340 +INSERT INTO num_exp_mul VALUES (8,2,'-2571300635581.146276407') +-- !query 340 schema +struct<> +-- !query 340 output + + + +-- !query 341 +INSERT INTO num_exp_div VALUES (8,2,'-.00218067233500788615') +-- !query 341 schema +struct<> +-- !query 341 output + + + +-- !query 342 +INSERT INTO num_exp_add VALUES (8,3,'74885.31') +-- !query 342 schema +struct<> +-- !query 342 output + + + +-- !query 343 +INSERT INTO num_exp_sub VALUES (8,3,'74876.69') +-- !query 343 schema +struct<> +-- !query 343 output + + + +-- !query 344 +INSERT INTO num_exp_mul VALUES (8,3,'322737.11') +-- !query 344 schema +struct<> +-- !query 344 output + + + +-- !query 345 +INSERT INTO num_exp_div VALUES (8,3,'17373.78190255220417633410') +-- !query 345 schema +struct<> +-- !query 345 output + + + +-- !query 346 +INSERT INTO num_exp_add VALUES (8,4,'7874342.4119') +-- !query 346 schema +struct<> +-- !query 346 output + + + +-- !query 347 +INSERT INTO num_exp_sub VALUES (8,4,'-7724580.4119') +-- !query 347 schema +struct<> +-- !query 347 output + + + +-- !query 348 +INSERT INTO num_exp_mul VALUES (8,4,'584031469984.4839') +-- !query 348 schema +struct<> +-- !query 348 output + + + +-- !query 349 +INSERT INTO num_exp_div VALUES (8,4,'.00960079113741758956') +-- !query 349 schema +struct<> +-- !query 349 output + + + +-- !query 350 +INSERT INTO num_exp_add VALUES (8,5,'91278.038491') +-- !query 350 schema +struct<> +-- !query 350 output + + + +-- !query 351 +INSERT INTO num_exp_sub VALUES (8,5,'58483.961509') +-- !query 351 schema +struct<> +-- !query 351 output + + + +-- !query 352 +INSERT INTO num_exp_mul VALUES (8,5,'1227826639.244571') +-- !query 352 schema +struct<> +-- !query 352 output + + + +-- !query 353 +INSERT INTO num_exp_div VALUES (8,5,'4.56673929509287019456') +-- !query 353 schema +struct<> +-- !query 353 output + + + +-- !query 354 +INSERT INTO num_exp_add VALUES (8,6,'168782.57763026') +-- !query 354 schema +struct<> +-- !query 354 output + + + +-- !query 355 +INSERT INTO num_exp_sub VALUES (8,6,'-19020.57763026') +-- !query 355 schema +struct<> +-- !query 355 output + + + +-- !query 356 +INSERT INTO num_exp_mul VALUES (8,6,'7031444034.53149906') +-- !query 356 schema +struct<> +-- !query 356 output + + + +-- !query 357 +INSERT INTO num_exp_div VALUES (8,6,'.79744134113322314424') +-- !query 357 schema +struct<> +-- !query 357 output + + + +-- !query 358 +INSERT INTO num_exp_add VALUES (8,7,'-82953604') +-- !query 358 schema +struct<> +-- !query 358 output + + + +-- !query 359 +INSERT INTO num_exp_sub VALUES (8,7,'83103366') +-- !query 359 schema +struct<> +-- !query 359 output + + + +-- !query 360 +INSERT INTO num_exp_mul VALUES (8,7,'-6217255985285') +-- !query 360 schema +struct<> +-- !query 360 output + + + +-- !query 361 +INSERT INTO num_exp_div VALUES (8,7,'-.00090187120721280172') +-- !query 361 schema +struct<> +-- !query 361 output + + + +-- !query 362 +INSERT INTO num_exp_add VALUES (8,8,'149762') +-- !query 362 schema +struct<> +-- !query 362 output + + + +-- !query 363 +INSERT INTO num_exp_sub VALUES (8,8,'0') +-- !query 363 schema +struct<> +-- !query 363 output + + + +-- !query 364 +INSERT INTO num_exp_mul VALUES (8,8,'5607164161') +-- !query 364 schema +struct<> +-- !query 364 output + + + +-- !query 365 +INSERT INTO num_exp_div VALUES (8,8,'1.00000000000000000000') +-- !query 365 schema +struct<> +-- !query 365 output + + + +-- !query 366 +INSERT INTO num_exp_add VALUES (8,9,'-24851923.045047420') +-- !query 366 schema +struct<> +-- !query 366 output + + + +-- !query 367 +INSERT INTO num_exp_sub VALUES (8,9,'25001685.045047420') +-- !query 367 schema +struct<> +-- !query 367 output + + + +-- !query 368 +INSERT INTO num_exp_mul VALUES (8,9,'-1866544013697.195857020') +-- !query 368 schema +struct<> +-- !query 368 output + + + +-- !query 369 +INSERT INTO num_exp_div VALUES (8,9,'-.00300403532938582735') +-- !query 369 schema +struct<> +-- !query 369 output + + + +-- !query 370 +INSERT INTO num_exp_add VALUES (9,0,'-24926804.045047420') +-- !query 370 schema +struct<> +-- !query 370 output + + + +-- !query 371 +INSERT INTO num_exp_sub VALUES (9,0,'-24926804.045047420') +-- !query 371 schema +struct<> +-- !query 371 output + + + +-- !query 372 +INSERT INTO num_exp_mul VALUES (9,0,'0') +-- !query 372 schema +struct<> +-- !query 372 output + + + +-- !query 373 +INSERT INTO num_exp_div VALUES (9,0,'NaN') +-- !query 373 schema +struct<> +-- !query 373 output + + + +-- !query 374 +INSERT INTO num_exp_add VALUES (9,1,'-24926804.045047420') +-- !query 374 schema +struct<> +-- !query 374 output + + + +-- !query 375 +INSERT INTO num_exp_sub VALUES (9,1,'-24926804.045047420') +-- !query 375 schema +struct<> +-- !query 375 output + + + +-- !query 376 +INSERT INTO num_exp_mul VALUES (9,1,'0') +-- !query 376 schema +struct<> +-- !query 376 output + + + +-- !query 377 +INSERT INTO num_exp_div VALUES (9,1,'NaN') +-- !query 377 schema +struct<> +-- !query 377 output + + + +-- !query 378 +INSERT INTO num_exp_add VALUES (9,2,'-59265296.260444467') +-- !query 378 schema +struct<> +-- !query 378 output + + + +-- !query 379 +INSERT INTO num_exp_sub VALUES (9,2,'9411688.170349627') +-- !query 379 schema +struct<> +-- !query 379 output + + + +-- !query 380 +INSERT INTO num_exp_mul VALUES (9,2,'855948866655588.453741509242968740') +-- !query 380 schema +struct<> +-- !query 380 output + + + +-- !query 381 +INSERT INTO num_exp_div VALUES (9,2,'.72591434384152961526') +-- !query 381 schema +struct<> +-- !query 381 output + + + +-- !query 382 +INSERT INTO num_exp_add VALUES (9,3,'-24926799.735047420') +-- !query 382 schema +struct<> +-- !query 382 output + + + +-- !query 383 +INSERT INTO num_exp_sub VALUES (9,3,'-24926808.355047420') +-- !query 383 schema +struct<> +-- !query 383 output + + + +-- !query 384 +INSERT INTO num_exp_mul VALUES (9,3,'-107434525.43415438020') +-- !query 384 schema +struct<> +-- !query 384 output + + + +-- !query 385 +INSERT INTO num_exp_div VALUES (9,3,'-5783481.21694835730858468677') +-- !query 385 schema +struct<> +-- !query 385 output + + + +-- !query 386 +INSERT INTO num_exp_add VALUES (9,4,'-17127342.633147420') +-- !query 386 schema +struct<> +-- !query 386 output + + + +-- !query 387 +INSERT INTO num_exp_sub VALUES (9,4,'-32726265.456947420') +-- !query 387 schema +struct<> +-- !query 387 output + + + +-- !query 388 +INSERT INTO num_exp_mul VALUES (9,4,'-194415646271340.1815956522980') +-- !query 388 schema +struct<> +-- !query 388 output + + + +-- !query 389 +INSERT INTO num_exp_div VALUES (9,4,'-3.19596478892958416484') +-- !query 389 schema +struct<> +-- !query 389 output + + + +-- !query 390 +INSERT INTO num_exp_add VALUES (9,5,'-24910407.006556420') +-- !query 390 schema +struct<> +-- !query 390 output + + + +-- !query 391 +INSERT INTO num_exp_sub VALUES (9,5,'-24943201.083538420') +-- !query 391 schema +struct<> +-- !query 391 output + + + +-- !query 392 +INSERT INTO num_exp_mul VALUES (9,5,'-408725765384.257043660243220') +-- !query 392 schema +struct<> +-- !query 392 output + + + +-- !query 393 +INSERT INTO num_exp_div VALUES (9,5,'-1520.20159364322004505807') +-- !query 393 schema +struct<> +-- !query 393 output + + + +-- !query 394 +INSERT INTO num_exp_add VALUES (9,6,'-24832902.467417160') +-- !query 394 schema +struct<> +-- !query 394 output + + + +-- !query 395 +INSERT INTO num_exp_sub VALUES (9,6,'-25020705.622677680') +-- !query 395 schema +struct<> +-- !query 395 output + + + +-- !query 396 +INSERT INTO num_exp_mul VALUES (9,6,'-2340666225110.29929521292692920') +-- !query 396 schema +struct<> +-- !query 396 output + + + +-- !query 397 +INSERT INTO num_exp_div VALUES (9,6,'-265.45671195426965751280') +-- !query 397 schema +struct<> +-- !query 397 output + + + +-- !query 398 +INSERT INTO num_exp_add VALUES (9,7,'-107955289.045047420') +-- !query 398 schema +struct<> +-- !query 398 output + + + +-- !query 399 +INSERT INTO num_exp_sub VALUES (9,7,'58101680.954952580') +-- !query 399 schema +struct<> +-- !query 399 output + + + +-- !query 400 +INSERT INTO num_exp_mul VALUES (9,7,'2069634775752159.035758700') +-- !query 400 schema +struct<> +-- !query 400 output + + + +-- !query 401 +INSERT INTO num_exp_div VALUES (9,7,'.30021990699995814689') +-- !query 401 schema +struct<> +-- !query 401 output + + + +-- !query 402 +INSERT INTO num_exp_add VALUES (9,8,'-24851923.045047420') +-- !query 402 schema +struct<> +-- !query 402 output + + + +-- !query 403 +INSERT INTO num_exp_sub VALUES (9,8,'-25001685.045047420') +-- !query 403 schema +struct<> +-- !query 403 output + + + +-- !query 404 +INSERT INTO num_exp_mul VALUES (9,8,'-1866544013697.195857020') +-- !query 404 schema +struct<> +-- !query 404 output + + + +-- !query 405 +INSERT INTO num_exp_div VALUES (9,8,'-332.88556569820675471748') +-- !query 405 schema +struct<> +-- !query 405 output + + + +-- !query 406 +INSERT INTO num_exp_add VALUES (9,9,'-49853608.090094840') +-- !query 406 schema +struct<> +-- !query 406 output + + + +-- !query 407 +INSERT INTO num_exp_sub VALUES (9,9,'0') +-- !query 407 schema +struct<> +-- !query 407 output + + + +-- !query 408 +INSERT INTO num_exp_mul VALUES (9,9,'621345559900192.420120630048656400') +-- !query 408 schema +struct<> +-- !query 408 output + + + +-- !query 409 +INSERT INTO num_exp_div VALUES (9,9,'1.00000000000000000000') +-- !query 409 schema +struct<> +-- !query 409 output + + + +-- !query 410 +INSERT INTO num_exp_sqrt VALUES (0,'0') +-- !query 410 schema +struct<> +-- !query 410 output + + + +-- !query 411 +INSERT INTO num_exp_sqrt VALUES (1,'0') +-- !query 411 schema +struct<> +-- !query 411 output + + + +-- !query 412 +INSERT INTO num_exp_sqrt VALUES (2,'5859.90547836712524903505') +-- !query 412 schema +struct<> +-- !query 412 output + + + +-- !query 413 +INSERT INTO num_exp_sqrt VALUES (3,'2.07605394920266944396') +-- !query 413 schema +struct<> +-- !query 413 output + + + +-- !query 414 +INSERT INTO num_exp_sqrt VALUES (4,'2792.75158435189147418923') +-- !query 414 schema +struct<> +-- !query 414 output + + + +-- !query 415 +INSERT INTO num_exp_sqrt VALUES (5,'128.05092147657509145473') +-- !query 415 schema +struct<> +-- !query 415 output + + + +-- !query 416 +INSERT INTO num_exp_sqrt VALUES (6,'306.43364311096782703406') +-- !query 416 schema +struct<> +-- !query 416 output + + + +-- !query 417 +INSERT INTO num_exp_sqrt VALUES (7,'9111.99676251039939975230') +-- !query 417 schema +struct<> +-- !query 417 output + + + +-- !query 418 +INSERT INTO num_exp_sqrt VALUES (8,'273.64392922189960397542') +-- !query 418 schema +struct<> +-- !query 418 output + + + +-- !query 419 +INSERT INTO num_exp_sqrt VALUES (9,'4992.67503899937593364766') +-- !query 419 schema +struct<> +-- !query 419 output + + + +-- !query 420 +INSERT INTO num_exp_ln VALUES (0,'NaN') +-- !query 420 schema +struct<> +-- !query 420 output + + + +-- !query 421 +INSERT INTO num_exp_ln VALUES (1,'NaN') +-- !query 421 schema +struct<> +-- !query 421 output + + + +-- !query 422 +INSERT INTO num_exp_ln VALUES (2,'17.35177750493897715514') +-- !query 422 schema +struct<> +-- !query 422 output + + + +-- !query 423 +INSERT INTO num_exp_ln VALUES (3,'1.46093790411565641971') +-- !query 423 schema +struct<> +-- !query 423 output + + + +-- !query 424 +INSERT INTO num_exp_ln VALUES (4,'15.86956523951936572464') +-- !query 424 schema +struct<> +-- !query 424 output + + + +-- !query 425 +INSERT INTO num_exp_ln VALUES (5,'9.70485601768871834038') +-- !query 425 schema +struct<> +-- !query 425 output + + + +-- !query 426 +INSERT INTO num_exp_ln VALUES (6,'11.45000246622944403127') +-- !query 426 schema +struct<> +-- !query 426 output + + + +-- !query 427 +INSERT INTO num_exp_ln VALUES (7,'18.23469429965478772991') +-- !query 427 schema +struct<> +-- !query 427 output + + + +-- !query 428 +INSERT INTO num_exp_ln VALUES (8,'11.22365546576315513668') +-- !query 428 schema +struct<> +-- !query 428 output + + + +-- !query 429 +INSERT INTO num_exp_ln VALUES (9,'17.03145425013166006962') +-- !query 429 schema +struct<> +-- !query 429 output + + + +-- !query 430 +INSERT INTO num_exp_log10 VALUES (0,'NaN') +-- !query 430 schema +struct<> +-- !query 430 output + + + +-- !query 431 +INSERT INTO num_exp_log10 VALUES (1,'NaN') +-- !query 431 schema +struct<> +-- !query 431 output + + + +-- !query 432 +INSERT INTO num_exp_log10 VALUES (2,'7.53578122160797276459') +-- !query 432 schema +struct<> +-- !query 432 output + + + +-- !query 433 +INSERT INTO num_exp_log10 VALUES (3,'.63447727016073160075') +-- !query 433 schema +struct<> +-- !query 433 output + + + +-- !query 434 +INSERT INTO num_exp_log10 VALUES (4,'6.89206461372691743345') +-- !query 434 schema +struct<> +-- !query 434 output + + + +-- !query 435 +INSERT INTO num_exp_log10 VALUES (5,'4.21476541614777768626') +-- !query 435 schema +struct<> +-- !query 435 output + + + +-- !query 436 +INSERT INTO num_exp_log10 VALUES (6,'4.97267288886207207671') +-- !query 436 schema +struct<> +-- !query 436 output + + + +-- !query 437 +INSERT INTO num_exp_log10 VALUES (7,'7.91922711353275546914') +-- !query 437 schema +struct<> +-- !query 437 output + + + +-- !query 438 +INSERT INTO num_exp_log10 VALUES (8,'4.87437163556421004138') +-- !query 438 schema +struct<> +-- !query 438 output + + + +-- !query 439 +INSERT INTO num_exp_log10 VALUES (9,'7.39666659961986567059') +-- !query 439 schema +struct<> +-- !query 439 output + + + +-- !query 440 +INSERT INTO num_exp_power_10_ln VALUES (0,'NaN') +-- !query 440 schema +struct<> +-- !query 440 output + + + +-- !query 441 +INSERT INTO num_exp_power_10_ln VALUES (1,'NaN') +-- !query 441 schema +struct<> +-- !query 441 output + + + +-- !query 442 +INSERT INTO num_exp_power_10_ln VALUES (2,'224790267919917955.13261618583642653184') +-- !query 442 schema +struct<> +-- !query 442 output + + + +-- !query 443 +INSERT INTO num_exp_power_10_ln VALUES (3,'28.90266599445155957393') +-- !query 443 schema +struct<> +-- !query 443 output + + + +-- !query 444 +INSERT INTO num_exp_power_10_ln VALUES (4,'7405685069594999.07733999469386277636') +-- !query 444 schema +struct<> +-- !query 444 output + + + +-- !query 445 +INSERT INTO num_exp_power_10_ln VALUES (5,'5068226527.32127265408584640098') +-- !query 445 schema +struct<> +-- !query 445 output + + + +-- !query 446 +INSERT INTO num_exp_power_10_ln VALUES (6,'281839893606.99372343357047819067') +-- !query 446 schema +struct<> +-- !query 446 output + + + +-- !query 447 +INSERT INTO num_exp_power_10_ln VALUES (7,'1716699575118597095.42330819910640247627') +-- !query 447 schema +struct<> +-- !query 447 output + + + +-- !query 448 +INSERT INTO num_exp_power_10_ln VALUES (8,'167361463828.07491320069016125952') +-- !query 448 schema +struct<> +-- !query 448 output + + + +-- !query 449 +INSERT INTO num_exp_power_10_ln VALUES (9,'107511333880052007.04141124673540337457') +-- !query 449 schema +struct<> +-- !query 449 output + + + +-- !query 450 +INSERT INTO num_data VALUES (0, '0') +-- !query 450 schema +struct<> +-- !query 450 output + + + +-- !query 451 +INSERT INTO num_data VALUES (1, '0') +-- !query 451 schema +struct<> +-- !query 451 output + + + +-- !query 452 +INSERT INTO num_data VALUES (2, '-34338492.215397047') +-- !query 452 schema +struct<> +-- !query 452 output + + + +-- !query 453 +INSERT INTO num_data VALUES (3, '4.31') +-- !query 453 schema +struct<> +-- !query 453 output + + + +-- !query 454 +INSERT INTO num_data VALUES (4, '7799461.4119') +-- !query 454 schema +struct<> +-- !query 454 output + + + +-- !query 455 +INSERT INTO num_data VALUES (5, '16397.038491') +-- !query 455 schema +struct<> +-- !query 455 output + + + +-- !query 456 +INSERT INTO num_data VALUES (6, '93901.57763026') +-- !query 456 schema +struct<> +-- !query 456 output + + + +-- !query 457 +INSERT INTO num_data VALUES (7, '-83028485') +-- !query 457 schema +struct<> +-- !query 457 output + + + +-- !query 458 +INSERT INTO num_data VALUES (8, '74881') +-- !query 458 schema +struct<> +-- !query 458 output + + + +-- !query 459 +INSERT INTO num_data VALUES (9, '-24926804.045047420') +-- !query 459 schema +struct<> +-- !query 459 output + + + +-- !query 460 +SELECT * FROM num_data +-- !query 460 schema +struct +-- !query 460 output +0 0 +1 0 +2 -34338492.215397047 +3 4.31 +4 7799461.4119 +5 16397.038491 +6 93901.57763026 +7 -83028485 +8 74881 +9 -24926804.04504742 + + +-- !query 461 +TRUNCATE TABLE num_result +-- !query 461 schema +struct<> +-- !query 461 output + + + +-- !query 462 +INSERT INTO num_result SELECT t1.id, t2.id, t1.val + t2.val + FROM num_data t1, num_data t2 +-- !query 462 schema +struct<> +-- !query 462 output + + + +-- !query 463 +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 463 schema +struct +-- !query 463 output + + + +-- !query 464 +TRUNCATE TABLE num_result +-- !query 464 schema +struct<> +-- !query 464 output + + + +-- !query 465 +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val + t2.val, 10) + FROM num_data t1, num_data t2 +-- !query 465 schema +struct<> +-- !query 465 output + + + +-- !query 466 +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 466 schema +struct +-- !query 466 output + + + +-- !query 467 +TRUNCATE TABLE num_result +-- !query 467 schema +struct<> +-- !query 467 output + + + +-- !query 468 +INSERT INTO num_result SELECT t1.id, t2.id, t1.val - t2.val + FROM num_data t1, num_data t2 +-- !query 468 schema +struct<> +-- !query 468 output + + + +-- !query 469 +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 469 schema +struct +-- !query 469 output + + + +-- !query 470 +TRUNCATE TABLE num_result +-- !query 470 schema +struct<> +-- !query 470 output + + + +-- !query 471 +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val - t2.val, 40) + FROM num_data t1, num_data t2 +-- !query 471 schema +struct<> +-- !query 471 output + + + +-- !query 472 +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 472 schema +struct +-- !query 472 output + + + +-- !query 473 +TRUNCATE TABLE num_result +-- !query 473 schema +struct<> +-- !query 473 output + + + +-- !query 474 +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 474 schema +struct<> +-- !query 474 output +org.apache.spark.sql.AnalysisException +`default`.`num_result` requires that the data to be inserted have the same number of columns as the target table: target table has 3 column(s) but the inserted data has 5 column(s), including 0 partition column(s) having constant value(s).; + + +-- !query 475 +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 475 schema +struct +-- !query 475 output + + + +-- !query 476 +TRUNCATE TABLE num_result +-- !query 476 schema +struct<> +-- !query 476 output + + + +-- !query 477 +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val * t2.val, 30) + FROM num_data t1, num_data t2 +-- !query 477 schema +struct<> +-- !query 477 output + + + +-- !query 478 +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 478 schema +struct +-- !query 478 output +2 2 1179132047626883.596862 1179132047626883.5968621359 +2 3 -147998901.448361 -147998901.4483612726 +2 4 -267821744976817.811114 -267821744976817.8111137107 +2 5 -563049578578.769243 -563049578578.7692425067 +2 6 -3224438592470.184498 -3224438592470.1844981193 +2 7 2851072985828710.485884 2851072985828710.485883795 +2 8 -2571300635581.146276 -2571300635581.146276407 +2 9 855948866655588.453742 855948866655588.4537415092 +3 2 -147998901.448361 -147998901.4483612726 +3 5 70671.235896 70671.23589621 +3 6 404715.799586 404715.7995864206 +3 9 -107434525.434154 -107434525.4341543802 +4 2 -267821744976817.811114 -267821744976817.8111137107 +4 4 60831598315717.141462 60831598315717.14146161 +4 5 127888068979.993505 127888068979.9935054429 +4 6 732381731243.745116 732381731243.7451157641 +4 9 -194415646271340.181596 -194415646271340.1815956523 +5 2 -563049578578.769243 -563049578578.7692425067 +5 3 70671.235896 70671.23589621 +5 4 127888068979.993505 127888068979.9935054429 +5 5 268862871.275336 268862871.2753355571 +5 6 1539707782.768998 1539707782.7689977863 +5 9 -408725765384.257044 -408725765384.2570436602 +6 2 -3224438592470.184498 -3224438592470.1844981193 +6 3 404715.799586 404715.7995864206 +6 4 732381731243.745116 732381731243.7451157641 +6 5 1539707782.768998 1539707782.7689977863 +6 6 8817506281.451745 8817506281.4517452373 +6 7 -7796505729750.377956 -7796505729750.3779561 +6 8 7031444034.531499 7031444034.53149906 +6 9 -2340666225110.299295 -2340666225110.2992952129 +7 2 2851072985828710.485884 2851072985828710.485883795 +7 6 -7796505729750.377956 -7796505729750.3779561 +7 9 2069634775752159.035759 2069634775752159.0357587 +8 2 -2571300635581.146276 -2571300635581.146276407 +8 6 7031444034.531499 7031444034.53149906 +8 9 -1866544013697.195857 -1866544013697.19585702 +9 2 855948866655588.453742 855948866655588.4537415092 +9 3 -107434525.434154 -107434525.4341543802 +9 4 -194415646271340.181596 -194415646271340.1815956523 +9 5 -408725765384.257044 -408725765384.2570436602 +9 6 -2340666225110.299295 -2340666225110.2992952129 +9 7 2069634775752159.035759 2069634775752159.0357587 +9 8 -1866544013697.195857 -1866544013697.19585702 +9 9 621345559900192.420121 621345559900192.42012063 + + +-- !query 479 +TRUNCATE TABLE num_result +-- !query 479 schema +struct<> +-- !query 479 output + + + +-- !query 480 +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 480 schema +struct<> +-- !query 480 output + + + +-- !query 481 +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 481 schema +struct +-- !query 481 output +2 3 -7967167.567378 -7967167.5673775051 +2 4 -4.402675 -4.4026748005 +2 5 -2094.188669 -2094.1886691456 +2 6 -365.685999 -365.6859989148 +2 7 0.413575 0.4135748378 +2 8 -458.574167 -458.5741672173 +2 9 1.377573 1.3775729995 +3 2 0 -0.0000001255 +3 4 0.000001 0.0000005526 +3 5 0.000263 0.0002628523 +3 6 0.000046 0.0000458991 +3 7 0 -0.0000000519 +3 8 0.000058 0.000057558 +3 9 0 -0.0000001729 +4 2 -0.227135 -0.22713465 +4 3 1809619.817146 1809619.8171461717 +4 5 475.66281 475.6628104631 +4 6 83.059961 83.0599613844 +4 7 -0.093937 -0.093937176 +4 8 104.158083 104.1580829837 +4 9 -0.312895 -0.3128945611 +5 2 -0.000478 -0.0004775119 +5 3 3804.417283 3804.4172832947 +5 4 0.002102 0.0021023296 +5 6 0.174619 0.1746194143 +5 7 -0.000197 -0.0001974869 +5 8 0.218975 0.2189746196 +5 9 -0.000658 -0.0006578075 +6 2 -0.002735 -0.0027345865 +6 3 21786.908963 21786.9089629374 +6 4 0.012039 0.0120394951 +6 5 5.72674 5.7267400867 +6 7 -0.001131 -0.0011309562 +6 8 1.254011 1.2540107321 +6 9 -0.003767 -0.0037670925 +7 2 2.417942 2.4179420715 +7 3 -19264149.651972 -19264149.6519721578 +7 4 -10.645413 -10.6454126273 +7 5 -5063.626889 -5063.6268888173 +7 6 -884.207562 -884.2075617401 +7 8 -1108.805772 -1108.8057718246 +7 9 3.330892 3.330891712 +8 2 -0.002181 -0.0021806723 +8 3 17373.781903 17373.7819025522 +8 4 0.009601 0.0096007911 +8 5 4.566739 4.5667392951 +8 6 0.797441 0.7974413411 +8 7 -0.000902 -0.0009018712 +8 9 -0.003004 -0.0030040353 +9 2 0.725914 0.7259143438 +9 3 -5783481.216948 -5783481.2169483573 +9 4 -3.195965 -3.1959647889 +9 5 -1520.201594 -1520.2015936432 +9 6 -265.456712 -265.4567119543 +9 7 0.30022 0.300219907 +9 8 -332.885566 -332.8855656982 + + +-- !query 482 +TRUNCATE TABLE num_result +-- !query 482 schema +struct<> +-- !query 482 output + + + +-- !query 483 +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 483 schema +struct<> +-- !query 483 output + + + +-- !query 484 +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 484 schema +struct +-- !query 484 output +2 3 -7967167.567378 -7967167.5673775051 +2 4 -4.402675 -4.4026748005 +2 5 -2094.188669 -2094.1886691456 +2 6 -365.685999 -365.6859989148 +2 7 0.413575 0.4135748378 +2 8 -458.574167 -458.5741672173 +2 9 1.377573 1.3775729995 +3 2 0 -0.0000001255 +3 4 0.000001 0.0000005526 +3 5 0.000263 0.0002628523 +3 6 0.000046 0.0000458991 +3 7 0 -0.0000000519 +3 8 0.000058 0.000057558 +3 9 0 -0.0000001729 +4 2 -0.227135 -0.22713465 +4 3 1809619.817146 1809619.8171461717 +4 5 475.66281 475.6628104631 +4 6 83.059961 83.0599613844 +4 7 -0.093937 -0.093937176 +4 8 104.158083 104.1580829837 +4 9 -0.312895 -0.3128945611 +5 2 -0.000478 -0.0004775119 +5 3 3804.417283 3804.4172832947 +5 4 0.002102 0.0021023296 +5 6 0.174619 0.1746194143 +5 7 -0.000197 -0.0001974869 +5 8 0.218975 0.2189746196 +5 9 -0.000658 -0.0006578075 +6 2 -0.002735 -0.0027345865 +6 3 21786.908963 21786.9089629374 +6 4 0.012039 0.0120394951 +6 5 5.72674 5.7267400867 +6 7 -0.001131 -0.0011309562 +6 8 1.254011 1.2540107321 +6 9 -0.003767 -0.0037670925 +7 2 2.417942 2.4179420715 +7 3 -19264149.651972 -19264149.6519721578 +7 4 -10.645413 -10.6454126273 +7 5 -5063.626889 -5063.6268888173 +7 6 -884.207562 -884.2075617401 +7 8 -1108.805772 -1108.8057718246 +7 9 3.330892 3.330891712 +8 2 -0.002181 -0.0021806723 +8 3 17373.781903 17373.7819025522 +8 4 0.009601 0.0096007911 +8 5 4.566739 4.5667392951 +8 6 0.797441 0.7974413411 +8 7 -0.000902 -0.0009018712 +8 9 -0.003004 -0.0030040353 +9 2 0.725914 0.7259143438 +9 3 -5783481.216948 -5783481.2169483573 +9 4 -3.195965 -3.1959647889 +9 5 -1520.201594 -1520.2015936432 +9 6 -265.456712 -265.4567119543 +9 7 0.30022 0.300219907 +9 8 -332.885566 -332.8855656982 + + +-- !query 485 +TRUNCATE TABLE num_result +-- !query 485 schema +struct<> +-- !query 485 output + + + +-- !query 486 +INSERT INTO num_result SELECT id, 0, SQRT(ABS(val)) + FROM num_data +-- !query 486 schema +struct<> +-- !query 486 output + + + +-- !query 487 +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 487 schema +struct +-- !query 487 output + + + +-- !query 488 +TRUNCATE TABLE num_result +-- !query 488 schema +struct<> +-- !query 488 output + + + +-- !query 489 +INSERT INTO num_result SELECT id, 0, LN(ABS(val)) + FROM num_data + WHERE val != '0.0' +-- !query 489 schema +struct<> +-- !query 489 output + + + +-- !query 490 +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 490 schema +struct +-- !query 490 output + + + +-- !query 491 +TRUNCATE TABLE num_result +-- !query 491 schema +struct<> +-- !query 491 output + + + +-- !query 492 +INSERT INTO num_result SELECT id, 0, LOG(cast('10' as decimal(38, 18)), ABS(val)) + FROM num_data + WHERE val != '0.0' +-- !query 492 schema +struct<> +-- !query 492 output + + + +-- !query 493 +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 493 schema +struct +-- !query 493 output + + + +-- !query 494 +TRUNCATE TABLE num_result +-- !query 494 schema +struct<> +-- !query 494 output + + + +-- !query 495 +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 495 schema +struct<> +-- !query 495 output + + + +-- !query 496 +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 496 schema +struct +-- !query 496 output +2 224790267919917472 224790267919917955.1326161858 +4 7405685069595001 7405685069594999.0773399947 +5 5068226527.321263 5068226527.3212726541 +6 281839893606.99365 281839893606.9937234336 +7 1716699575118595840 1716699575118597095.4233081991 +8 167361463828.0749 167361463828.0749132007 +9 107511333880051856 107511333880052007.0414112467 + + +-- !query 497 +SELECT AVG(val) FROM num_data +-- !query 497 schema +struct +-- !query 497 output +-13430913.5922423207 + + +-- !query 498 +CREATE TABLE fract_only (id int, val decimal(4,4)) USING parquet +-- !query 498 schema +struct<> +-- !query 498 output + + + +-- !query 499 +INSERT INTO fract_only VALUES (1, '0.0') +-- !query 499 schema +struct<> +-- !query 499 output + + + +-- !query 500 +INSERT INTO fract_only VALUES (2, '0.1') +-- !query 500 schema +struct<> +-- !query 500 output + + + +-- !query 501 +INSERT INTO fract_only VALUES (4, '-0.9999') +-- !query 501 schema +struct<> +-- !query 501 output + + + +-- !query 502 +INSERT INTO fract_only VALUES (5, '0.99994') +-- !query 502 schema +struct<> +-- !query 502 output + + + +-- !query 503 +INSERT INTO fract_only VALUES (7, '0.00001') +-- !query 503 schema +struct<> +-- !query 503 output + + + +-- !query 504 +INSERT INTO fract_only VALUES (8, '0.00017') +-- !query 504 schema +struct<> +-- !query 504 output + + + +-- !query 505 +SELECT * FROM fract_only +-- !query 505 schema +struct +-- !query 505 output +1 0 +2 0.1 +4 -0.9999 +5 0.9999 +7 0 +8 0.0002 + + +-- !query 506 +DROP TABLE fract_only +-- !query 506 schema +struct<> +-- !query 506 output + + + +-- !query 507 +SELECT decimal(double('NaN')) +-- !query 507 schema +struct +-- !query 507 output +NULL + + +-- !query 508 +SELECT decimal(double('Infinity')) +-- !query 508 schema +struct +-- !query 508 output +NULL + + +-- !query 509 +SELECT decimal(double('-Infinity')) +-- !query 509 schema +struct +-- !query 509 output +NULL + + +-- !query 510 +SELECT decimal(float('NaN')) +-- !query 510 schema +struct +-- !query 510 output +NULL + + +-- !query 511 +SELECT decimal(float('Infinity')) +-- !query 511 schema +struct +-- !query 511 output +NULL + + +-- !query 512 +SELECT decimal(float('-Infinity')) +-- !query 512 schema +struct +-- !query 512 output +NULL + + +-- !query 513 +CREATE TABLE ceil_floor_round (a decimal(38, 18)) USING parquet +-- !query 513 schema +struct<> +-- !query 513 output + + + +-- !query 514 +INSERT INTO ceil_floor_round VALUES ('-5.5') +-- !query 514 schema +struct<> +-- !query 514 output + + + +-- !query 515 +INSERT INTO ceil_floor_round VALUES ('-5.499999') +-- !query 515 schema +struct<> +-- !query 515 output + + + +-- !query 516 +INSERT INTO ceil_floor_round VALUES ('9.5') +-- !query 516 schema +struct<> +-- !query 516 output + + + +-- !query 517 +INSERT INTO ceil_floor_round VALUES ('9.4999999') +-- !query 517 schema +struct<> +-- !query 517 output + + + +-- !query 518 +INSERT INTO ceil_floor_round VALUES ('0.0') +-- !query 518 schema +struct<> +-- !query 518 output + + + +-- !query 519 +INSERT INTO ceil_floor_round VALUES ('0.0000001') +-- !query 519 schema +struct<> +-- !query 519 output + + + +-- !query 520 +INSERT INTO ceil_floor_round VALUES ('-0.000001') +-- !query 520 schema +struct<> +-- !query 520 output + + + +-- !query 521 +SELECT a, ceil(a), ceiling(a), floor(a), round(a) FROM ceil_floor_round +-- !query 521 schema +struct +-- !query 521 output +-0.000001 0 0 -1 0 +-5.499999 -5 -5 -6 -5 +-5.5 -5 -5 -6 -6 +0 0 0 0 0 +0.0000001 1 1 0 0 +9.4999999 10 10 9 9 +9.5 10 10 9 10 + + +-- !query 522 +DROP TABLE ceil_floor_round +-- !query 522 schema +struct<> +-- !query 522 output + + + +-- !query 523 +CREATE TABLE num_input_test (n1 decimal(38, 18)) USING parquet +-- !query 523 schema +struct<> +-- !query 523 output + + + +-- !query 524 +INSERT INTO num_input_test VALUES (trim(' 123')) +-- !query 524 schema +struct<> +-- !query 524 output + + + +-- !query 525 +INSERT INTO num_input_test VALUES (trim(' 3245874 ')) +-- !query 525 schema +struct<> +-- !query 525 output + + + +-- !query 526 +INSERT INTO num_input_test VALUES (trim(' -93853')) +-- !query 526 schema +struct<> +-- !query 526 output + + + +-- !query 527 +INSERT INTO num_input_test VALUES ('555.50') +-- !query 527 schema +struct<> +-- !query 527 output + + + +-- !query 528 +INSERT INTO num_input_test VALUES ('-555.50') +-- !query 528 schema +struct<> +-- !query 528 output + + + +-- !query 529 +SELECT * FROM num_input_test +-- !query 529 schema +struct +-- !query 529 output +-555.5 +-93853 +123 +3245874 +555.5 + + +-- !query 530 +select cast(999999999999999999999 as decimal(38, 0))/1000000000000000000000 +-- !query 530 schema +struct<(CAST(CAST(999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) / CAST(1000000000000000000000 AS DECIMAL(38,0))):decimal(38,6)> +-- !query 530 output +1 + + +-- !query 531 +select mod(cast(999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query 531 schema +struct<(CAST(CAST(999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) % CAST(1000000000000000000000 AS DECIMAL(38,0))):decimal(22,0)> +-- !query 531 output +999999999999999999999 + + +-- !query 532 +select mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query 532 schema +struct<(CAST(CAST(-9999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) % CAST(1000000000000000000000 AS DECIMAL(38,0))):decimal(22,0)> +-- !query 532 output +-999999999999999999999 + + +-- !query 533 +select mod (70.0,70) +-- !query 533 schema +struct<(CAST(70.0 AS DECIMAL(3,1)) % CAST(CAST(70 AS DECIMAL(2,0)) AS DECIMAL(3,1))):decimal(3,1)> +-- !query 533 output +0 + + +-- !query 534 +select 70.0 / 70 +-- !query 534 schema +struct<(CAST(70.0 AS DECIMAL(3,1)) / CAST(CAST(70 AS DECIMAL(2,0)) AS DECIMAL(3,1))):decimal(8,6)> +-- !query 534 output +1 + + +-- !query 535 +select 12345678901234567890 % 123 +-- !query 535 schema +struct<(CAST(12345678901234567890 AS DECIMAL(20,0)) % CAST(CAST(123 AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(3,0)> +-- !query 535 output +78 + + +-- !query 536 +select exp(0.0) +-- !query 536 schema +struct +-- !query 536 output +1.0 + + +-- !query 537 +select exp(1.0) +-- !query 537 schema +struct +-- !query 537 output +2.718281828459045 + + +-- !query 538 +select exp(32.999) +-- !query 538 schema +struct +-- !query 538 output +2.1442904349215556E14 + + +-- !query 539 +select exp(-32.999) +-- !query 539 schema +struct +-- !query 539 output +4.663547361468238E-15 + + +-- !query 540 +select exp(123.456) +-- !query 540 schema +struct +-- !query 540 output +4.132944352778106E53 + + +-- !query 541 +select exp(-123.456) +-- !query 541 schema +struct +-- !query 541 output +2.4195825412645934E-54 + + +-- !query 542 +select exp(1234.5678) +-- !query 542 schema +struct +-- !query 542 output +Infinity + + +-- !query 543 +select * from range(cast(0.0 as decimal(38, 18)), cast(4.0 as decimal(38, 18))) +-- !query 543 schema +struct +-- !query 543 output +0 +1 +2 +3 + + +-- !query 544 +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 544 schema +struct +-- !query 544 output +0 +1 +2 +3 + + +-- !query 545 +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 545 schema +struct +-- !query 545 output +0 +2 +4 + + +-- !query 546 +select ln(1.2345678e-28) +-- !query 546 schema +struct +-- !query 546 output +-64.26166165451762 + + +-- !query 547 +select ln(0.0456789) +-- !query 547 schema +struct +-- !query 547 output +-3.0861187944847437 + + +-- !query 548 +select ln(0.99949452) +-- !query 548 schema +struct +-- !query 548 output +-5.056077980832118E-4 + + +-- !query 549 +select ln(1.00049687395) +-- !query 549 schema +struct +-- !query 549 output +4.967505490136803E-4 + + +-- !query 550 +select ln(1234.567890123456789) +-- !query 550 schema +struct +-- !query 550 output +7.11847630129779 + + +-- !query 551 +select ln(5.80397490724e5) +-- !query 551 schema +struct +-- !query 551 output +13.271468476626518 + + +-- !query 552 +select ln(9.342536355e34) +-- !query 552 schema +struct +-- !query 552 output +80.52247093552418 + + +-- !query 553 +select log(3.4634998359873254962349856073435545) +-- !query 553 schema +struct +-- !query 553 output +1.2422795911259166 + + +-- !query 554 +select log(9.999999999999999999) +-- !query 554 schema +struct +-- !query 554 output +2.302585092994046 + + +-- !query 555 +select log(10.00000000000000000) +-- !query 555 schema +struct +-- !query 555 output +2.302585092994046 + + +-- !query 556 +select log(10.00000000000000001) +-- !query 556 schema +struct +-- !query 556 output +2.302585092994046 + + +-- !query 557 +select log(590489.45235237) +-- !query 557 schema +struct +-- !query 557 output +13.288707052228641 + + +-- !query 558 +select log(0.99923, 4.58934e34) +-- !query 558 schema +struct +-- !query 558 output +-103611.55579543479 + + +-- !query 559 +select log(1.000016, 8.452010e18) +-- !query 559 schema +struct +-- !query 559 output +2723830.287707013 + + +-- !query 560 +SELECT SUM(decimal(9999)) FROM range(1, 100001) +-- !query 560 schema +struct +-- !query 560 output +999900000 + + +-- !query 561 +SELECT SUM(decimal(-9999)) FROM range(1, 100001) +-- !query 561 schema +struct +-- !query 561 output +-999900000 + + +-- !query 562 +DROP TABLE num_data +-- !query 562 schema +struct<> +-- !query 562 output + + + +-- !query 563 +DROP TABLE num_exp_add +-- !query 563 schema +struct<> +-- !query 563 output + + + +-- !query 564 +DROP TABLE num_exp_sub +-- !query 564 schema +struct<> +-- !query 564 output + + + +-- !query 565 +DROP TABLE num_exp_div +-- !query 565 schema +struct<> +-- !query 565 output + + + +-- !query 566 +DROP TABLE num_exp_mul +-- !query 566 schema +struct<> +-- !query 566 output + + + +-- !query 567 +DROP TABLE num_exp_sqrt +-- !query 567 schema +struct<> +-- !query 567 output + + + +-- !query 568 +DROP TABLE num_exp_ln +-- !query 568 schema +struct<> +-- !query 568 output + + + +-- !query 569 +DROP TABLE num_exp_log10 +-- !query 569 schema +struct<> +-- !query 569 output + + + +-- !query 570 +DROP TABLE num_exp_power_10_ln +-- !query 570 schema +struct<> +-- !query 570 output + + + +-- !query 571 +DROP TABLE num_result +-- !query 571 schema +struct<> +-- !query 571 output + + + +-- !query 572 +DROP TABLE num_input_test +-- !query 572 schema +struct<> +-- !query 572 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/select_implicit.sql.out new file mode 100644 index 0000000000000..0675820b381da --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/select_implicit.sql.out @@ -0,0 +1,416 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 38 + + +-- !query 0 +CREATE TABLE test_missing_target (a int, b int, c string, d string) using parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO test_missing_target VALUES (0, 1, 'XXXX', 'A') +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO test_missing_target VALUES (1, 2, 'ABAB', 'b') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO test_missing_target VALUES (2, 2, 'ABAB', 'c') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO test_missing_target VALUES (3, 3, 'BBBB', 'D') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO test_missing_target VALUES (4, 3, 'BBBB', 'e') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +INSERT INTO test_missing_target VALUES (5, 3, 'bbbb', 'F') +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +INSERT INTO test_missing_target VALUES (6, 4, 'cccc', 'g') +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +INSERT INTO test_missing_target VALUES (7, 4, 'cccc', 'h') +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +INSERT INTO test_missing_target VALUES (8, 4, 'CCCC', 'I') +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +INSERT INTO test_missing_target VALUES (9, 4, 'CCCC', 'j') +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +SELECT c, count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c +-- !query 11 schema +struct +-- !query 11 output +ABAB 2 +BBBB 2 +CCCC 2 +XXXX 1 +bbbb 1 +cccc 2 + + +-- !query 12 +SELECT count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c +-- !query 12 schema +struct +-- !query 12 output +2 +2 +2 +1 +1 +2 + + +-- !query 13 +SELECT count(*) FROM test_missing_target GROUP BY a ORDER BY b +-- !query 13 schema +struct<> +-- !query 13 output +org.apache.spark.sql.AnalysisException +cannot resolve '`b`' given input columns: [count(1)]; line 1 pos 61 + + +-- !query 14 +SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b +-- !query 14 schema +struct +-- !query 14 output +1 +2 +3 +4 + + +-- !query 15 +SELECT test_missing_target.b, count(*) + FROM test_missing_target GROUP BY b ORDER BY b +-- !query 15 schema +struct +-- !query 15 output +1 1 +2 2 +3 3 +4 4 + + +-- !query 16 +SELECT c FROM test_missing_target ORDER BY a +-- !query 16 schema +struct +-- !query 16 output +XXXX +ABAB +ABAB +BBBB +BBBB +bbbb +cccc +cccc +CCCC +CCCC + + +-- !query 17 +SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b desc +-- !query 17 schema +struct +-- !query 17 output +4 +3 +2 +1 + + +-- !query 18 +SELECT count(*) FROM test_missing_target ORDER BY 1 desc +-- !query 18 schema +struct +-- !query 18 output +10 + + +-- !query 19 +SELECT c, count(*) FROM test_missing_target GROUP BY 1 ORDER BY 1 +-- !query 19 schema +struct +-- !query 19 output +ABAB 2 +BBBB 2 +CCCC 2 +XXXX 1 +bbbb 1 +cccc 2 + + +-- !query 20 +SELECT c, count(*) FROM test_missing_target GROUP BY 3 +-- !query 20 schema +struct<> +-- !query 20 output +org.apache.spark.sql.AnalysisException +GROUP BY position 3 is not in select list (valid range is [1, 2]); line 1 pos 53 + + +-- !query 21 +SELECT count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY b ORDER BY b +-- !query 21 schema +struct<> +-- !query 21 output +org.apache.spark.sql.AnalysisException +Reference 'b' is ambiguous, could be: x.b, y.b.; line 3 pos 10 + + +-- !query 22 +SELECT a, a FROM test_missing_target + ORDER BY a +-- !query 22 schema +struct +-- !query 22 output +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 + + +-- !query 23 +SELECT a/2, a/2 FROM test_missing_target + ORDER BY a/2 +-- !query 23 schema +struct<(a div 2):int,(a div 2):int> +-- !query 23 output +0 0 +0 0 +1 1 +1 1 +2 2 +2 2 +3 3 +3 3 +4 4 +4 4 + + +-- !query 24 +SELECT a/2, a/2 FROM test_missing_target + GROUP BY a/2 ORDER BY a/2 +-- !query 24 schema +struct<(a div 2):int,(a div 2):int> +-- !query 24 output +0 0 +1 1 +2 2 +3 3 +4 4 + + +-- !query 25 +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 25 schema +struct +-- !query 25 output +1 1 +2 2 +3 3 +4 4 + + +-- !query 26 +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 26 schema +struct +-- !query 26 output +1 +2 +3 +4 + + +-- !query 27 +SELECT a%2, count(b) FROM test_missing_target +GROUP BY test_missing_target.a%2 +ORDER BY test_missing_target.a%2 +-- !query 27 schema +struct<(a % 2):int,count(b):bigint> +-- !query 27 output +0 5 +1 5 + + +-- !query 28 +SELECT count(c) FROM test_missing_target +GROUP BY lower(test_missing_target.c) +ORDER BY lower(test_missing_target.c) +-- !query 28 schema +struct +-- !query 28 output +2 +3 +4 +1 + + +-- !query 29 +SELECT count(a) FROM test_missing_target GROUP BY a ORDER BY b +-- !query 29 schema +struct<> +-- !query 29 output +org.apache.spark.sql.AnalysisException +cannot resolve '`b`' given input columns: [count(a)]; line 1 pos 61 + + +-- !query 30 +SELECT count(b) FROM test_missing_target GROUP BY b/2 ORDER BY b/2 +-- !query 30 schema +struct +-- !query 30 output +1 +5 +4 + + +-- !query 31 +SELECT lower(test_missing_target.c), count(c) + FROM test_missing_target GROUP BY lower(c) ORDER BY lower(c) +-- !query 31 schema +struct +-- !query 31 output +abab 2 +bbbb 3 +cccc 4 +xxxx 1 + + +-- !query 32 +SELECT a FROM test_missing_target ORDER BY upper(d) +-- !query 32 schema +struct +-- !query 32 output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query 33 +SELECT count(b) FROM test_missing_target + GROUP BY (b + 1) / 2 ORDER BY (b + 1) / 2 desc +-- !query 33 schema +struct +-- !query 33 output +7 +3 + + +-- !query 34 +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 34 schema +struct<> +-- !query 34 output +org.apache.spark.sql.AnalysisException +Reference 'b' is ambiguous, could be: x.b, y.b.; line 3 pos 10 + + +-- !query 35 +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 35 schema +struct<(b div 2):int,count(b):bigint> +-- !query 35 output +0 1 +1 5 +2 4 + + +-- !query 36 +SELECT count(b) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b/2 +-- !query 36 schema +struct<> +-- !query 36 output +org.apache.spark.sql.AnalysisException +Reference 'b' is ambiguous, could be: x.b, y.b.; line 1 pos 13 + + +-- !query 37 +DROP TABLE test_missing_target +-- !query 37 schema +struct<> +-- !query 37 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out new file mode 100644 index 0000000000000..200fecce17bc0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out @@ -0,0 +1,130 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 14 + + +-- !query 0 +CREATE TABLE TIMESTAMP_TBL (d1 timestamp) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02') +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02 03:04:05') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-08') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO TIMESTAMP_TBL VALUES ('2001-09-22T18:19:20') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +SELECT '' AS `64`, d1 FROM TIMESTAMP_TBL +-- !query 5 schema +struct<64:string,d1:timestamp> +-- !query 5 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 6 +SELECT '' AS `48`, d1 FROM TIMESTAMP_TBL + WHERE d1 > timestamp '1997-01-02' +-- !query 6 schema +struct<48:string,d1:timestamp> +-- !query 6 output +1997-01-02 03:04:05 + 1997-02-10 17:32:01 + 2001-09-22 18:19:20 + + +-- !query 7 +SELECT '' AS `15`, d1 FROM TIMESTAMP_TBL + WHERE d1 < timestamp '1997-01-02' +-- !query 7 schema +struct<15:string,d1:timestamp> +-- !query 7 output + + + +-- !query 8 +SELECT '' AS one, d1 FROM TIMESTAMP_TBL + WHERE d1 = timestamp '1997-01-02' +-- !query 8 schema +struct +-- !query 8 output +1997-01-02 00:00:00 + + +-- !query 9 +SELECT '' AS `63`, d1 FROM TIMESTAMP_TBL + WHERE d1 != timestamp '1997-01-02' +-- !query 9 schema +struct<63:string,d1:timestamp> +-- !query 9 output +1997-01-02 03:04:05 + 1997-02-10 17:32:01 + 2001-09-22 18:19:20 + + +-- !query 10 +SELECT '' AS `16`, d1 FROM TIMESTAMP_TBL + WHERE d1 <= timestamp '1997-01-02' +-- !query 10 schema +struct<16:string,d1:timestamp> +-- !query 10 output +1997-01-02 00:00:00 + + +-- !query 11 +SELECT '' AS `49`, d1 FROM TIMESTAMP_TBL + WHERE d1 >= timestamp '1997-01-02' +-- !query 11 schema +struct<49:string,d1:timestamp> +-- !query 11 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 12 +SELECT '' AS date_trunc_week, date_trunc( 'week', timestamp '2004-02-29 15:44:17.71393' ) AS week_trunc +-- !query 12 schema +struct +-- !query 12 output +2004-02-23 00:00:00 + + +-- !query 13 +DROP TABLE TIMESTAMP_TBL +-- !query 13 schema +struct<> +-- !query 13 output + diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out index 12c1d1617679f..a4cd408c04bf8 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/dateTimeOperations.sql.out @@ -16,7 +16,7 @@ select cast(1 as tinyint) + interval 2 day struct<> -- !query 1 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS TINYINT) + interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) + interval 2 days)' (tinyint and calendarinterval).; line 1 pos 7 +cannot resolve '(CAST(1 AS TINYINT) + interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) + interval 2 days)' (tinyint and interval).; line 1 pos 7 -- !query 2 @@ -25,7 +25,7 @@ select cast(1 as smallint) + interval 2 day struct<> -- !query 2 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS SMALLINT) + interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) + interval 2 days)' (smallint and calendarinterval).; line 1 pos 7 +cannot resolve '(CAST(1 AS SMALLINT) + interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) + interval 2 days)' (smallint and interval).; line 1 pos 7 -- !query 3 @@ -34,7 +34,7 @@ select cast(1 as int) + interval 2 day struct<> -- !query 3 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS INT) + interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS INT) + interval 2 days)' (int and calendarinterval).; line 1 pos 7 +cannot resolve '(CAST(1 AS INT) + interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS INT) + interval 2 days)' (int and interval).; line 1 pos 7 -- !query 4 @@ -43,7 +43,7 @@ select cast(1 as bigint) + interval 2 day struct<> -- !query 4 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BIGINT) + interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) + interval 2 days)' (bigint and calendarinterval).; line 1 pos 7 +cannot resolve '(CAST(1 AS BIGINT) + interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) + interval 2 days)' (bigint and interval).; line 1 pos 7 -- !query 5 @@ -52,7 +52,7 @@ select cast(1 as float) + interval 2 day struct<> -- !query 5 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS FLOAT) + interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) + interval 2 days)' (float and calendarinterval).; line 1 pos 7 +cannot resolve '(CAST(1 AS FLOAT) + interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) + interval 2 days)' (float and interval).; line 1 pos 7 -- !query 6 @@ -61,7 +61,7 @@ select cast(1 as double) + interval 2 day struct<> -- !query 6 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DOUBLE) + interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) + interval 2 days)' (double and calendarinterval).; line 1 pos 7 +cannot resolve '(CAST(1 AS DOUBLE) + interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) + interval 2 days)' (double and interval).; line 1 pos 7 -- !query 7 @@ -70,7 +70,7 @@ select cast(1 as decimal(10, 0)) + interval 2 day struct<> -- !query 7 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(10,0)) + interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) + interval 2 days)' (decimal(10,0) and calendarinterval).; line 1 pos 7 +cannot resolve '(CAST(1 AS DECIMAL(10,0)) + interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) + interval 2 days)' (decimal(10,0) and interval).; line 1 pos 7 -- !query 8 @@ -95,7 +95,7 @@ select cast('1' as binary) + interval 2 day struct<> -- !query 10 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('1' AS BINARY) + interval 2 days)' due to data type mismatch: differing types in '(CAST('1' AS BINARY) + interval 2 days)' (binary and calendarinterval).; line 1 pos 7 +cannot resolve '(CAST('1' AS BINARY) + interval 2 days)' due to data type mismatch: differing types in '(CAST('1' AS BINARY) + interval 2 days)' (binary and interval).; line 1 pos 7 -- !query 11 @@ -104,7 +104,7 @@ select cast(1 as boolean) + interval 2 day struct<> -- !query 11 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BOOLEAN) + interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) + interval 2 days)' (boolean and calendarinterval).; line 1 pos 7 +cannot resolve '(CAST(1 AS BOOLEAN) + interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) + interval 2 days)' (boolean and interval).; line 1 pos 7 -- !query 12 @@ -129,7 +129,7 @@ select interval 2 day + cast(1 as tinyint) struct<> -- !query 14 output org.apache.spark.sql.AnalysisException -cannot resolve '(interval 2 days + CAST(1 AS TINYINT))' due to data type mismatch: differing types in '(interval 2 days + CAST(1 AS TINYINT))' (calendarinterval and tinyint).; line 1 pos 7 +cannot resolve '(interval 2 days + CAST(1 AS TINYINT))' due to data type mismatch: differing types in '(interval 2 days + CAST(1 AS TINYINT))' (interval and tinyint).; line 1 pos 7 -- !query 15 @@ -138,7 +138,7 @@ select interval 2 day + cast(1 as smallint) struct<> -- !query 15 output org.apache.spark.sql.AnalysisException -cannot resolve '(interval 2 days + CAST(1 AS SMALLINT))' due to data type mismatch: differing types in '(interval 2 days + CAST(1 AS SMALLINT))' (calendarinterval and smallint).; line 1 pos 7 +cannot resolve '(interval 2 days + CAST(1 AS SMALLINT))' due to data type mismatch: differing types in '(interval 2 days + CAST(1 AS SMALLINT))' (interval and smallint).; line 1 pos 7 -- !query 16 @@ -147,7 +147,7 @@ select interval 2 day + cast(1 as int) struct<> -- !query 16 output org.apache.spark.sql.AnalysisException -cannot resolve '(interval 2 days + CAST(1 AS INT))' due to data type mismatch: differing types in '(interval 2 days + CAST(1 AS INT))' (calendarinterval and int).; line 1 pos 7 +cannot resolve '(interval 2 days + CAST(1 AS INT))' due to data type mismatch: differing types in '(interval 2 days + CAST(1 AS INT))' (interval and int).; line 1 pos 7 -- !query 17 @@ -156,7 +156,7 @@ select interval 2 day + cast(1 as bigint) struct<> -- !query 17 output org.apache.spark.sql.AnalysisException -cannot resolve '(interval 2 days + CAST(1 AS BIGINT))' due to data type mismatch: differing types in '(interval 2 days + CAST(1 AS BIGINT))' (calendarinterval and bigint).; line 1 pos 7 +cannot resolve '(interval 2 days + CAST(1 AS BIGINT))' due to data type mismatch: differing types in '(interval 2 days + CAST(1 AS BIGINT))' (interval and bigint).; line 1 pos 7 -- !query 18 @@ -165,7 +165,7 @@ select interval 2 day + cast(1 as float) struct<> -- !query 18 output org.apache.spark.sql.AnalysisException -cannot resolve '(interval 2 days + CAST(1 AS FLOAT))' due to data type mismatch: differing types in '(interval 2 days + CAST(1 AS FLOAT))' (calendarinterval and float).; line 1 pos 7 +cannot resolve '(interval 2 days + CAST(1 AS FLOAT))' due to data type mismatch: differing types in '(interval 2 days + CAST(1 AS FLOAT))' (interval and float).; line 1 pos 7 -- !query 19 @@ -174,7 +174,7 @@ select interval 2 day + cast(1 as double) struct<> -- !query 19 output org.apache.spark.sql.AnalysisException -cannot resolve '(interval 2 days + CAST(1 AS DOUBLE))' due to data type mismatch: differing types in '(interval 2 days + CAST(1 AS DOUBLE))' (calendarinterval and double).; line 1 pos 7 +cannot resolve '(interval 2 days + CAST(1 AS DOUBLE))' due to data type mismatch: differing types in '(interval 2 days + CAST(1 AS DOUBLE))' (interval and double).; line 1 pos 7 -- !query 20 @@ -183,7 +183,7 @@ select interval 2 day + cast(1 as decimal(10, 0)) struct<> -- !query 20 output org.apache.spark.sql.AnalysisException -cannot resolve '(interval 2 days + CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(interval 2 days + CAST(1 AS DECIMAL(10,0)))' (calendarinterval and decimal(10,0)).; line 1 pos 7 +cannot resolve '(interval 2 days + CAST(1 AS DECIMAL(10,0)))' due to data type mismatch: differing types in '(interval 2 days + CAST(1 AS DECIMAL(10,0)))' (interval and decimal(10,0)).; line 1 pos 7 -- !query 21 @@ -208,7 +208,7 @@ select interval 2 day + cast('1' as binary) struct<> -- !query 23 output org.apache.spark.sql.AnalysisException -cannot resolve '(interval 2 days + CAST('1' AS BINARY))' due to data type mismatch: differing types in '(interval 2 days + CAST('1' AS BINARY))' (calendarinterval and binary).; line 1 pos 7 +cannot resolve '(interval 2 days + CAST('1' AS BINARY))' due to data type mismatch: differing types in '(interval 2 days + CAST('1' AS BINARY))' (interval and binary).; line 1 pos 7 -- !query 24 @@ -217,7 +217,7 @@ select interval 2 day + cast(1 as boolean) struct<> -- !query 24 output org.apache.spark.sql.AnalysisException -cannot resolve '(interval 2 days + CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(interval 2 days + CAST(1 AS BOOLEAN))' (calendarinterval and boolean).; line 1 pos 7 +cannot resolve '(interval 2 days + CAST(1 AS BOOLEAN))' due to data type mismatch: differing types in '(interval 2 days + CAST(1 AS BOOLEAN))' (interval and boolean).; line 1 pos 7 -- !query 25 @@ -242,7 +242,7 @@ select cast(1 as tinyint) - interval 2 day struct<> -- !query 27 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS TINYINT) - interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) - interval 2 days)' (tinyint and calendarinterval).; line 1 pos 7 +cannot resolve '(CAST(1 AS TINYINT) - interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS TINYINT) - interval 2 days)' (tinyint and interval).; line 1 pos 7 -- !query 28 @@ -251,7 +251,7 @@ select cast(1 as smallint) - interval 2 day struct<> -- !query 28 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS SMALLINT) - interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) - interval 2 days)' (smallint and calendarinterval).; line 1 pos 7 +cannot resolve '(CAST(1 AS SMALLINT) - interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS SMALLINT) - interval 2 days)' (smallint and interval).; line 1 pos 7 -- !query 29 @@ -260,7 +260,7 @@ select cast(1 as int) - interval 2 day struct<> -- !query 29 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS INT) - interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS INT) - interval 2 days)' (int and calendarinterval).; line 1 pos 7 +cannot resolve '(CAST(1 AS INT) - interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS INT) - interval 2 days)' (int and interval).; line 1 pos 7 -- !query 30 @@ -269,7 +269,7 @@ select cast(1 as bigint) - interval 2 day struct<> -- !query 30 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BIGINT) - interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) - interval 2 days)' (bigint and calendarinterval).; line 1 pos 7 +cannot resolve '(CAST(1 AS BIGINT) - interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS BIGINT) - interval 2 days)' (bigint and interval).; line 1 pos 7 -- !query 31 @@ -278,7 +278,7 @@ select cast(1 as float) - interval 2 day struct<> -- !query 31 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS FLOAT) - interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) - interval 2 days)' (float and calendarinterval).; line 1 pos 7 +cannot resolve '(CAST(1 AS FLOAT) - interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS FLOAT) - interval 2 days)' (float and interval).; line 1 pos 7 -- !query 32 @@ -287,7 +287,7 @@ select cast(1 as double) - interval 2 day struct<> -- !query 32 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DOUBLE) - interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) - interval 2 days)' (double and calendarinterval).; line 1 pos 7 +cannot resolve '(CAST(1 AS DOUBLE) - interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS DOUBLE) - interval 2 days)' (double and interval).; line 1 pos 7 -- !query 33 @@ -296,7 +296,7 @@ select cast(1 as decimal(10, 0)) - interval 2 day struct<> -- !query 33 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS DECIMAL(10,0)) - interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) - interval 2 days)' (decimal(10,0) and calendarinterval).; line 1 pos 7 +cannot resolve '(CAST(1 AS DECIMAL(10,0)) - interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS DECIMAL(10,0)) - interval 2 days)' (decimal(10,0) and interval).; line 1 pos 7 -- !query 34 @@ -321,7 +321,7 @@ select cast('1' as binary) - interval 2 day struct<> -- !query 36 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST('1' AS BINARY) - interval 2 days)' due to data type mismatch: differing types in '(CAST('1' AS BINARY) - interval 2 days)' (binary and calendarinterval).; line 1 pos 7 +cannot resolve '(CAST('1' AS BINARY) - interval 2 days)' due to data type mismatch: differing types in '(CAST('1' AS BINARY) - interval 2 days)' (binary and interval).; line 1 pos 7 -- !query 37 @@ -330,7 +330,7 @@ select cast(1 as boolean) - interval 2 day struct<> -- !query 37 output org.apache.spark.sql.AnalysisException -cannot resolve '(CAST(1 AS BOOLEAN) - interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) - interval 2 days)' (boolean and calendarinterval).; line 1 pos 7 +cannot resolve '(CAST(1 AS BOOLEAN) - interval 2 days)' due to data type mismatch: differing types in '(CAST(1 AS BOOLEAN) - interval 2 days)' (boolean and interval).; line 1 pos 7 -- !query 38 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out index 4fa2032c66c60..5b77bf9f35f25 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out @@ -168,7 +168,7 @@ SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as string) DESC RANGE BETWE struct<> -- !query 20 output org.apache.spark.sql.AnalysisException -cannot resolve 'RANGE BETWEEN CURRENT ROW AND CAST(1 AS STRING) FOLLOWING' due to data type mismatch: The data type of the upper bound 'string' does not match the expected data type '(numeric or calendarinterval)'.; line 1 pos 21 +cannot resolve 'RANGE BETWEEN CURRENT ROW AND CAST(1 AS STRING) FOLLOWING' due to data type mismatch: The data type of the upper bound 'string' does not match the expected data type '(numeric or interval)'.; line 1 pos 21 -- !query 21 @@ -177,7 +177,7 @@ SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('1' as binary) DESC RANGE BET struct<> -- !query 21 output org.apache.spark.sql.AnalysisException -cannot resolve 'RANGE BETWEEN CURRENT ROW AND CAST(1 AS BINARY) FOLLOWING' due to data type mismatch: The data type of the upper bound 'binary' does not match the expected data type '(numeric or calendarinterval)'.; line 1 pos 21 +cannot resolve 'RANGE BETWEEN CURRENT ROW AND CAST(1 AS BINARY) FOLLOWING' due to data type mismatch: The data type of the upper bound 'binary' does not match the expected data type '(numeric or interval)'.; line 1 pos 21 -- !query 22 @@ -186,7 +186,7 @@ SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast(1 as boolean) DESC RANGE BETW struct<> -- !query 22 output org.apache.spark.sql.AnalysisException -cannot resolve 'RANGE BETWEEN CURRENT ROW AND CAST(1 AS BOOLEAN) FOLLOWING' due to data type mismatch: The data type of the upper bound 'boolean' does not match the expected data type '(numeric or calendarinterval)'.; line 1 pos 21 +cannot resolve 'RANGE BETWEEN CURRENT ROW AND CAST(1 AS BOOLEAN) FOLLOWING' due to data type mismatch: The data type of the upper bound 'boolean' does not match the expected data type '(numeric or interval)'.; line 1 pos 21 -- !query 23 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-select_having.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-select_having.sql.out new file mode 100644 index 0000000000000..f731d11c6d3da --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-select_having.sql.out @@ -0,0 +1,187 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 22 + + +-- !query 0 +CREATE TABLE test_having (a int, b int, c string, d string) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO test_having VALUES (0, 1, 'XXXX', 'A') +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO test_having VALUES (1, 2, 'AAAA', 'b') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO test_having VALUES (2, 2, 'AAAA', 'c') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO test_having VALUES (3, 3, 'BBBB', 'D') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO test_having VALUES (4, 3, 'BBBB', 'e') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +INSERT INTO test_having VALUES (5, 3, 'bbbb', 'F') +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +INSERT INTO test_having VALUES (6, 4, 'cccc', 'g') +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +INSERT INTO test_having VALUES (7, 4, 'cccc', 'h') +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +INSERT INTO test_having VALUES (8, 4, 'CCCC', 'I') +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +INSERT INTO test_having VALUES (9, 4, 'CCCC', 'j') +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +SELECT udf(b), udf(c) FROM test_having + GROUP BY b, c HAVING udf(count(*)) = 1 ORDER BY udf(b), udf(c) +-- !query 11 schema +struct +-- !query 11 output +1 XXXX +3 bbbb + + +-- !query 12 +SELECT udf(b), udf(c) FROM test_having + GROUP BY b, c HAVING udf(b) = 3 ORDER BY udf(b), udf(c) +-- !query 12 schema +struct +-- !query 12 output +3 BBBB +3 bbbb + + +-- !query 13 +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 13 schema +struct +-- !query 13 output +XXXX 0 +bbbb 5 + + +-- !query 14 +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 14 schema +struct +-- !query 14 output + + + +-- !query 15 +SELECT udf(min(udf(a))), udf(udf(max(a))) FROM test_having HAVING udf(min(a)) < udf(max(udf(a))) +-- !query 15 schema +struct +-- !query 15 output +0 9 + + +-- !query 16 +SELECT udf(a) FROM test_having HAVING udf(min(a)) < udf(max(a)) +-- !query 16 schema +struct<> +-- !query 16 output +org.apache.spark.sql.AnalysisException +grouping expressions sequence is empty, and 'default.test_having.`a`' is not an aggregate function. Wrap '(min(default.test_having.`a`) AS `min(a#x)`, max(default.test_having.`a`) AS `max(a#x)`)' in windowing function(s) or wrap 'default.test_having.`a`' in first() (or first_value) if you don't care which value you get.; + + +-- !query 17 +SELECT 1 AS one FROM test_having HAVING udf(a) > 1 +-- !query 17 schema +struct<> +-- !query 17 output +org.apache.spark.sql.AnalysisException +cannot resolve '`a`' given input columns: [one]; line 1 pos 44 + + +-- !query 18 +SELECT 1 AS one FROM test_having HAVING udf(udf(1) > udf(2)) +-- !query 18 schema +struct +-- !query 18 output + + + +-- !query 19 +SELECT 1 AS one FROM test_having HAVING udf(udf(1) < udf(2)) +-- !query 19 schema +struct +-- !query 19 output +1 + + +-- !query 20 +SELECT 1 AS one FROM test_having WHERE 1/udf(a) = 1 HAVING 1 < 2 +-- !query 20 schema +struct +-- !query 20 output +1 + + +-- !query 21 +DROP TABLE test_having +-- !query 21 schema +struct<> +-- !query 21 output + diff --git a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-select_implicit.sql.out new file mode 100755 index 0000000000000..e6a5995d24d25 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-select_implicit.sql.out @@ -0,0 +1,420 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 38 + + +-- !query 0 +CREATE TABLE test_missing_target (a int, b int, c string, d string) using parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO test_missing_target VALUES (0, 1, 'XXXX', 'A') +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO test_missing_target VALUES (1, 2, 'ABAB', 'b') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO test_missing_target VALUES (2, 2, 'ABAB', 'c') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO test_missing_target VALUES (3, 3, 'BBBB', 'D') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO test_missing_target VALUES (4, 3, 'BBBB', 'e') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +INSERT INTO test_missing_target VALUES (5, 3, 'bbbb', 'F') +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +INSERT INTO test_missing_target VALUES (6, 4, 'cccc', 'g') +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +INSERT INTO test_missing_target VALUES (7, 4, 'cccc', 'h') +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +INSERT INTO test_missing_target VALUES (8, 4, 'CCCC', 'I') +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +INSERT INTO test_missing_target VALUES (9, 4, 'CCCC', 'j') +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY +test_missing_target.c +ORDER BY udf(c) +-- !query 11 schema +struct +-- !query 11 output +ABAB 2 +BBBB 2 +CCCC 2 +XXXX 1 +bbbb 1 +cccc 2 + + +-- !query 12 +SELECT udf(count(*)) FROM test_missing_target GROUP BY test_missing_target.c +ORDER BY udf(c) +-- !query 12 schema +struct +-- !query 12 output +2 +2 +2 +1 +1 +2 + + +-- !query 13 +SELECT udf(count(*)) FROM test_missing_target GROUP BY a ORDER BY udf(b) +-- !query 13 schema +struct<> +-- !query 13 output +org.apache.spark.sql.AnalysisException +cannot resolve '`b`' given input columns: [CAST(udf(cast(count(1) as string)) AS BIGINT)]; line 1 pos 70 + + +-- !query 14 +SELECT udf(count(*)) FROM test_missing_target GROUP BY b ORDER BY udf(b) +-- !query 14 schema +struct +-- !query 14 output +1 +2 +3 +4 + + +-- !query 15 +SELECT udf(test_missing_target.b), udf(count(*)) + FROM test_missing_target GROUP BY b ORDER BY udf(b) +-- !query 15 schema +struct +-- !query 15 output +1 1 +2 2 +3 3 +4 4 + + +-- !query 16 +SELECT udf(c) FROM test_missing_target ORDER BY udf(a) +-- !query 16 schema +struct +-- !query 16 output +XXXX +ABAB +ABAB +BBBB +BBBB +bbbb +cccc +cccc +CCCC +CCCC + + +-- !query 17 +SELECT udf(count(*)) FROM test_missing_target GROUP BY b ORDER BY udf(b) desc +-- !query 17 schema +struct +-- !query 17 output +4 +3 +2 +1 + + +-- !query 18 +SELECT udf(count(*)) FROM test_missing_target ORDER BY udf(1) desc +-- !query 18 schema +struct +-- !query 18 output +10 + + +-- !query 19 +SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY 1 ORDER BY 1 +-- !query 19 schema +struct +-- !query 19 output +ABAB 2 +BBBB 2 +CCCC 2 +XXXX 1 +bbbb 1 +cccc 2 + + +-- !query 20 +SELECT udf(c), udf(count(*)) FROM test_missing_target GROUP BY 3 +-- !query 20 schema +struct<> +-- !query 20 output +org.apache.spark.sql.AnalysisException +GROUP BY position 3 is not in select list (valid range is [1, 2]); line 1 pos 63 + + +-- !query 21 +SELECT udf(count(*)) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY b ORDER BY udf(b) +-- !query 21 schema +struct<> +-- !query 21 output +org.apache.spark.sql.AnalysisException +Reference 'b' is ambiguous, could be: x.b, y.b.; line 3 pos 10 + + +-- !query 22 +SELECT udf(a), udf(a) FROM test_missing_target + ORDER BY udf(a) +-- !query 22 schema +struct +-- !query 22 output +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 + + +-- !query 23 +SELECT udf(udf(a)/2), udf(udf(a)/2) FROM test_missing_target + ORDER BY udf(udf(a)/2) +-- !query 23 schema +struct +-- !query 23 output +0 0 +0 0 +1 1 +1 1 +2 2 +2 2 +3 3 +3 3 +4 4 +4 4 + + +-- !query 24 +SELECT udf(a/2), udf(a/2) FROM test_missing_target + GROUP BY a/2 ORDER BY udf(a/2) +-- !query 24 schema +struct +-- !query 24 output +0 0 +1 1 +2 2 +3 3 +4 4 + + +-- !query 25 +SELECT udf(x.b), udf(count(*)) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY x.b ORDER BY udf(x.b) +-- !query 25 schema +struct +-- !query 25 output +1 1 +2 2 +3 3 +4 4 + + +-- !query 26 +SELECT udf(count(*)) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY x.b ORDER BY udf(x.b) +-- !query 26 schema +struct +-- !query 26 output +1 +2 +3 +4 + + +-- !query 27 +SELECT a%2, udf(count(udf(b))) FROM test_missing_target +GROUP BY test_missing_target.a%2 +ORDER BY udf(test_missing_target.a%2) +-- !query 27 schema +struct<(a % 2):int,CAST(udf(cast(count(cast(udf(cast(b as string)) as int)) as string)) AS BIGINT):bigint> +-- !query 27 output +0 5 +1 5 + + +-- !query 28 +SELECT udf(count(c)) FROM test_missing_target +GROUP BY lower(test_missing_target.c) +ORDER BY udf(lower(test_missing_target.c)) +-- !query 28 schema +struct +-- !query 28 output +2 +3 +4 +1 + + +-- !query 29 +SELECT udf(count(udf(a))) FROM test_missing_target GROUP BY a ORDER BY udf(b) +-- !query 29 schema +struct<> +-- !query 29 output +org.apache.spark.sql.AnalysisException +cannot resolve '`b`' given input columns: [CAST(udf(cast(count(cast(udf(cast(a as string)) as int)) as string)) AS BIGINT)]; line 1 pos 75 + + +-- !query 30 +SELECT udf(count(b)) FROM test_missing_target GROUP BY b/2 ORDER BY udf(b/2) +-- !query 30 schema +struct +-- !query 30 output +1 +5 +4 + + +-- !query 31 +SELECT udf(lower(test_missing_target.c)), udf(count(udf(c))) + FROM test_missing_target GROUP BY lower(c) ORDER BY udf(lower(c)) +-- !query 31 schema +struct +-- !query 31 output +abab 2 +bbbb 3 +cccc 4 +xxxx 1 + + +-- !query 32 +SELECT udf(a) FROM test_missing_target ORDER BY udf(upper(udf(d))) +-- !query 32 schema +struct +-- !query 32 output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query 33 +SELECT udf(count(b)) FROM test_missing_target + GROUP BY (b + 1) / 2 ORDER BY udf((b + 1) / 2) desc +-- !query 33 schema +struct +-- !query 33 output +7 +3 + + +-- !query 34 +SELECT udf(count(udf(x.a))) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY b/2 ORDER BY udf(b/2) +-- !query 34 schema +struct<> +-- !query 34 output +org.apache.spark.sql.AnalysisException +Reference 'b' is ambiguous, could be: x.b, y.b.; line 3 pos 10 + + +-- !query 35 +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 x.b/2 ORDER BY udf(x.b/2) +-- !query 35 schema +struct +-- !query 35 output +0 1 +1 5 +2 4 + + +-- !query 36 +SELECT udf(count(udf(b))) FROM test_missing_target x, test_missing_target y + WHERE udf(x.a) = udf(y.a) + GROUP BY x.b/2 +-- !query 36 schema +struct<> +-- !query 36 output +org.apache.spark.sql.AnalysisException +Reference 'b' is ambiguous, could be: x.b, y.b.; line 1 pos 21 + + +-- !query 37 +DROP TABLE test_missing_target +-- !query 37 schema +struct<> +-- !query 37 output + diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-cross-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-cross-join.sql.out new file mode 100644 index 0000000000000..98d3ad37a8dfa --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-cross-join.sql.out @@ -0,0 +1,140 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 13 + + +-- !query 0 +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT * FROM nt1 cross join nt2 +-- !query 2 schema +struct +-- !query 2 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 3 +SELECT * FROM nt1 cross join nt2 where udf(nt1.k) = udf(nt2.k) +-- !query 3 schema +struct +-- !query 3 output +one 1 one 1 +one 1 one 5 +two 2 two 22 + + +-- !query 4 +SELECT * FROM nt1 cross join nt2 on (udf(nt1.k) = udf(nt2.k)) +-- !query 4 schema +struct +-- !query 4 output +one 1 one 1 +one 1 one 5 +two 2 two 22 + + +-- !query 5 +SELECT * FROM nt1 cross join nt2 where udf(nt1.v1) = "1" and udf(nt2.v2) = "22" +-- !query 5 schema +struct +-- !query 5 output +one 1 two 22 + + +-- !query 6 +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 6 schema +struct +-- !query 6 output +one two + + +-- !query 7 +create temporary view A(a, va) as select * from nt1 +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +create temporary view B(b, vb) as select * from nt1 +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +create temporary view C(c, vc) as select * from nt1 +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +create temporary view D(d, vd) as select * from nt1 +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +select * from ((A join B on (udf(a) = udf(b))) cross join C) join D on (udf(a) = udf(d)) +-- !query 11 schema +struct +-- !query 11 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 12 +SELECT * FROM nt1 CROSS JOIN nt2 ON (udf(nt1.k) > udf(nt2.k)) +-- !query 12 schema +struct +-- !query 12 output +three 3 one 1 +three 3 one 5 +two 2 one 1 +two 2 one 5 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out new file mode 100644 index 0000000000000..b7bfad0e538ac --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out @@ -0,0 +1,346 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 27 + + +-- !query 0 +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (0), (1), (2), (2), (2), (2), (3), (null), (null) AS tab1(c1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1), (2), (2), (3), (5), (5), (null) AS tab2(c1) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW tab3 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (2, 3), + (2, 2) + AS tab3(k, v) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +CREATE TEMPORARY VIEW tab4 AS SELECT * FROM VALUES + (1, 2), + (2, 3), + (2, 2), + (2, 2), + (2, 20) + AS tab4(k, v) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 +-- !query 4 schema +struct +-- !query 4 output +0 +2 +2 +NULL + + +-- !query 5 +SELECT udf(c1) FROM tab1 +MINUS ALL +SELECT udf(c1) FROM tab2 +-- !query 5 schema +struct +-- !query 5 output +0 +2 +2 +NULL + + +-- !query 6 +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 WHERE udf(c1) IS NOT NULL +-- !query 6 schema +struct +-- !query 6 output +0 +2 +2 +NULL +NULL + + +-- !query 7 +SELECT udf(c1) FROM tab1 WHERE udf(c1) > 5 +EXCEPT ALL +SELECT udf(c1) FROM tab2 +-- !query 7 schema +struct +-- !query 7 output + + + +-- !query 8 +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 WHERE udf(c1 > udf(6)) +-- !query 8 schema +struct +-- !query 8 output +0 +1 +2 +2 +2 +2 +3 +NULL +NULL + + +-- !query 9 +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT CAST(udf(1) AS BIGINT) +-- !query 9 schema +struct +-- !query 9 output +0 +2 +2 +2 +2 +3 +NULL +NULL + + +-- !query 10 +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT array(1) +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.AnalysisException +ExceptAll can only be performed on tables with the compatible column types. array <> int at the first column of the second table; + + +-- !query 11 +SELECT udf(k), v FROM tab3 +EXCEPT ALL +SELECT k, udf(v) FROM tab4 +-- !query 11 schema +struct +-- !query 11 output +1 2 +1 3 + + +-- !query 12 +SELECT k, udf(v) FROM tab4 +EXCEPT ALL +SELECT udf(k), v FROM tab3 +-- !query 12 schema +struct +-- !query 12 output +2 2 +2 20 + + +-- !query 13 +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 13 schema +struct +-- !query 13 output +2 2 +2 20 + + +-- !query 14 +SELECT udf(k), v FROM tab4 +EXCEPT ALL +SELECT k, udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), udf(v) FROM tab4 +-- !query 14 schema +struct +-- !query 14 output + + + +-- !query 15 +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 15 schema +struct +-- !query 15 output +1 3 + + +-- !query 16 +SELECT k FROM tab3 +EXCEPT ALL +SELECT k, v FROM tab4 +-- !query 16 schema +struct<> +-- !query 16 output +org.apache.spark.sql.AnalysisException +ExceptAll can only be performed on tables with the same number of columns, but the first table has 1 columns and the second table has 2 columns; + + +-- !query 17 +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 17 schema +struct +-- !query 17 output +1 3 + + +-- !query 18 +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 18 schema +struct +-- !query 18 output +1 3 + + +-- !query 19 +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 19 schema +struct +-- !query 19 output + + + +-- !query 20 +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 20 schema +struct +-- !query 20 output + + + +-- !query 21 +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 21 schema +struct +-- !query 21 output +1 2 +1 2 +1 2 +2 20 +2 20 +2 3 +2 3 + + +-- !query 22 +SELECT udf(v) FROM tab3 GROUP BY v +EXCEPT ALL +SELECT udf(k) FROM tab4 GROUP BY k +-- !query 22 schema +struct +-- !query 22 output +3 + + +-- !query 23 +DROP VIEW IF EXISTS tab1 +-- !query 23 schema +struct<> +-- !query 23 output + + + +-- !query 24 +DROP VIEW IF EXISTS tab2 +-- !query 24 schema +struct<> +-- !query 24 output + + + +-- !query 25 +DROP VIEW IF EXISTS tab3 +-- !query 25 schema +struct<> +-- !query 25 output + + + +-- !query 26 +DROP VIEW IF EXISTS tab4 +-- !query 26 schema +struct<> +-- !query 26 output + diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out new file mode 100644 index 0000000000000..cb8a4e84a95cf --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out @@ -0,0 +1,87 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 8 + + +-- !query 0 +create temporary view t1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", NULL) + as t1(k, v) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5), + ("one", NULL), + (NULL, 5) + as t2(k, v) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT udf(k), udf(v) FROM t1 EXCEPT SELECT udf(k), udf(v) FROM t2 +-- !query 2 schema +struct +-- !query 2 output +three 3 +two 2 + + +-- !query 3 +SELECT * FROM t1 EXCEPT SELECT * FROM t1 where udf(v) <> 1 and v <> udf(2) +-- !query 3 schema +struct +-- !query 3 output +one 1 +one NULL +two 2 + + +-- !query 4 +SELECT * FROM t1 where udf(v) <> 1 and v <> udf(22) EXCEPT SELECT * FROM t1 where udf(v) <> 2 and v >= udf(3) +-- !query 4 schema +struct +-- !query 4 output +two 2 + + +-- !query 5 +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 5 schema +struct +-- !query 5 output +one 1 +one NULL + + +-- !query 6 +SELECT * FROM t2 where v >= udf(1) and udf(v) <> 22 EXCEPT SELECT * FROM t1 +-- !query 6 schema +struct +-- !query 6 output +NULL 5 +one 5 + + +-- !query 7 +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 7 schema +struct +-- !query 7 output +NULL +two diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out new file mode 100644 index 0000000000000..31e9e08e2ca8c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out @@ -0,0 +1,394 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 29 + + +-- !query 0 +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 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT a + b, b, udf(SUM(a - b)) FROM testData GROUP BY a + b, b WITH CUBE +-- !query 1 schema +struct<(a + b):int,b:int,CAST(udf(cast(sum(cast((a - b) as bigint)) as string)) AS BIGINT):bigint> +-- !query 1 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 2 +SELECT a, udf(b), SUM(b) FROM testData GROUP BY a, b WITH CUBE +-- !query 2 schema +struct +-- !query 2 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 3 +SELECT udf(a + b), b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP +-- !query 3 schema +struct +-- !query 3 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 4 +SELECT a, b, udf(SUM(b)) FROM testData GROUP BY a, b WITH ROLLUP +-- !query 4 schema +struct +-- !query 4 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 5 +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 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY udf(course), year +-- !query 6 schema +struct +-- !query 6 output +NULL NULL 113000 +Java NULL 50000 +Java 2012 20000 +Java 2013 30000 +dotNET NULL 63000 +dotNET 2012 15000 +dotNET 2013 48000 + + +-- !query 7 +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, udf(year) +-- !query 7 schema +struct +-- !query 7 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 8 +SELECT course, udf(year), SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year) +-- !query 8 schema +struct +-- !query 8 output +Java NULL 50000 +NULL 2012 35000 +NULL 2013 78000 +dotNET NULL 63000 + + +-- !query 9 +SELECT course, year, udf(SUM(earnings)) FROM courseSales GROUP BY course, year GROUPING SETS(course) +-- !query 9 schema +struct +-- !query 9 output +Java NULL 50000 +dotNET NULL 63000 + + +-- !query 10 +SELECT udf(course), year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year) +-- !query 10 schema +struct +-- !query 10 output +NULL 2012 35000 +NULL 2013 78000 + + +-- !query 11 +SELECT course, udf(SUM(earnings)) AS sum FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, udf(sum) +-- !query 11 schema +struct +-- !query 11 output +NULL 113000 +Java 20000 +Java 30000 +Java 50000 +dotNET 5000 +dotNET 10000 +dotNET 48000 +dotNET 63000 + + +-- !query 12 +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 12 schema +struct +-- !query 12 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 13 +SELECT udf(course), udf(year), GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales +GROUP BY CUBE(course, year) +-- !query 13 schema +struct +-- !query 13 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 14 +SELECT course, udf(year), GROUPING(course) FROM courseSales GROUP BY course, year +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.AnalysisException +grouping() can only be used with GroupingSets/Cube/Rollup; + + +-- !query 15 +SELECT course, udf(year), GROUPING_ID(course, year) FROM courseSales GROUP BY course, year +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.AnalysisException +grouping_id() can only be used with GroupingSets/Cube/Rollup; + + +-- !query 16 +SELECT course, year, grouping__id FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, udf(year) +-- !query 16 schema +struct +-- !query 16 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 17 +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 17 schema +struct +-- !query 17 output +NULL NULL +Java NULL +dotNET NULL + + +-- !query 18 +SELECT course, udf(year) FROM courseSales GROUP BY course, year HAVING GROUPING(course) > 0 +-- !query 18 schema +struct<> +-- !query 18 output +org.apache.spark.sql.AnalysisException +grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; + + +-- !query 19 +SELECT course, udf(udf(year)) FROM courseSales GROUP BY course, year HAVING GROUPING_ID(course) > 0 +-- !query 19 schema +struct<> +-- !query 19 output +org.apache.spark.sql.AnalysisException +grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; + + +-- !query 20 +SELECT udf(course), year FROM courseSales GROUP BY CUBE(course, year) HAVING grouping__id > 0 +-- !query 20 schema +struct +-- !query 20 output +Java NULL +NULL 2012 +NULL 2013 +NULL NULL +dotNET NULL + + +-- !query 21 +SELECT course, year, GROUPING(course), GROUPING(year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, udf(year) +-- !query 21 schema +struct +-- !query 21 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 22 +SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, udf(year) +-- !query 22 schema +struct +-- !query 22 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 23 +SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GROUPING(course) +-- !query 23 schema +struct<> +-- !query 23 output +org.apache.spark.sql.AnalysisException +grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; + + +-- !query 24 +SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GROUPING_ID(course) +-- !query 24 schema +struct<> +-- !query 24 output +org.apache.spark.sql.AnalysisException +grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; + + +-- !query 25 +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, udf(course), year +-- !query 25 schema +struct +-- !query 25 output +Java 2012 +Java 2013 +dotNET 2012 +dotNET 2013 +Java NULL +dotNET NULL +NULL 2012 +NULL 2013 +NULL NULL + + +-- !query 26 +SELECT udf(a + b) AS k1, udf(b) AS k2, SUM(a - b) FROM testData GROUP BY CUBE(k1, k2) +-- !query 26 schema +struct +-- !query 26 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 27 +SELECT udf(udf(a + b)) AS k, b, SUM(a - b) FROM testData GROUP BY ROLLUP(k, b) +-- !query 27 schema +struct +-- !query 27 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 28 +SELECT udf(a + b), udf(udf(b)) AS k, SUM(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k) +-- !query 28 schema +struct +-- !query 28 output +NULL 1 3 +NULL 2 0 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out new file mode 100644 index 0000000000000..789c91fa3870b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out @@ -0,0 +1,512 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 51 + + +-- !query 0 +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 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT udf(a), udf(COUNT(b)) FROM testData +-- !query 1 schema +struct<> +-- !query 1 output +org.apache.spark.sql.AnalysisException +grouping expressions sequence is empty, and 'testdata.`a`' is not an aggregate function. Wrap '(CAST(udf(cast(count(b) as string)) AS BIGINT) AS `CAST(udf(cast(count(b) as string)) AS BIGINT)`)' in windowing function(s) or wrap 'testdata.`a`' in first() (or first_value) if you don't care which value you get.; + + +-- !query 2 +SELECT COUNT(udf(a)), udf(COUNT(b)) FROM testData +-- !query 2 schema +struct +-- !query 2 output +7 7 + + +-- !query 3 +SELECT udf(a), COUNT(udf(b)) FROM testData GROUP BY a +-- !query 3 schema +struct +-- !query 3 output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query 4 +SELECT udf(a), udf(COUNT(udf(b))) FROM testData GROUP BY b +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.AnalysisException +expression 'testdata.`a`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; + + +-- !query 5 +SELECT COUNT(udf(a)), COUNT(udf(b)) FROM testData GROUP BY udf(a) +-- !query 5 schema +struct +-- !query 5 output +0 1 +2 2 +2 2 +3 2 + + +-- !query 6 +SELECT 'foo', COUNT(udf(a)) FROM testData GROUP BY 1 +-- !query 6 schema +struct +-- !query 6 output +foo 7 + + +-- !query 7 +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY udf(1) +-- !query 7 schema +struct +-- !query 7 output + + + +-- !query 8 +SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY 1 +-- !query 8 schema +struct +-- !query 8 output + + + +-- !query 9 +SELECT 'foo', MAX(STRUCT(udf(a))) FROM testData WHERE a = 0 GROUP BY 1 +-- !query 9 schema +struct> +-- !query 9 output + + + +-- !query 10 +SELECT udf(a + b), udf(COUNT(b)) FROM testData GROUP BY a + b +-- !query 10 schema +struct +-- !query 10 output +2 1 +3 2 +4 2 +5 1 +NULL 1 + + +-- !query 11 +SELECT udf(a + 2), udf(COUNT(b)) FROM testData GROUP BY a + 1 +-- !query 11 schema +struct<> +-- !query 11 output +org.apache.spark.sql.AnalysisException +expression 'testdata.`a`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; + + +-- !query 12 +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 12 schema +struct +-- !query 12 output +-0.2723801058145729 -1.5069204152249134 1 3 2.142857142857143 0.8095238095238094 0.8997354108424372 15 7 + + +-- !query 13 +SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a +-- !query 13 schema +struct +-- !query 13 output +1 1 + + +-- !query 14 +SELECT a AS k, COUNT(udf(b)) FROM testData GROUP BY k +-- !query 14 schema +struct +-- !query 14 output +1 2 +2 2 +3 2 +NULL 1 + + +-- !query 15 +SELECT a AS k, udf(COUNT(b)) FROM testData GROUP BY k HAVING k > 1 +-- !query 15 schema +struct +-- !query 15 output +2 2 +3 2 + + +-- !query 16 +SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k +-- !query 16 schema +struct<> +-- !query 16 output +org.apache.spark.sql.AnalysisException +aggregate functions are not allowed in GROUP BY, but found CAST(udf(cast(count(b) as string)) AS BIGINT); + + +-- !query 17 +CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES +(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v) +-- !query 17 schema +struct<> +-- !query 17 output + + + +-- !query 18 +SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY a +-- !query 18 schema +struct<> +-- !query 18 output +org.apache.spark.sql.AnalysisException +expression 'testdatahassamenamewithalias.`k`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; + + +-- !query 19 +set spark.sql.groupByAliases=false +-- !query 19 schema +struct +-- !query 19 output +spark.sql.groupByAliases false + + +-- !query 20 +SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k +-- !query 20 schema +struct<> +-- !query 20 output +org.apache.spark.sql.AnalysisException +cannot resolve '`k`' given input columns: [testdata.a, testdata.b]; line 1 pos 57 + + +-- !query 21 +SELECT a, COUNT(udf(1)) FROM testData WHERE false GROUP BY a +-- !query 21 schema +struct +-- !query 21 output + + + +-- !query 22 +SELECT udf(COUNT(1)) FROM testData WHERE false +-- !query 22 schema +struct +-- !query 22 output +0 + + +-- !query 23 +SELECT 1 FROM (SELECT udf(COUNT(1)) FROM testData WHERE false) t +-- !query 23 schema +struct<1:int> +-- !query 23 output +1 + + +-- !query 24 +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 24 schema +struct<1:int> +-- !query 24 output + + + +-- !query 25 +SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*) + FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y) +-- !query 25 schema +struct +-- !query 25 output +1.0 1.0 3 + + +-- !query 26 +SELECT udf(1) FROM range(10) HAVING true +-- !query 26 schema +struct +-- !query 26 output +1 + + +-- !query 27 +SELECT udf(udf(1)) FROM range(10) HAVING MAX(id) > 0 +-- !query 27 schema +struct +-- !query 27 output +1 + + +-- !query 28 +SELECT udf(id) FROM range(10) HAVING id > 0 +-- !query 28 schema +struct<> +-- !query 28 output +org.apache.spark.sql.AnalysisException +grouping expressions sequence is empty, and '`id`' is not an aggregate function. Wrap '()' in windowing function(s) or wrap '`id`' in first() (or first_value) if you don't care which value you get.; + + +-- !query 29 +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 29 schema +struct<> +-- !query 29 output + + + +-- !query 30 +SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0 +-- !query 30 schema +struct +-- !query 30 output +NULL NULL NULL + + +-- !query 31 +SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4 +-- !query 31 schema +struct +-- !query 31 output +NULL NULL NULL + + +-- !query 32 +SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5 +-- !query 32 schema +struct +-- !query 32 output +false true true + + +-- !query 33 +SELECT k, every(v), udf(some(v)), any(v) FROM test_agg GROUP BY k +-- !query 33 schema +struct +-- !query 33 output +1 false true true +2 true true true +3 false false false +4 NULL NULL NULL +5 false true true + + +-- !query 34 +SELECT udf(k), every(v) FROM test_agg GROUP BY k HAVING every(v) = false +-- !query 34 schema +struct +-- !query 34 output +1 false +3 false +5 false + + +-- !query 35 +SELECT k, udf(every(v)) FROM test_agg GROUP BY k HAVING every(v) IS NULL +-- !query 35 schema +struct +-- !query 35 output +4 NULL + + +-- !query 36 +SELECT 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 k +-- !query 36 schema +struct +-- !query 36 output +2 true + + +-- !query 37 +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 k +-- !query 37 schema +struct +-- !query 37 output + + + +-- !query 38 +SELECT every(udf(1)) +-- !query 38 schema +struct<> +-- !query 38 output +org.apache.spark.sql.AnalysisException +cannot resolve 'every(CAST(udf(cast(1 as string)) AS INT))' due to data type mismatch: Input to function 'every' should have been boolean, but it's [int].; line 1 pos 7 + + +-- !query 39 +SELECT some(udf(1S)) +-- !query 39 schema +struct<> +-- !query 39 output +org.apache.spark.sql.AnalysisException +cannot resolve 'some(CAST(udf(cast(1 as string)) AS SMALLINT))' due to data type mismatch: Input to function 'some' should have been boolean, but it's [smallint].; line 1 pos 7 + + +-- !query 40 +SELECT any(udf(1L)) +-- !query 40 schema +struct<> +-- !query 40 output +org.apache.spark.sql.AnalysisException +cannot resolve 'any(CAST(udf(cast(1 as string)) AS BIGINT))' due to data type mismatch: Input to function 'any' should have been boolean, but it's [bigint].; line 1 pos 7 + + +-- !query 41 +SELECT udf(every("true")) +-- !query 41 schema +struct<> +-- !query 41 output +org.apache.spark.sql.AnalysisException +cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 11 + + +-- !query 42 +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query 42 schema +struct +-- !query 42 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 43 +SELECT k, udf(udf(v)), some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query 43 schema +struct +-- !query 43 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 44 +SELECT udf(udf(k)), v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query 44 schema +struct +-- !query 44 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 45 +SELECT udf(count(*)) FROM test_agg HAVING count(*) > 1L +-- !query 45 schema +struct +-- !query 45 output +10 + + +-- !query 46 +SELECT k, udf(max(v)) FROM test_agg GROUP BY k HAVING max(v) = true +-- !query 46 schema +struct +-- !query 46 output +1 true +2 true +5 true + + +-- !query 47 +SELECT * FROM (SELECT udf(COUNT(*)) AS cnt FROM test_agg) WHERE cnt > 1L +-- !query 47 schema +struct +-- !query 47 output +10 + + +-- !query 48 +SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L +-- !query 48 schema +struct<> +-- !query 48 output +org.apache.spark.sql.AnalysisException + +Aggregate/Window/Generate expressions are not valid in where clause of the query. +Expression in where clause: [(count(1) > 1L)] +Invalid expressions: [count(1)]; + + +-- !query 49 +SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L +-- !query 49 schema +struct<> +-- !query 49 output +org.apache.spark.sql.AnalysisException + +Aggregate/Window/Generate expressions are not valid in where clause of the query. +Expression in where clause: [((count(1) + 1L) > 1L)] +Invalid expressions: [count(1)]; + + +-- !query 50 +SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 +-- !query 50 schema +struct<> +-- !query 50 output +org.apache.spark.sql.AnalysisException + +Aggregate/Window/Generate expressions are not valid in where clause of the query. +Expression in where clause: [(((test_agg.`k` = 1) OR (test_agg.`k` = 2)) OR (((count(1) + 1L) > 1L) OR (max(test_agg.`k`) > 1)))] +Invalid expressions: [count(1), max(test_agg.`k`)]; diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out new file mode 100644 index 0000000000000..2cf24e50c80a5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out @@ -0,0 +1,153 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 17 + + +-- !query 0 +select udf(col1), udf(col2) from values ("one", 1) +-- !query 0 schema +struct +-- !query 0 output +one 1 + + +-- !query 1 +select udf(col1), udf(udf(col2)) from values ("one", 1) as data +-- !query 1 schema +struct +-- !query 1 output +one 1 + + +-- !query 2 +select udf(a), b from values ("one", 1) as data(a, b) +-- !query 2 schema +struct +-- !query 2 output +one 1 + + +-- !query 3 +select udf(a) from values 1, 2, 3 as data(a) +-- !query 3 schema +struct +-- !query 3 output +1 +2 +3 + + +-- !query 4 +select udf(a), b from values ("one", 1), ("two", 2), ("three", null) as data(a, b) +-- !query 4 schema +struct +-- !query 4 output +one 1 +three NULL +two 2 + + +-- !query 5 +select udf(a), b from values ("one", null), ("two", null) as data(a, b) +-- !query 5 schema +struct +-- !query 5 output +one NULL +two NULL + + +-- !query 6 +select udf(a), b from values ("one", 1), ("two", 2L) as data(a, b) +-- !query 6 schema +struct +-- !query 6 output +one 1 +two 2 + + +-- !query 7 +select udf(udf(a)), udf(b) from values ("one", 1 + 0), ("two", 1 + 3L) as data(a, b) +-- !query 7 schema +struct +-- !query 7 output +one 1 +two 4 + + +-- !query 8 +select udf(a), b from values ("one", array(0, 1)), ("two", array(2, 3)) as data(a, b) +-- !query 8 schema +struct> +-- !query 8 output +one [0,1] +two [2,3] + + +-- !query 9 +select udf(a), b from values ("one", 2.0), ("two", 3.0D) as data(a, b) +-- !query 9 schema +struct +-- !query 9 output +one 2.0 +two 3.0 + + +-- !query 10 +select udf(a), b from values ("one", rand(5)), ("two", 3.0D) as data(a, b) +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.AnalysisException +cannot evaluate expression rand(5) in inline table definition; line 1 pos 37 + + +-- !query 11 +select udf(a), udf(b) from values ("one", 2.0), ("two") as data(a, b) +-- !query 11 schema +struct<> +-- !query 11 output +org.apache.spark.sql.AnalysisException +expected 2 columns but found 1 columns in row 1; line 1 pos 27 + + +-- !query 12 +select udf(a), udf(b) from values ("one", array(0, 1)), ("two", struct(1, 2)) as data(a, b) +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.AnalysisException +incompatible types found in column b for inline table; line 1 pos 27 + + +-- !query 13 +select udf(a), udf(b) from values ("one"), ("two") as data(a, b) +-- !query 13 schema +struct<> +-- !query 13 output +org.apache.spark.sql.AnalysisException +expected 2 columns but found 1 columns in row 0; line 1 pos 27 + + +-- !query 14 +select udf(a), udf(b) from values ("one", random_not_exist_func(1)), ("two", 2) as data(a, b) +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.AnalysisException +Undefined function: 'random_not_exist_func'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 42 + + +-- !query 15 +select udf(a), udf(b) from values ("one", count(1)), ("two", 2) as data(a, b) +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.AnalysisException +cannot evaluate expression count(1) in inline table definition; line 1 pos 42 + + +-- !query 16 +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 16 schema +struct> +-- !query 16 output +1991-12-06 00:00:00 [1991-12-06 01:00:00.0,1991-12-06 12:00:00.0] diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out new file mode 100644 index 0000000000000..0cb82be2da228 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out @@ -0,0 +1,307 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 22 + + +-- !query 0 +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 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +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 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT k, udf(v) FROM tab2 +-- !query 2 schema +struct +-- !query 2 output +1 2 +1 2 +2 3 +NULL NULL +NULL NULL + + +-- !query 3 +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab1 WHERE udf(k) = 1 +-- !query 3 schema +struct +-- !query 3 output +1 2 +1 2 +1 3 +1 3 + + +-- !query 4 +SELECT udf(k), udf(v) FROM tab1 WHERE k > udf(2) +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2 +-- !query 4 schema +struct +-- !query 4 output + + + +-- !query 5 +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab2 WHERE udf(udf(k)) > 3 +-- !query 5 schema +struct +-- !query 5 output + + + +-- !query 6 +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT CAST(udf(1) AS BIGINT), CAST(udf(2) AS BIGINT) +-- !query 6 schema +struct +-- !query 6 output +1 2 + + +-- !query 7 +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT array(1), udf(2) +-- !query 7 schema +struct<> +-- !query 7 output +org.apache.spark.sql.AnalysisException +IntersectAll can only be performed on tables with the compatible column types. array <> int at the first column of the second table; + + +-- !query 8 +SELECT udf(k) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2 +-- !query 8 schema +struct<> +-- !query 8 output +org.apache.spark.sql.AnalysisException +IntersectAll can only be performed on tables with the same number of columns, but the first table has 1 columns and the second table has 2 columns; + + +-- !query 9 +SELECT udf(k), v FROM tab2 +INTERSECT ALL +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2 +-- !query 9 schema +struct +-- !query 9 output +1 2 +1 2 +2 3 +NULL NULL +NULL NULL + + +-- !query 10 +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 10 schema +struct +-- !query 10 output +1 2 +1 2 +1 3 +2 3 +NULL NULL +NULL NULL + + +-- !query 11 +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 11 schema +struct +-- !query 11 output +1 3 + + +-- !query 12 +( + ( + ( + 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 12 schema +struct +-- !query 12 output + + + +-- !query 13 +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 13 schema +struct +-- !query 13 output +1 2 +1 2 +1 2 +1 2 +1 2 +1 2 +1 2 +1 2 +2 3 + + +-- !query 14 +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 14 schema +struct +-- !query 14 output + + + +-- !query 15 +SELECT udf(v) FROM tab1 GROUP BY v +INTERSECT ALL +SELECT udf(udf(k)) FROM tab2 GROUP BY k +-- !query 15 schema +struct +-- !query 15 output +2 +3 +NULL + + +-- !query 16 +SET spark.sql.legacy.setopsPrecedence.enabled= true +-- !query 16 schema +struct +-- !query 16 output +spark.sql.legacy.setopsPrecedence.enabled true + + +-- !query 17 +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 17 schema +struct +-- !query 17 output +1 2 +1 2 +2 3 +NULL NULL +NULL NULL + + +-- !query 18 +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 18 schema +struct +-- !query 18 output +1 2 +2 3 +NULL NULL + + +-- !query 19 +SET spark.sql.legacy.setopsPrecedence.enabled = false +-- !query 19 schema +struct +-- !query 19 output +spark.sql.legacy.setopsPrecedence.enabled false + + +-- !query 20 +DROP VIEW IF EXISTS tab1 +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +DROP VIEW IF EXISTS tab2 +-- !query 21 schema +struct<> +-- !query 21 output + diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-join-empty-relation.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-join-empty-relation.sql.out new file mode 100644 index 0000000000000..e79d01fb14d60 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-join-empty-relation.sql.out @@ -0,0 +1,194 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 24 + + +-- !query 0 +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW empty_table as SELECT a FROM t2 WHERE false +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT udf(t1.a), udf(empty_table.a) FROM t1 INNER JOIN empty_table ON (udf(t1.a) = udf(udf(empty_table.a))) +-- !query 3 schema +struct +-- !query 3 output + + + +-- !query 4 +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 4 schema +struct +-- !query 4 output + + + +-- !query 5 +SELECT udf(udf(t1.a)), empty_table.a FROM t1 LEFT OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) +-- !query 5 schema +struct +-- !query 5 output +1 NULL + + +-- !query 6 +SELECT udf(t1.a), udf(empty_table.a) FROM t1 RIGHT OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) +-- !query 6 schema +struct +-- !query 6 output + + + +-- !query 7 +SELECT udf(t1.a), empty_table.a FROM t1 FULL OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) +-- !query 7 schema +struct +-- !query 7 output +1 NULL + + +-- !query 8 +SELECT udf(udf(t1.a)) FROM t1 LEFT SEMI JOIN empty_table ON (udf(t1.a) = udf(udf(empty_table.a))) +-- !query 8 schema +struct +-- !query 8 output + + + +-- !query 9 +SELECT udf(t1.a) FROM t1 LEFT ANTI JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) +-- !query 9 schema +struct +-- !query 9 output +1 + + +-- !query 10 +SELECT udf(empty_table.a), udf(t1.a) FROM empty_table INNER JOIN t1 ON (udf(udf(empty_table.a)) = udf(t1.a)) +-- !query 10 schema +struct +-- !query 10 output + + + +-- !query 11 +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 11 schema +struct +-- !query 11 output + + + +-- !query 12 +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 12 schema +struct +-- !query 12 output + + + +-- !query 13 +SELECT empty_table.a, udf(t1.a) FROM empty_table RIGHT OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) +-- !query 13 schema +struct +-- !query 13 output +NULL 1 + + +-- !query 14 +SELECT empty_table.a, udf(udf(t1.a)) FROM empty_table FULL OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) +-- !query 14 schema +struct +-- !query 14 output +NULL 1 + + +-- !query 15 +SELECT udf(udf(empty_table.a)) FROM empty_table LEFT SEMI JOIN t1 ON (udf(empty_table.a) = udf(udf(t1.a))) +-- !query 15 schema +struct +-- !query 15 output + + + +-- !query 16 +SELECT empty_table.a FROM empty_table LEFT ANTI JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) +-- !query 16 schema +struct +-- !query 16 output + + + +-- !query 17 +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 17 schema +struct +-- !query 17 output + + + +-- !query 18 +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 18 schema +struct +-- !query 18 output + + + +-- !query 19 +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 19 schema +struct +-- !query 19 output + + + +-- !query 20 +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 20 schema +struct +-- !query 20 output + + + +-- !query 21 +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 21 schema +struct +-- !query 21 output + + + +-- !query 22 +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 22 schema +struct +-- !query 22 output + + + +-- !query 23 +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 23 schema +struct +-- !query 23 output + diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-outer-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-outer-join.sql.out new file mode 100644 index 0000000000000..819f786070882 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-outer-join.sql.out @@ -0,0 +1,88 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 8 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(-234), (145), (367), (975), (298) +as t1(int_col1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +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 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +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 2 schema +struct +-- !query 2 output +-367 -734 +-507 -1014 +-769 -1538 +-800 -1600 + + +-- !query 3 +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (97) as t1(int_col1) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (0) as t2(int_col1) +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +set spark.sql.crossJoin.enabled = true +-- !query 5 schema +struct +-- !query 5 output +spark.sql.crossJoin.enabled true + + +-- !query 6 +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 6 schema +struct +-- !query 6 output +97 + + +-- !query 7 +set spark.sql.crossJoin.enabled = false +-- !query 7 schema +struct +-- !query 7 output +spark.sql.crossJoin.enabled false diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out new file mode 100644 index 0000000000000..cb9e4d736c9a0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out @@ -0,0 +1,460 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 30 + + +-- !query 0 +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 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view years as select * from values + (2012, 1), + (2013, 2) + as years(y, s) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +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 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT * FROM ( + SELECT udf(year), course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR course IN ('dotNET', 'Java') +) +-- !query 3 schema +struct +-- !query 3 output +2012 15000 20000 +2013 48000 30000 + + +-- !query 4 +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (2012, 2013) +) +-- !query 4 schema +struct +-- !query 4 output +Java 20000 30000 +dotNET 15000 48000 + + +-- !query 5 +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)), udf(avg(earnings)) + FOR course IN ('dotNET', 'Java') +) +-- !query 5 schema +struct +-- !query 5 output +2012 15000 7500.0 20000 20000.0 +2013 48000 48000.0 30000 30000.0 + + +-- !query 6 +SELECT * FROM ( + SELECT udf(course) as course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR course IN ('dotNET', 'Java') +) +-- !query 6 schema +struct +-- !query 6 output +63000 50000 + + +-- !query 7 +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(udf(earnings))), udf(min(year)) + FOR course IN ('dotNET', 'Java') +) +-- !query 7 schema +struct +-- !query 7 output +63000 2012 50000 2012 + + +-- !query 8 +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 8 schema +struct +-- !query 8 output +Java 2012 20000 NULL +Java 2013 NULL 30000 +dotNET 2012 15000 NULL +dotNET 2013 NULL 48000 + + +-- !query 9 +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 9 schema +struct +-- !query 9 output +2012 15000 1 20000 1 +2013 48000 2 30000 2 + + +-- !query 10 +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 10 schema +struct +-- !query 10 output +2012 15000 20000 +2013 96000 60000 + + +-- !query 11 +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 11 schema +struct<2012_s:bigint,2013_s:bigint,2012_a:double,2013_a:double,c:string> +-- !query 11 output +15000 48000 7500.0 48000.0 dotNET +20000 30000 20000.0 30000.0 Java + + +-- !query 12 +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 12 schema +struct +-- !query 12 output +15000 48000 7500.0 48000.0 dotNET +20000 30000 20000.0 30000.0 Java + + +-- !query 13 +SELECT * FROM courseSales +PIVOT ( + udf(abs(earnings)) + FOR year IN (2012, 2013) +) +-- !query 13 schema +struct<> +-- !query 13 output +org.apache.spark.sql.AnalysisException +Aggregate expression required for pivot, but 'coursesales.`earnings`' did not appear in any aggregate function.; + + +-- !query 14 +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)), year + FOR course IN ('dotNET', 'Java') +) +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.AnalysisException +Aggregate expression required for pivot, but '__auto_generated_subquery_name.`year`' did not appear in any aggregate function.; + + +-- !query 15 +SELECT * FROM ( + SELECT course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR year IN (2012, 2013) +) +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.AnalysisException +cannot resolve '`year`' given input columns: [__auto_generated_subquery_name.course, __auto_generated_subquery_name.earnings]; line 4 pos 0 + + +-- !query 16 +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 16 schema +struct +-- !query 16 output +2012 15000 7501.0 20000 20001.0 +2013 48000 48001.0 30000 30001.0 + + +-- !query 17 +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(udf(avg(earnings))) + FOR course IN ('dotNET', 'Java') +) +-- !query 17 schema +struct<> +-- !query 17 output +org.apache.spark.sql.AnalysisException +It is not allowed to use an aggregate function in the argument of another aggregate function. Please use the inner aggregate function in a sub-query.; + + +-- !query 18 +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 18 schema +struct +-- !query 18 output +1 15000 NULL +2 NULL 30000 + + +-- !query 19 +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 19 schema +struct +-- !query 19 output +2012 NULL 20000 +2013 48000 NULL + + +-- !query 20 +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 20 schema +struct<> +-- !query 20 output +org.apache.spark.sql.AnalysisException +Invalid pivot value 'dotNET': value data type string does not match pivot column data type struct; + + +-- !query 21 +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (s, 2013) +) +-- !query 21 schema +struct<> +-- !query 21 output +org.apache.spark.sql.AnalysisException +cannot resolve '`s`' given input columns: [coursesales.course, coursesales.earnings, coursesales.year]; line 4 pos 15 + + +-- !query 22 +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (course, 2013) +) +-- !query 22 schema +struct<> +-- !query 22 output +org.apache.spark.sql.AnalysisException +Literal expressions required for pivot values, found 'course#x'; + + +-- !query 23 +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 23 schema +struct +-- !query 23 output +2012 35000 NULL +2013 NULL 78000 + + +-- !query 24 +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 24 schema +struct +-- !query 24 output +2012 15000 NULL +2013 NULL 30000 + + +-- !query 25 +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 25 schema +struct +-- !query 25 output +2012 35000 NULL +2013 NULL 78000 + + +-- !query 26 +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 26 schema +struct +-- !query 26 output +2012 15000 NULL +2013 NULL 30000 + + +-- !query 27 +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 27 schema +struct<> +-- !query 27 output +org.apache.spark.sql.AnalysisException +Invalid pivot column 'm#x'. Pivot columns must be comparable.; + + +-- !query 28 +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 28 schema +struct<> +-- !query 28 output +org.apache.spark.sql.AnalysisException +Invalid pivot column 'named_struct(course, course#x, m, m#x)'. Pivot columns must be comparable.; + + +-- !query 29 +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 29 schema +struct +-- !query 29 output +a z b y c x d w 63000 50000 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out new file mode 100644 index 0000000000000..e1747f466779f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out @@ -0,0 +1,72 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 8 + + +-- !query 0 +-- 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) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE FUNCTION myDoubleAvg AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +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 2 schema +struct +-- !query 2 output +102.5 102.5 102.5 + + +-- !query 3 +SELECT default.myDoubleAvg(udf(int_col1), udf(3)) as my_avg from t1 +-- !query 3 schema +struct<> +-- !query 3 output +org.apache.spark.sql.AnalysisException +Invalid number of arguments for function default.myDoubleAvg. Expected: 1; Found: 2; line 1 pos 7 + + +-- !query 4 +CREATE FUNCTION udaf1 AS 'test.non.existent.udaf' +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +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 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +Can not load class 'test.non.existent.udaf' when registering the function 'default.udaf1', please make sure it is on the classpath; line 1 pos 94 + + +-- !query 6 +DROP FUNCTION myDoubleAvg +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +DROP FUNCTION udaf1 +-- !query 7 schema +struct<> +-- !query 7 output + diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out index d3e1c30222885..9354d5e3117f4 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out @@ -21,10 +21,10 @@ struct<> -- !query 1 -SELECT udf(val), cate, count(val) OVER(PARTITION BY cate ORDER BY val ROWS CURRENT ROW) FROM testData -ORDER BY cate, val +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 1 schema -struct +struct -- !query 1 output NULL NULL 0 3 NULL 1 @@ -38,10 +38,10 @@ NULL a 0 -- !query 2 -SELECT udf(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 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 2 schema -struct +struct -- !query 2 output NULL NULL 3 3 NULL 3 @@ -55,27 +55,20 @@ NULL a 1 -- !query 3 -SELECT val_long, udf(cate), sum(val_long) OVER(PARTITION BY cate ORDER BY val_long -ROWS BETWEEN CURRENT ROW AND CAST(2147483648 AS int) FOLLOWING) FROM testData ORDER BY cate, val_long +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 3 schema -struct +struct<> -- !query 3 output -NULL NULL 1 -1 NULL 1 -1 a 2147483654 -1 a 2147483653 -2 a 2147483652 -2147483650 a 2147483650 -NULL b 2147483653 -3 b 2147483653 -2147483650 b 2147483650 +org.apache.spark.sql.AnalysisException +cannot resolve 'ROWS BETWEEN CURRENT ROW AND 2147483648L FOLLOWING' due to data type mismatch: The data type of the upper bound 'bigint' does not match the expected data type 'int'.; line 1 pos 46 -- !query 4 -SELECT udf(val), cate, count(val) OVER(PARTITION BY cate ORDER BY val RANGE 1 PRECEDING) FROM testData -ORDER BY cate, val +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 4 schema -struct +struct -- !query 4 output NULL NULL 0 3 NULL 1 @@ -89,10 +82,10 @@ NULL a 0 -- !query 5 -SELECT val, udf(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, 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 5 schema -struct +struct -- !query 5 output NULL NULL NULL 3 NULL 3 @@ -106,10 +99,10 @@ NULL a NULL -- !query 6 -SELECT val_long, udf(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_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 6 schema -struct +struct -- !query 6 output NULL NULL NULL 1 NULL 1 @@ -123,10 +116,10 @@ NULL b NULL -- !query 7 -SELECT val_double, udf(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_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 7 schema -struct +struct -- !query 7 output NULL NULL NULL 1.0 NULL 1.0 @@ -140,10 +133,10 @@ NULL NULL NULL -- !query 8 -SELECT val_date, udf(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_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 8 schema -struct +struct -- !query 8 output NULL NULL NULL 2017-08-01 NULL 2017-08-01 @@ -157,11 +150,11 @@ NULL NULL NULL -- !query 9 -SELECT val_timestamp, udf(cate), avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +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 cate, val_timestamp +ORDER BY udf(cate), val_timestamp -- !query 9 schema -struct +struct -- !query 9 output NULL NULL NULL 2017-07-31 17:00:00 NULL 1.5015456E9 @@ -192,48 +185,48 @@ NULL a NULL -- !query 11 -SELECT udf(val), cate, count(val) OVER(PARTITION BY cate -ROWS BETWEEN UNBOUNDED FOLLOWING AND CAST(1 as int) FOLLOWING) FROM testData ORDER BY cate, val +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 11 schema struct<> -- !query 11 output org.apache.spark.sql.AnalysisException -cannot resolve 'ROWS BETWEEN UNBOUNDED FOLLOWING AND CAST(1 AS INT) FOLLOWING' due to data type mismatch: Window frame upper bound 'cast(1 as int)' does not follow the lower bound 'unboundedfollowing$()'.; line 1 pos 38 +cannot resolve 'ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING' due to data type mismatch: Window frame upper bound '1' does not follow the lower bound 'unboundedfollowing$()'.; line 1 pos 38 -- !query 12 -SELECT udf(val), cate, count(val) OVER(PARTITION BY cate +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 12 schema struct<> -- !query 12 output org.apache.spark.sql.AnalysisException -cannot resolve '(PARTITION BY testdata.`cate` RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: A range window frame cannot be used in an unordered window specification.; line 1 pos 38 +cannot resolve '(PARTITION BY CAST(udf(cast(cate as string)) AS STRING) RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: A range window frame cannot be used in an unordered window specification.; line 1 pos 38 -- !query 13 -SELECT udf(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 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 13 schema struct<> -- !query 13 output org.apache.spark.sql.AnalysisException -cannot resolve '(PARTITION BY testdata.`cate` ORDER BY testdata.`val` ASC NULLS FIRST, testdata.`cate` ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: A range window frame with value boundaries cannot be used in a window specification with multiple order by expressions: val#x ASC NULLS FIRST,cate#x ASC NULLS FIRST; line 1 pos 38 +cannot resolve '(PARTITION BY CAST(udf(cast(cate as string)) AS STRING) ORDER BY CAST(udf(cast(val as string)) AS INT) ASC NULLS FIRST, testdata.`cate` ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: A range window frame with value boundaries cannot be used in a window specification with multiple order by expressions: cast(udf(cast(val#x as string)) as int) ASC NULLS FIRST,cate#x ASC NULLS FIRST; line 1 pos 38 -- !query 14 -SELECT udf(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 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 14 schema struct<> -- !query 14 output org.apache.spark.sql.AnalysisException -cannot resolve '(PARTITION BY testdata.`cate` ORDER BY current_timestamp() ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: The data type 'timestamp' used in the order specification does not match the data type 'int' which is used in the range frame.; line 1 pos 38 +cannot resolve '(PARTITION BY CAST(udf(cast(cate as string)) AS STRING) ORDER BY current_timestamp() ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: The data type 'timestamp' used in the order specification does not match the data type 'int' which is used in the range frame.; line 1 pos 38 -- !query 15 -SELECT udf(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 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 15 schema struct<> -- !query 15 output @@ -242,8 +235,8 @@ cannot resolve 'RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING' due to data type mism -- !query 16 -SELECT udf(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 +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 16 schema struct<> -- !query 16 output @@ -252,46 +245,46 @@ org.apache.spark.sql.catalyst.parser.ParseException Frame bound value must be a literal.(line 2, pos 30) == SQL == -SELECT udf(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 +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 17 SELECT udf(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, -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, +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, +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(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 +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 cate ORDER BY val) -ORDER BY cate, val +WINDOW w AS (PARTITION BY udf(cate) ORDER BY udf(val)) +ORDER BY cate, udf(val) -- !query 17 schema struct,collect_set:array,skewness:double,kurtosis:double> -- !query 17 output diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index e005a3e9a258e..e49ef012f5ebb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -725,13 +725,6 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { } test("SPARK-26021: NaN and -0.0 in grouping expressions") { - import java.lang.Float.floatToRawIntBits - import java.lang.Double.doubleToRawLongBits - - // 0.0/0.0 and NaN are different values. - assert(floatToRawIntBits(0.0f/0.0f) != floatToRawIntBits(Float.NaN)) - assert(doubleToRawLongBits(0.0/0.0) != doubleToRawLongBits(Double.NaN)) - checkAnswer( Seq(0.0f, -0.0f, 0.0f/0.0f, Float.NaN).toDF("f").groupBy("f").count(), Row(0.0f, 2) :: Row(Float.NaN, 2) :: Nil) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index e8ddd4e1fd974..8782f95fe517f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -585,14 +585,6 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { checkAnswer(df2, testData.selectExpr("value")) assert(df2.schema.map(_.name) === Seq("value")) } - - // With SQL config caseSensitive ON, AnalysisException should be thrown - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - val e = intercept[AnalysisException] { - testData("KEY") - }.getMessage - assert(e.contains("Cannot resolve column name")) - } } test("drop unknown column (no-op) with column reference") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index f4ba2f0673c0a..a07553008fd78 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -697,13 +697,6 @@ class DataFrameWindowFunctionsSuite extends QueryTest with SharedSQLContext { } test("NaN and -0.0 in window partition keys") { - import java.lang.Float.floatToRawIntBits - import java.lang.Double.doubleToRawLongBits - - // 0.0/0.0 and NaN are different values. - assert(floatToRawIntBits(0.0f/0.0f) != floatToRawIntBits(Float.NaN)) - assert(doubleToRawLongBits(0.0/0.0) != doubleToRawLongBits(Double.NaN)) - val df = Seq( (Float.NaN, Double.NaN), (0.0f/0.0f, 0.0/0.0), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index 89195284a5b5f..51e26d42812ce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -340,7 +340,7 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext with Befo } def validateErrorMessage(msg: String): Unit = { val msg1 = "cannot save interval data type into external storage." - val msg2 = "data source does not support calendarinterval data type." + val msg2 = "data source does not support interval data type." assert(msg.toLowerCase(Locale.ROOT).contains(msg1) || msg.toLowerCase(Locale.ROOT).contains(msg2)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index 24e7564259c83..ae6e9037acd21 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -232,7 +232,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSQLContext { df.select(to_json($"c")).collect() } assert(e.getMessage.contains( - "Unable to convert column a of type calendarinterval to JSON.")) + "Unable to convert column a of type interval to JSON.")) // interval type is invalid for converting to JSON. We can't use it as value type of a map. val df2 = baseDf @@ -240,7 +240,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSQLContext { val e2 = intercept[AnalysisException] { df2.select(to_json($"c")).collect() } - assert(e2.getMessage.contains("Unable to convert column col1 of type calendarinterval to JSON")) + assert(e2.getMessage.contains("Unable to convert column col1 of type interval to JSON")) } test("roundtrip in to_json and from_json - struct") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index 7de5e826f6676..39c87c9eeb47d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -18,8 +18,10 @@ package org.apache.spark.sql.execution import scala.io.Source -import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.{AnalysisException, FastOperator} +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} +import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext @@ -137,5 +139,56 @@ class QueryExecutionSuite extends SharedSQLContext { (_: LogicalPlan) => throw new Error("error")) val error = intercept[Error](qe.toString) assert(error.getMessage.contains("error")) + + spark.experimental.extraStrategies = Nil + } + + test("SPARK-28346: clone the query plan between different stages") { + val tag1 = new TreeNodeTag[String]("a") + val tag2 = new TreeNodeTag[String]("b") + val tag3 = new TreeNodeTag[String]("c") + + def assertNoTag(tag: TreeNodeTag[String], plans: QueryPlan[_]*): Unit = { + plans.foreach { plan => + assert(plan.getTagValue(tag).isEmpty) + } + } + + val df = spark.range(10) + val analyzedPlan = df.queryExecution.analyzed + val cachedPlan = df.queryExecution.withCachedData + val optimizedPlan = df.queryExecution.optimizedPlan + + analyzedPlan.setTagValue(tag1, "v") + assertNoTag(tag1, cachedPlan, optimizedPlan) + + cachedPlan.setTagValue(tag2, "v") + assertNoTag(tag2, analyzedPlan, optimizedPlan) + + optimizedPlan.setTagValue(tag3, "v") + assertNoTag(tag3, analyzedPlan, cachedPlan) + + val tag4 = new TreeNodeTag[String]("d") + try { + spark.experimental.extraStrategies = Seq(new SparkStrategy() { + override def apply(plan: LogicalPlan): Seq[SparkPlan] = { + plan.foreach { + case r: org.apache.spark.sql.catalyst.plans.logical.Range => + r.setTagValue(tag4, "v") + case _ => + } + Seq(FastOperator(plan.output)) + } + }) + // trigger planning + df.queryExecution.sparkPlan + assert(optimizedPlan.getTagValue(tag4).isEmpty) + } finally { + spark.experimental.extraStrategies = Nil + } + + val tag5 = new TreeNodeTag[String]("e") + df.queryExecution.executedPlan.setTagValue(tag5, "v") + assertNoTag(tag5, df.queryExecution.sparkPlan) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala index c4662c8999e42..bafc0337bdc0e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.benchmark import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.internal.SQLConf /** * Benchmark to measure whole stage codegen performance. @@ -130,6 +131,20 @@ object MiscBenchmark extends SqlBasedBenchmark { df.selectExpr("*", "explode(arr) as arr_col") .select("col", "arr_col.*").count } + + withSQLConf(SQLConf.NESTED_PRUNING_ON_EXPRESSIONS.key -> "true") { + codegenBenchmark("generate big nested struct array", M) { + import spark.implicits._ + val df = spark.sparkContext.parallelize(Seq(("1", + Array.fill(M)({ + val i = math.random + (i.toString, (i + 1).toString, (i + 2).toString, (i + 3).toString) + })))).toDF("col", "arr") + .selectExpr("col", "struct(col, arr) as st") + .selectExpr("col", "st.col as col1", "explode(st.arr) as arr_col") + df.collect() + } + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala index e74099202a1df..7b2ccca403bb2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala @@ -187,7 +187,7 @@ class PartitionBatchPruningSuite val result = df.collect().map(_(0)).toArray assert(result.length === 1) - val (readPartitions, readBatches) = df.queryExecution.sparkPlan.collect { + val (readPartitions, readBatches) = df.queryExecution.executedPlan.collect { case in: InMemoryTableScanExec => (in.readPartitions.value, in.readBatches.value) }.head assert(readPartitions === 5) @@ -208,7 +208,7 @@ class PartitionBatchPruningSuite df.collect().map(_(0)).toArray } - val (readPartitions, readBatches) = df.queryExecution.sparkPlan.collect { + val (readPartitions, readBatches) = df.queryExecution.executedPlan.collect { case in: InMemoryTableScanExec => (in.readPartitions.value, in.readBatches.value) }.head diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index fdb50a6dd929c..cf17025980ee3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.csv import java.io.{ByteArrayOutputStream, EOFException, File, FileOutputStream} import java.nio.charset.{Charset, StandardCharsets, UnsupportedCharsetException} -import java.nio.file.Files +import java.nio.file.{Files, StandardOpenOption} import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat import java.util.Locale @@ -28,6 +28,7 @@ import java.util.zip.GZIPOutputStream import scala.collection.JavaConverters._ import scala.util.Properties +import com.univocity.parsers.common.TextParsingException import org.apache.commons.lang3.time.FastDateFormat import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.GzipCodec @@ -2085,4 +2086,27 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te } } } + + test("SPARK-28431: prevent CSV datasource throw TextParsingException with large size message") { + withTempPath { path => + val maxCharsPerCol = 10000 + val str = "a" * (maxCharsPerCol + 1) + + Files.write( + path.toPath, + str.getBytes(StandardCharsets.UTF_8), + StandardOpenOption.CREATE, StandardOpenOption.WRITE + ) + + val errMsg = intercept[TextParsingException] { + spark.read + .option("maxCharsPerColumn", maxCharsPerCol) + .csv(path.getAbsolutePath) + .count() + }.getMessage + + assert(errMsg.contains("..."), + "expect the TextParsingException truncate the error content to be 1000 length.") + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index a47fc18cce9cd..89eaac8e5927c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -857,10 +857,7 @@ class JDBCSuite extends QueryTest Some(ArrayType(DecimalType.SYSTEM_DEFAULT))) assert(Postgres.getJDBCType(FloatType).map(_.databaseTypeDefinition).get == "FLOAT4") assert(Postgres.getJDBCType(DoubleType).map(_.databaseTypeDefinition).get == "FLOAT8") - val errMsg = intercept[IllegalArgumentException] { - Postgres.getJDBCType(ByteType) - } - assert(errMsg.getMessage contains "Unsupported type in postgresql: ByteType") + assert(Postgres.getJDBCType(ByteType).map(_.databaseTypeDefinition).get == "SMALLINT") } test("DerbyDialect jdbc type mapping") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala index c90090aca3d4e..c173bdb953703 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala @@ -24,7 +24,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalog.v2.Identifier -import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG @@ -39,6 +39,8 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn before { spark.conf.set("spark.sql.catalog.testcat", classOf[TestInMemoryTableCatalog].getName) + spark.conf.set( + "spark.sql.catalog.testcat_atomic", classOf[TestStagingInMemoryCatalog].getName) spark.conf.set("spark.sql.catalog.testcat2", classOf[TestInMemoryTableCatalog].getName) spark.conf.set(V2_SESSION_CATALOG.key, classOf[TestInMemoryTableCatalog].getName) @@ -50,6 +52,7 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn after { spark.catalog("testcat").asInstanceOf[TestInMemoryTableCatalog].clearTables() + spark.catalog("testcat_atomic").asInstanceOf[TestInMemoryTableCatalog].clearTables() spark.catalog("session").asInstanceOf[TestInMemoryTableCatalog].clearTables() } @@ -159,20 +162,168 @@ class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAn } test("CreateTableAsSelect: use v2 plan because catalog is set") { + val basicCatalog = spark.catalog("testcat").asTableCatalog + val atomicCatalog = spark.catalog("testcat_atomic").asTableCatalog + val basicIdentifier = "testcat.table_name" + val atomicIdentifier = "testcat_atomic.table_name" + + Seq((basicCatalog, basicIdentifier), (atomicCatalog, atomicIdentifier)).foreach { + case (catalog, identifier) => + spark.sql(s"CREATE TABLE $identifier USING foo AS SELECT id, data FROM source") + + val table = catalog.loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name == identifier) + assert(table.partitioning.isEmpty) + assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.schema == new StructType() + .add("id", LongType, nullable = false) + .add("data", StringType)) + + val rdd = spark.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) + checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), spark.table("source")) + } + } + + test("ReplaceTableAsSelect: basic v2 implementation.") { + val basicCatalog = spark.catalog("testcat").asTableCatalog + val atomicCatalog = spark.catalog("testcat_atomic").asTableCatalog + val basicIdentifier = "testcat.table_name" + val atomicIdentifier = "testcat_atomic.table_name" + + Seq((basicCatalog, basicIdentifier), (atomicCatalog, atomicIdentifier)).foreach { + case (catalog, identifier) => + spark.sql(s"CREATE TABLE $identifier USING foo AS SELECT id, data FROM source") + val originalTable = catalog.loadTable(Identifier.of(Array(), "table_name")) + + spark.sql(s"REPLACE TABLE $identifier USING foo AS SELECT id FROM source") + val replacedTable = catalog.loadTable(Identifier.of(Array(), "table_name")) + + assert(replacedTable != originalTable, "Table should have been replaced.") + assert(replacedTable.name == identifier) + assert(replacedTable.partitioning.isEmpty) + assert(replacedTable.properties == Map("provider" -> "foo").asJava) + assert(replacedTable.schema == new StructType() + .add("id", LongType, nullable = false)) + + val rdd = spark.sparkContext.parallelize(replacedTable.asInstanceOf[InMemoryTable].rows) + checkAnswer( + spark.internalCreateDataFrame(rdd, replacedTable.schema), + spark.table("source").select("id")) + } + } + + test("ReplaceTableAsSelect: Non-atomic catalog drops the table if the write fails.") { spark.sql("CREATE TABLE testcat.table_name USING foo AS SELECT id, data FROM source") + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + assert(table.asInstanceOf[InMemoryTable].rows.nonEmpty) + + intercept[Exception] { + spark.sql("REPLACE TABLE testcat.table_name" + + s" USING foo OPTIONS (`${TestInMemoryTableCatalog.SIMULATE_FAILED_WRITE_OPTION}`=true)" + + s" AS SELECT id FROM source") + } + assert(!testCatalog.tableExists(Identifier.of(Array(), "table_name")), + "Table should have been dropped as a result of the replace.") + } + + test("ReplaceTableAsSelect: Non-atomic catalog drops the table permanently if the" + + " subsequent table creation fails.") { + spark.sql("CREATE TABLE testcat.table_name USING foo AS SELECT id, data FROM source") val testCatalog = spark.catalog("testcat").asTableCatalog val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + assert(table.asInstanceOf[InMemoryTable].rows.nonEmpty) - assert(table.name == "testcat.table_name") - assert(table.partitioning.isEmpty) - assert(table.properties == Map("provider" -> "foo").asJava) - assert(table.schema == new StructType() - .add("id", LongType, nullable = false) - .add("data", StringType)) + intercept[Exception] { + spark.sql("REPLACE TABLE testcat.table_name" + + s" USING foo" + + s" TBLPROPERTIES (`${TestInMemoryTableCatalog.SIMULATE_FAILED_CREATE_PROPERTY}`=true)" + + s" AS SELECT id FROM source") + } - val rdd = spark.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) - checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), spark.table("source")) + assert(!testCatalog.tableExists(Identifier.of(Array(), "table_name")), + "Table should have been dropped and failed to be created.") + } + + test("ReplaceTableAsSelect: Atomic catalog does not drop the table when replace fails.") { + spark.sql("CREATE TABLE testcat_atomic.table_name USING foo AS SELECT id, data FROM source") + val testCatalog = spark.catalog("testcat_atomic").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + + intercept[Exception] { + spark.sql("REPLACE TABLE testcat_atomic.table_name" + + s" USING foo OPTIONS (`${TestInMemoryTableCatalog.SIMULATE_FAILED_WRITE_OPTION}=true)" + + s" AS SELECT id FROM source") + } + + var maybeReplacedTable = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + assert(maybeReplacedTable === table, "Table should not have changed.") + + intercept[Exception] { + spark.sql("REPLACE TABLE testcat_atomic.table_name" + + s" USING foo" + + s" TBLPROPERTIES (`${TestInMemoryTableCatalog.SIMULATE_FAILED_CREATE_PROPERTY}`=true)" + + s" AS SELECT id FROM source") + } + + maybeReplacedTable = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + assert(maybeReplacedTable === table, "Table should not have changed.") + } + + test("ReplaceTable: Erases the table contents and changes the metadata.") { + spark.sql(s"CREATE TABLE testcat.table_name USING $orc2 AS SELECT id, data FROM source") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + assert(table.asInstanceOf[InMemoryTable].rows.nonEmpty) + + spark.sql("REPLACE TABLE testcat.table_name (id bigint) USING foo") + val replaced = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + + assert(replaced.asInstanceOf[InMemoryTable].rows.isEmpty, + "Replaced table should have no rows after committing.") + assert(replaced.schema().fields.length === 1, + "Replaced table should have new schema.") + assert(replaced.schema().fields(0).name === "id", + "Replaced table should have new schema.") + } + + test("ReplaceTableAsSelect: CREATE OR REPLACE new table has same behavior as CTAS.") { + Seq("testcat", "testcat_atomic").foreach { catalog => + spark.sql(s"CREATE TABLE $catalog.created USING $orc2 AS SELECT id, data FROM source") + spark.sql( + s"CREATE OR REPLACE TABLE $catalog.replaced USING $orc2 AS SELECT id, data FROM source") + + val testCatalog = spark.catalog(catalog).asTableCatalog + val createdTable = testCatalog.loadTable(Identifier.of(Array(), "created")) + val replacedTable = testCatalog.loadTable(Identifier.of(Array(), "replaced")) + + assert(createdTable.asInstanceOf[InMemoryTable].rows === + replacedTable.asInstanceOf[InMemoryTable].rows) + assert(createdTable.schema === replacedTable.schema) + } + } + + test("ReplaceTableAsSelect: REPLACE TABLE throws exception if table does not exist.") { + Seq("testcat", "testcat_atomic").foreach { catalog => + spark.sql(s"CREATE TABLE $catalog.created USING $orc2 AS SELECT id, data FROM source") + intercept[CannotReplaceMissingTableException] { + spark.sql(s"REPLACE TABLE $catalog.replaced USING $orc2 AS SELECT id, data FROM source") + } + } + } + + test("ReplaceTableAsSelect: REPLACE TABLE throws exception if table is dropped before commit.") { + import TestInMemoryTableCatalog._ + spark.sql(s"CREATE TABLE testcat_atomic.created USING $orc2 AS SELECT id, data FROM source") + intercept[CannotReplaceMissingTableException] { + spark.sql(s"REPLACE TABLE testcat_atomic.replaced" + + s" USING $orc2" + + s" TBLPROPERTIES (`$SIMULATE_DROP_BEFORE_REPLACE_PROPERTY`=true)" + + s" AS SELECT id, data FROM source") + } } test("CreateTableAsSelect: use v2 plan and session catalog when provider is v2") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala index 380df7a365967..95398082b580d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala @@ -23,11 +23,11 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.sql.catalog.v2.{CatalogV2Implicits, Identifier, TableCatalog, TableChange} +import org.apache.spark.sql.catalog.v2.{CatalogV2Implicits, Identifier, StagingTableCatalog, TableCatalog, TableChange} import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.sources.v2.reader.{Batch, InputPartition, PartitionReader, PartitionReaderFactory, Scan, ScanBuilder} import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriter, DataWriterFactory, SupportsTruncate, WriteBuilder, WriterCommitMessage} import org.apache.spark.sql.types.StructType @@ -38,7 +38,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap class TestInMemoryTableCatalog extends TableCatalog { import CatalogV2Implicits._ - private val tables: util.Map[Identifier, InMemoryTable] = + protected val tables: util.Map[Identifier, InMemoryTable] = new ConcurrentHashMap[Identifier, InMemoryTable]() private var _name: Option[String] = None @@ -66,11 +66,10 @@ class TestInMemoryTableCatalog extends TableCatalog { schema: StructType, partitions: Array[Transform], properties: util.Map[String, String]): Table = { - if (tables.containsKey(ident)) { throw new TableAlreadyExistsException(ident) } - + TestInMemoryTableCatalog.maybeSimulateFailedTableCreation(properties) if (partitions.nonEmpty) { throw new UnsupportedOperationException( s"Catalog $name: Partitioned tables are not supported") @@ -104,7 +103,9 @@ class TestInMemoryTableCatalog extends TableCatalog { } } - override def dropTable(ident: Identifier): Boolean = Option(tables.remove(ident)).isDefined + override def dropTable(ident: Identifier): Boolean = { + Option(tables.remove(ident)).isDefined + } def clearTables(): Unit = { tables.clear() @@ -114,7 +115,7 @@ class TestInMemoryTableCatalog extends TableCatalog { /** * A simple in-memory table. Rows are stored as a buffered group produced by each output task. */ -private class InMemoryTable( +class InMemoryTable( val name: String, val schema: StructType, override val properties: util.Map[String, String]) @@ -155,6 +156,7 @@ private class InMemoryTable( } override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { + TestInMemoryTableCatalog.maybeSimulateFailedTableWrite(options) new WriteBuilder with SupportsTruncate { private var shouldTruncate: Boolean = false @@ -196,7 +198,142 @@ private class InMemoryTable( } } -private class BufferedRows extends WriterCommitMessage with InputPartition with Serializable { +object TestInMemoryTableCatalog { + val SIMULATE_FAILED_WRITE_OPTION = "spark.sql.test.simulateFailedWrite" + val SIMULATE_FAILED_CREATE_PROPERTY = "spark.sql.test.simulateFailedCreate" + val SIMULATE_DROP_BEFORE_REPLACE_PROPERTY = "spark.sql.test.simulateDropBeforeReplace" + + def maybeSimulateFailedTableCreation(tableProperties: util.Map[String, String]): Unit = { + if ("true".equalsIgnoreCase( + tableProperties.get(TestInMemoryTableCatalog.SIMULATE_FAILED_CREATE_PROPERTY))) { + throw new IllegalStateException("Manual create table failure.") + } + } + + def maybeSimulateFailedTableWrite(tableOptions: CaseInsensitiveStringMap): Unit = { + if (tableOptions.getBoolean( + TestInMemoryTableCatalog.SIMULATE_FAILED_WRITE_OPTION, false)) { + throw new IllegalStateException("Manual write to table failure.") + } + } +} + +class TestStagingInMemoryCatalog + extends TestInMemoryTableCatalog with StagingTableCatalog { + import CatalogV2Implicits.IdentifierHelper + import org.apache.spark.sql.sources.v2.TestInMemoryTableCatalog._ + + override def stageCreate( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): StagedTable = { + validateStagedTable(partitions, properties) + new TestStagedCreateTable( + ident, + new InMemoryTable(s"$name.${ident.quoted}", schema, properties)) + } + + override def stageReplace( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): StagedTable = { + validateStagedTable(partitions, properties) + new TestStagedReplaceTable( + ident, + new InMemoryTable(s"$name.${ident.quoted}", schema, properties)) + } + + override def stageCreateOrReplace( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): StagedTable = { + validateStagedTable(partitions, properties) + new TestStagedCreateOrReplaceTable( + ident, + new InMemoryTable(s"$name.${ident.quoted}", schema, properties)) + } + + private def validateStagedTable( + partitions: Array[Transform], + properties: util.Map[String, String]): Unit = { + if (partitions.nonEmpty) { + throw new UnsupportedOperationException( + s"Catalog $name: Partitioned tables are not supported") + } + + maybeSimulateFailedTableCreation(properties) + } + + private abstract class TestStagedTable( + ident: Identifier, + delegateTable: InMemoryTable) + extends StagedTable with SupportsWrite with SupportsRead { + + override def abortStagedChanges(): Unit = {} + + override def name(): String = delegateTable.name + + override def schema(): StructType = delegateTable.schema + + override def capabilities(): util.Set[TableCapability] = delegateTable.capabilities + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { + delegateTable.newWriteBuilder(options) + } + + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + delegateTable.newScanBuilder(options) + } + } + + private class TestStagedCreateTable( + ident: Identifier, + delegateTable: InMemoryTable) extends TestStagedTable(ident, delegateTable) { + + override def commitStagedChanges(): Unit = { + val maybePreCommittedTable = tables.putIfAbsent(ident, delegateTable) + if (maybePreCommittedTable != null) { + throw new TableAlreadyExistsException( + s"Table with identifier $ident and name $name was already created.") + } + } + } + + private class TestStagedReplaceTable( + ident: Identifier, + delegateTable: InMemoryTable) extends TestStagedTable(ident, delegateTable) { + + override def commitStagedChanges(): Unit = { + maybeSimulateDropBeforeCommit() + val maybePreCommittedTable = tables.replace(ident, delegateTable) + if (maybePreCommittedTable == null) { + throw new CannotReplaceMissingTableException(ident) + } + } + + private def maybeSimulateDropBeforeCommit(): Unit = { + if ("true".equalsIgnoreCase( + delegateTable.properties.get(SIMULATE_DROP_BEFORE_REPLACE_PROPERTY))) { + tables.remove(ident) + } + } + } + + private class TestStagedCreateOrReplaceTable( + ident: Identifier, + delegateTable: InMemoryTable) extends TestStagedTable(ident, delegateTable) { + + override def commitStagedChanges(): Unit = { + tables.put(ident, delegateTable) + } + } +} + + +class BufferedRows extends WriterCommitMessage with InputPartition with Serializable { val rows = new mutable.ArrayBuffer[InternalRow]() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala index c6921010a002f..5bd75c850fe76 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala @@ -85,6 +85,7 @@ class ContinuousSuiteBase extends StreamTest { } class ContinuousSuite extends ContinuousSuiteBase { + import IntegratedUDFTestUtils._ import testImplicits._ test("basic") { @@ -252,6 +253,26 @@ class ContinuousSuite extends ContinuousSuiteBase { assert(expected.map(Row(_)).subsetOf(results.toSet), s"Result set ${results.toSet} are not a superset of $expected!") } + + Seq(TestScalaUDF("udf"), TestPythonUDF("udf"), TestScalarPandasUDF("udf")).foreach { udf => + test(s"continuous mode with various UDFs - ${udf.prettyName}") { + assume( + shouldTestScalarPandasUDFs && udf.isInstanceOf[TestScalarPandasUDF] || + shouldTestPythonUDFs && udf.isInstanceOf[TestPythonUDF] || + udf.isInstanceOf[TestScalaUDF]) + + val input = ContinuousMemoryStream[Int] + val df = input.toDF() + + testStream(df.select(udf(df("value")).cast("int")))( + AddData(input, 0, 1, 2), + CheckAnswer(0, 1, 2), + StopStream, + AddData(input, 3, 4, 5), + StartStream(), + CheckAnswer(0, 1, 2, 3, 4, 5)) + } + } } class ContinuousStressSuite extends ContinuousSuiteBase { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala new file mode 100644 index 0000000000000..8f2257f77d2a0 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.thriftserver + +import java.util.UUID + +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType +import org.apache.hive.service.cli._ +import org.apache.hive.service.cli.operation.GetTableTypesOperation +import org.apache.hive.service.cli.session.HiveSession + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.util.{Utils => SparkUtils} + +/** + * Spark's own GetTableTypesOperation + * + * @param sqlContext SQLContext to use + * @param parentSession a HiveSession from SessionManager + */ +private[hive] class SparkGetTableTypesOperation( + sqlContext: SQLContext, + parentSession: HiveSession) + extends GetTableTypesOperation(parentSession) with SparkMetadataOperationUtils with Logging { + + private var statementId: String = _ + + override def close(): Unit = { + super.close() + HiveThriftServer2.listener.onOperationClosed(statementId) + } + + override def runInternal(): Unit = { + statementId = UUID.randomUUID().toString + val logMsg = "Listing table types" + logInfo(s"$logMsg with $statementId") + setState(OperationState.RUNNING) + // Always use the latest class loader provided by executionHive's state. + val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader + Thread.currentThread().setContextClassLoader(executionHiveClassLoader) + + if (isAuthV2Enabled) { + authorizeMetaGets(HiveOperationType.GET_TABLETYPES, null) + } + + HiveThriftServer2.listener.onStatementStart( + statementId, + parentSession.getSessionHandle.getSessionId.toString, + logMsg, + statementId, + parentSession.getUsername) + + try { + val tableTypes = CatalogTableType.tableTypes.map(tableTypeString).toSet + tableTypes.foreach { tableType => + rowSet.addRow(Array[AnyRef](tableType)) + } + setState(OperationState.FINISHED) + } catch { + case e: HiveSQLException => + setState(OperationState.ERROR) + HiveThriftServer2.listener.onStatementError( + statementId, e.getMessage, SparkUtils.exceptionString(e)) + throw e + } + HiveThriftServer2.listener.onStatementFinish(statementId) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala index 952de42083c42..6441dc50f49fe 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala @@ -53,7 +53,7 @@ private[hive] class SparkGetTablesOperation( tableName: String, tableTypes: JList[String]) extends GetTablesOperation(parentSession, catalogName, schemaName, tableName, tableTypes) - with Logging{ + with SparkMetadataOperationUtils with Logging { private var statementId: String = _ @@ -146,11 +146,4 @@ private[hive] class SparkGetTablesOperation( rowSet.addRow(rowData) } } - - private def tableTypeString(tableType: CatalogTableType): String = tableType match { - case EXTERNAL | MANAGED => "TABLE" - case VIEW => "VIEW" - case t => - throw new IllegalArgumentException(s"Unknown table type is found at showCreateHiveTable: $t") - } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationUtils.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationUtils.scala new file mode 100644 index 0000000000000..f4c4b04bada2a --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationUtils.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.hive.thriftserver + +import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.sql.catalyst.catalog.CatalogTableType.{EXTERNAL, MANAGED, VIEW} + +/** + * Utils for metadata operations. + */ +private[hive] trait SparkMetadataOperationUtils { + + def tableTypeString(tableType: CatalogTableType): String = tableType match { + case EXTERNAL | MANAGED => "TABLE" + case VIEW => "VIEW" + case t => + throw new IllegalArgumentException(s"Unknown table type is found: $t") + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index 44b0908502e99..9b4198d7e7a77 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -21,13 +21,13 @@ import java.util.{List => JList, Map => JMap} import java.util.concurrent.ConcurrentHashMap import org.apache.hive.service.cli._ -import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, GetColumnsOperation, GetSchemasOperation, MetadataOperation, Operation, OperationManager} +import org.apache.hive.service.cli.operation._ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.HiveUtils -import org.apache.spark.sql.hive.thriftserver.{ReflectionUtils, SparkExecuteStatementOperation, SparkGetColumnsOperation, SparkGetSchemasOperation, SparkGetTablesOperation} +import org.apache.spark.sql.hive.thriftserver._ import org.apache.spark.sql.internal.SQLConf /** @@ -100,7 +100,7 @@ private[thriftserver] class SparkSQLOperationManager() columnName: String): GetColumnsOperation = synchronized { val sqlContext = sessionToContexts.get(parentSession.getSessionHandle) require(sqlContext != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + - s" initialized or had already closed.") + " initialized or had already closed.") val operation = new SparkGetColumnsOperation(sqlContext, parentSession, catalogName, schemaName, tableName, columnName) handleToOperation.put(operation.getHandle, operation) @@ -108,6 +108,17 @@ private[thriftserver] class SparkSQLOperationManager() operation } + override def newGetTableTypesOperation( + parentSession: HiveSession): GetTableTypesOperation = synchronized { + val sqlContext = sessionToContexts.get(parentSession.getSessionHandle) + require(sqlContext != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + + " initialized or had already closed.") + val operation = new SparkGetTableTypesOperation(sqlContext, parentSession) + handleToOperation.put(operation.getHandle, operation) + logDebug(s"Created GetTableTypesOperation with session=$parentSession.") + operation + } + def setConfMap(conf: SQLConf, confMap: java.util.Map[String, String]): Unit = { val iterator = confMap.entrySet().iterator() while (iterator.hasNext) { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala index e524861b320f6..80a7db5405ca8 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala @@ -166,4 +166,20 @@ class SparkMetadataOperationSuite extends HiveThriftJdbcTest { checkResult(metaData.getColumns(null, "%", "table_not_exist", null), Seq.empty) } } + + test("Spark's own GetTableTypesOperation(SparkGetTableTypesOperation)") { + def checkResult(rs: ResultSet, tableTypes: Seq[String]): Unit = { + for (i <- tableTypes.indices) { + assert(rs.next()) + assert(rs.getString("TABLE_TYPE") === tableTypes(i)) + } + // Make sure there are no more elements + assert(!rs.next()) + } + + withJdbcStatement() { statement => + val metaData = statement.getConnection.getMetaData + checkResult(metaData.getTableTypes, Seq("TABLE", "VIEW")) + } + } } diff --git a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java index 3ae012a72764f..b372f55cedd1c 100644 --- a/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java +++ b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java @@ -39,7 +39,7 @@ public class GetTableTypesOperation extends MetadataOperation { protected static TableSchema RESULT_SET_SCHEMA = new TableSchema() .addStringColumn("TABLE_TYPE", "Table type name."); - private final RowSet rowSet; + protected final RowSet rowSet; private final TableTypeMapping tableTypeMapping; protected GetTableTypesOperation(HiveSession parentSession) { diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java index 875af7150fc16..cf330cbf4100b 100644 --- a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java @@ -39,7 +39,7 @@ public class GetTableTypesOperation extends MetadataOperation { protected static TableSchema RESULT_SET_SCHEMA = new TableSchema() .addStringColumn("TABLE_TYPE", "Table type name."); - private final RowSet rowSet; + protected final RowSet rowSet; private final TableTypeMapping tableTypeMapping; protected GetTableTypesOperation(HiveSession parentSession) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 0ed464dad91b1..ee1734b1f232c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.execution +import java.util.Locale + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.ErrorMsg @@ -186,10 +188,15 @@ case class InsertIntoHiveTable( } val partitionAttributes = partitionColumnNames.takeRight(numDynamicPartitions).map { name => - query.resolve(name :: Nil, sparkSession.sessionState.analyzer.resolver).getOrElse { + val attr = query.resolve(name :: Nil, sparkSession.sessionState.analyzer.resolver).getOrElse { throw new AnalysisException( s"Unable to resolve $name given [${query.output.map(_.name).mkString(", ")}]") }.asInstanceOf[Attribute] + // SPARK-28054: Hive metastore is not case preserving and keeps partition columns + // with lower cased names. Hive will validate the column names in the partition directories + // during `loadDynamicPartitions`. Spark needs to write partition directories with lower-cased + // column names in order to make `loadDynamicPartitions` work. + attr.withName(name.toLowerCase(Locale.ROOT)) } saveAsHiveFile( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala index 62d3bad63bf58..4be3cd45454c6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala @@ -83,16 +83,6 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { jobId = java.util.UUID.randomUUID().toString, outputPath = outputLocation) - // SPARK-28054: Hive metastore is not case preserving and keeps partition columns - // with lower cased names, Hive will validate the column names in partition spec and - // the partition paths. Besides lowercasing the column names in the partition spec, - // we also need to lowercase the column names in written partition paths. - // scalastyle:off caselocale - val hiveCompatiblePartitionColumns = partitionAttributes.map { attr => - attr.withName(attr.name.toLowerCase) - } - // scalastyle:on caselocale - FileFormatWriter.write( sparkSession = sparkSession, plan = plan, @@ -101,7 +91,7 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { outputSpec = FileFormatWriter.OutputSpec(outputLocation, customPartitionLocations, outputColumns), hadoopConf = hadoopConf, - partitionColumns = hiveCompatiblePartitionColumns, + partitionColumns = partitionAttributes, bucketSpec = None, statsTrackers = Seq(basicWriteJobStatsTracker(hadoopConf)), options = Map.empty) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala index 3104fb4d8173c..0ea941c8e0d8e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala @@ -127,7 +127,7 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { spark.udf.register("testType", () => new IntervalData()) sql("select testType()").write.mode("overwrite").orc(orcDir) }.getMessage - assert(msg.contains("ORC data source does not support calendarinterval data type.")) + assert(msg.contains("ORC data source does not support interval data type.")) // read path msg = intercept[AnalysisException] { @@ -135,14 +135,14 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { spark.range(1).write.mode("overwrite").orc(orcDir) spark.read.schema(schema).orc(orcDir).collect() }.getMessage - assert(msg.contains("ORC data source does not support calendarinterval data type.")) + assert(msg.contains("ORC data source does not support interval data type.")) msg = intercept[AnalysisException] { val schema = StructType(StructField("a", new IntervalUDT(), true) :: Nil) spark.range(1).write.mode("overwrite").orc(orcDir) spark.read.schema(schema).orc(orcDir).collect() }.getMessage - assert(msg.contains("ORC data source does not support calendarinterval data type.")) + assert(msg.contains("ORC data source does not support interval data type.")) } }